From 6d75f92142c7bf2c4e531242c4b106b4c0d71221 Mon Sep 17 00:00:00 2001 From: Steve Carlin Date: Sat, 10 Jun 2023 09:07:28 -0700 Subject: [PATCH 001/179] HIVE-27399: Add support for lateral views with CBO (Stephen Carlin reviewed by Krisztian Kasa, Stamatis Zampetakis) Previously, the CBO step was skipped when query contained lateral views. Among the changes that needed to be done: When the HiveTableFunctionScan is created for a lateral view, all of the inputs from the RelNode are mapped into the new "lateral() function, along with the function containing the lateral view. So for example, if the syntax for the SQL were ...LATERAL VIEW explode(field1) on a table with field1 and field2 where field1 is an array, the resulting HiveTableFunctionScan function would be lateral(explode(field1), field1, field2). The ASTConverter needed to be changed to handle conversion of the lateral view back into an AST that contains the LATERAL_VIEW node. The non-"lateral view" HiveTableFunctionScan still generates an AST without the LATERAL_VIEW node which is more optimal. The ASTConverter checks to see if the HiveTableFunctionScan contains the a "LATERAL" function in order to keep the LATERAL_VIEW node. The OpProcFactory code needed to be changed in order to continue to allow a filter optimization. There is pushdown (ppd) logic that was being hit when there was no CBO for lateral views. CBO runs a different path for the ppd logic which is in the OpProcFactory. The lateral view contains 2 node paths that are joined together. The filter can be pushed down the path that contains the base table, but not the path that contains the UDTF. By adding this logic, it prevents a performance regression. A new rule was also needed for CBO. There is a HiveFilter-HiveTableFunctionScan transpose rule. These RelNodes can be transposed when there is at least one AND condition that can be pushed through the HiveTableFunctionScan. A condition can be pushed through if there are no inputRefs that rely on the output of the UDTF function. We cannot use the Calcite FilterTableScanTransposeRule since that rule uses LogicalFilter and LogicalTableFunctionScan which do not exist in the HiveFilter and HiveTableFunctionScan The hive.cbo.fallback.strategy property was set to NEVER in some tests involving lateral views to ensure that CBO runs fine without errors. The q.out changes due to this commit are highlighted below. In some places, an additional "Select Operator" was added. This should not have any performance impact. Small changes (UDFToInteger, IF CASE transformation, etc.,) in plans since lateral view queries are now pass through CBO. In some cases lineage information is lost due to constant folding perfomed in CBO. This is not a regression but something that was always there before, since we don't output lineage for constants (HIVE-27726). Minor column name changes in plans. Minor stat changes most likely cause we pass through CBO. Close apache/hive#4442 --- .../hadoop/hive/ql/optimizer/calcite/Bug.java | 5 + .../ql/optimizer/calcite/HiveCalciteUtil.java | 8 + .../reloperators/HiveTableFunctionScan.java | 23 +- .../HiveFilterTableFunctionTransposeRule.java | 160 +++ ...eOptimizeInlineArrayTableFunctionRule.java | 21 +- .../calcite/translator/ASTConverter.java | 93 +- .../translator/PlanModifierForASTConv.java | 10 + .../hadoop/hive/ql/parse/CalcitePlanner.java | 31 +- .../hive/ql/parse/SemanticAnalyzer.java | 25 + .../ql/parse/relnodegen/LateralViewPlan.java | 39 +- .../hadoop/hive/ql/ppd/OpProcFactory.java | 38 +- .../annotate_stats_lateral_view_join.q | 1 + .../clientpositive/annotate_stats_udtf.q | 2 +- .../clientpositive/cbo_rp_windowing_2.q | 3 + .../queries/clientpositive/lateral_view.q | 3 +- .../queries/clientpositive/lateral_view_cbo.q | 25 + .../queries/clientpositive/lateral_view_cp.q | 1 + .../clientpositive/lateral_view_explode2.q | 1 + .../clientpositive/lateral_view_multi.q | 3 +- .../clientpositive/lateral_view_noalias.q | 3 +- .../clientpositive/lateral_view_onview.q | 3 +- .../clientpositive/lateral_view_onview2.q | 1 + .../queries/clientpositive/lateral_view_ppd.q | 3 +- .../clientpositive/lateral_view_unionall.q | 1 + .../test/queries/clientpositive/lvj_mapjoin.q | 3 +- .../multi_insert_lateral_view.q | 1 + .../reduce_deduplicate_null_keys.q | 2 + .../clientpositive/select_column_pruning.q | 3 +- .../clientpositive/skewjoin_mapjoin6.q | 1 + .../queries/clientpositive/skewjoinopt10.q | 1 + .../test/queries/clientpositive/tablevalues.q | 36 +- .../queries/clientpositive/tez_union_udtf.q | 1 + .../queries/clientpositive/udtf_explode.q | 1 + .../clientpositive/udtf_parse_url_tuple.q | 1 + .../test/queries/clientpositive/udtf_stack.q | 3 +- ql/src/test/queries/clientpositive/union26.q | 1 + .../clientpositive/union_lateralview.q | 1 + .../lateral_view_explain_v0_limit.q.out | 2 +- .../lateral_view_explain_v1_limit.q.out | 2 +- .../lateral_view_explain_v2_limit.q.out | 2 +- .../clientnegative/udf_assert_true2.q.out | 4 +- .../annotate_stats_lateral_view_join.q.out | 168 +-- .../clientpositive/llap/create_view.q.out | 4 +- .../clientpositive/llap/lateral_view.q.out | 6 +- .../llap/lateral_view_cbo.q.out | 113 ++ .../clientpositive/llap/lateral_view_cp.q.out | 38 +- .../llap/lateral_view_multi.q.out | 1250 +++++++++-------- .../llap/lateral_view_noalias.q.out | 2 + .../llap/lateral_view_onview.q.out | 8 + .../llap/lateral_view_onview2.q.out | 2 + .../llap/lateral_view_ppd.q.out | 235 ++-- .../clientpositive/llap/lineage2.q.out | 4 +- .../clientpositive/llap/lvj_mapjoin.q.out | 144 +- .../llap/ppd_field_garbage.q.out | 2 +- .../llap/reduce_deduplicate_null_keys.q.out | 4 +- .../llap/skewjoin_mapjoin6.q.out | 86 +- .../clientpositive/llap/tablevalues.q.out | 126 -- .../results/clientpositive/llap/union26.q.out | 122 +- .../llap/union_lateralview.q.out | 48 +- .../llap/unionall_lateralview.q.out | 2 +- .../results/clientpositive/nonmr_fetch.q.out | 54 +- .../clientpositive/skewjoinopt10.q.out | 76 +- .../clientpositive/tez/explainanalyze_3.q.out | 2 +- .../clientpositive/tez/explainuser_3.q.out | 2 +- .../clientpositive/tez/tez_union_udtf.q.out | 73 +- 65 files changed, 1860 insertions(+), 1279 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterTableFunctionTransposeRule.java create mode 100644 ql/src/test/queries/clientpositive/lateral_view_cbo.q create mode 100644 ql/src/test/results/clientpositive/llap/lateral_view_cbo.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java index 268521ecc22c..8d5c207c6888 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java @@ -88,4 +88,9 @@ public final class Bug { * Whether CALCITE-5669 is fixed. */ public static final boolean CALCITE_5669_FIXED = false; + + /** + * Whether CALCITE-5985 is fixed. + */ + public static final boolean CALCITE_5985_FIXED = false; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java index 723931cc1bf1..c944af8f6b22 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java @@ -1138,6 +1138,14 @@ public static Set getInputRefs(RexNode expr) { return irefColl.getInputRefSet(); } + public static Set getInputRefs(List exprs) { + InputRefsCollector irefColl = new InputRefsCollector(true); + for (RexNode expr : exprs) { + expr.accept(irefColl); + } + return irefColl.getInputRefSet(); + } + private static class InputRefsCollector extends RexVisitorImpl { private final Set inputRefSet = new HashSet(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java index 8ec35d21a19b..881e57720f0c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableFunctionScan.java @@ -28,6 +28,7 @@ import org.apache.calcite.rel.metadata.RelColumnMapping; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexNode; +import org.apache.commons.collections4.CollectionUtils; import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException; public class HiveTableFunctionScan extends TableFunctionScan implements HiveRelNode { @@ -48,7 +49,7 @@ public class HiveTableFunctionScan extends TableFunctionScan implements HiveRelN * @param columnMappings * columnMappings - Column mappings associated with this function */ - private HiveTableFunctionScan(RelOptCluster cluster, RelTraitSet traitSet, List inputs, + protected HiveTableFunctionScan(RelOptCluster cluster, RelTraitSet traitSet, List inputs, RexNode rexCall, Type elementType, RelDataType rowType, Set columnMappings) { super(cluster, traitSet, inputs, rexCall, elementType, rowType, columnMappings); } @@ -56,8 +57,8 @@ private HiveTableFunctionScan(RelOptCluster cluster, RelTraitSet traitSet, List< public static HiveTableFunctionScan create(RelOptCluster cluster, RelTraitSet traitSet, List inputs, RexNode rexCall, Type elementType, RelDataType rowType, Set columnMappings) throws CalciteSemanticException { - return new HiveTableFunctionScan(cluster, traitSet, - inputs, rexCall, elementType, rowType, columnMappings); + return new HiveTableFunctionScan(cluster, traitSet, inputs, rexCall, elementType, rowType, + columnMappings); } @Override @@ -67,4 +68,20 @@ public TableFunctionScan copy(RelTraitSet traitSet, List inputs, RexNod elementType, rowType, columnMappings); } + /** + * Check to see if the inputRef is in the column mappings. + */ + public boolean containsInputRefMapping(int inputRef) { + Set columnMappings = getColumnMappings(); + if (CollectionUtils.isEmpty(columnMappings)) { + return false; + } + + for (RelColumnMapping rcm : columnMappings) { + if (rcm.iInputColumn == inputRef) { + return true; + } + } + return false; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterTableFunctionTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterTableFunctionTransposeRule.java new file mode 100644 index 000000000000..c6e83a8678b6 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterTableFunctionTransposeRule.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.optimizer.calcite.rules; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.metadata.RelColumnMapping; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.optimizer.calcite.Bug; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +/** + * Rule to transpose Filter and TableFunctionScan RelNodes + * + * We cannot use Calcite's FilterTableFunctionTransposeRule because that rule + * uses LogicalFilter and LogicalTableFunctionScan. We should remove this + * class when CALCITE-5985 is fixed (and remove the CALCITE_5985_FIXED entry + * in Bug.java) + */ +public class HiveFilterTableFunctionTransposeRule extends RelOptRule { + + public static final HiveFilterTableFunctionTransposeRule INSTANCE = + new HiveFilterTableFunctionTransposeRule(HiveRelFactories.HIVE_BUILDER); + + public HiveFilterTableFunctionTransposeRule(RelBuilderFactory relBuilderFactory) { + super(operand(HiveFilter.class, operand(HiveTableFunctionScan.class, any())), + relBuilderFactory, null); + } + + @Override + public boolean matches(RelOptRuleCall call) { + if (Bug.CALCITE_5985_FIXED) { + throw new IllegalStateException("Class is redundant after fix is merged into Calcite"); + } + + final Filter filterRel = call.rel(0); + final HiveTableFunctionScan tfs = call.rel(1); + + RexNode condition = filterRel.getCondition(); + if (!HiveCalciteUtil.isDeterministic(condition)) { + return false; + } + + // The TableFunctionScan is always created such that all the input RelNode + // fields are present in its RelNode. If a Filter has an InputRef that is + // greater then the number of the RelNode below the TableFunctionScan, that + // means it was a field created by the TableFunctionScan and thus the Filter + // cannot be pushed through. + // + // We check for each individual conjunction (breaking it up by top level 'and' + // conditions). + for (RexNode ce : RelOptUtil.conjunctions(filterRel.getCondition())) { + if (canBePushed(HiveCalciteUtil.getInputRefs(ce), tfs)) { + return true; + } + } + return false; + } + + public void onMatch(RelOptRuleCall call) { + final Filter filter = call.rel(0); + final HiveTableFunctionScan tfs = call.rel(1); + final RelBuilder builder = call.builder(); + + final List newPartKeyFilterConditions = new ArrayList<>(); + final List unpushedFilterConditions = new ArrayList<>(); + + // Check for each individual 'and' condition so that we can push partial + // expressions through. + for (RexNode ce : RelOptUtil.conjunctions(filter.getCondition())) { + // We can only push if all the InputRef pointers are referencing the + // input RelNode to the TableFunctionScan + if (canBePushed(HiveCalciteUtil.getInputRefs(ce), tfs)) { + newPartKeyFilterConditions.add(ce); + } else { + unpushedFilterConditions.add(ce); + } + } + + // The "matches" check should guarantee there's something to push. + final RexNode filterCondToPushBelowProj = RexUtil.composeConjunction( + filter.getCluster().getRexBuilder(), newPartKeyFilterConditions, true); + + builder.push(tfs.getInput(0)).filter(filterCondToPushBelowProj); + + // If there are conditions that cannot be pushed through, generate the RexNode + final RexNode unpushedFilCondAboveProj = unpushedFilterConditions.isEmpty() + ? null + : RexUtil.composeConjunction(filter.getCluster().getRexBuilder(), + unpushedFilterConditions, true); + + // Generate the new TableFunctionScanNode with the Filter InputRel + final RelNode tableFunctionScanNode = tfs.copy(tfs.getTraitSet(), + ImmutableList.of(builder.build()), tfs.getCall(), tfs.getElementType(), + tfs.getRowType(), tfs.getColumnMappings()); + + builder.clear(); + builder.push(tableFunctionScanNode); + + if (unpushedFilCondAboveProj != null) { + builder.filter(unpushedFilCondAboveProj); + } + + call.transformTo(builder.build()); + } + + // If any of the inputRefs are references to a field that is not mapped into the inputRelNode, + // the condition cannot be pushed. + private boolean canBePushed(Set inputRefs, HiveTableFunctionScan tfs) { + Set columnMappings = tfs.getColumnMappings(); + if (inputRefs.isEmpty()) { + return true; + } + + if (CollectionUtils.isEmpty(columnMappings)) { + return false; + } + + for (Integer inputRef : inputRefs) { + if (!tfs.containsInputRefMapping(inputRef)) { + return false; + } + } + return true; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveOptimizeInlineArrayTableFunctionRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveOptimizeInlineArrayTableFunctionRule.java index e78b5172eddc..2f0ba5d11abd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveOptimizeInlineArrayTableFunctionRule.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveOptimizeInlineArrayTableFunctionRule.java @@ -26,6 +26,7 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.tools.RelBuilderFactory; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; @@ -36,6 +37,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; /** @@ -73,12 +75,17 @@ public boolean matches(RelOptRuleCall call) { } RexCall udtfCall = (RexCall) tableFunctionScanRel.getCall(); - if (!FunctionRegistry.INLINE_FUNC_NAME.equalsIgnoreCase(udtfCall.getOperator().getName())) { + if (udtfCall.getOperator() != SqlStdOperatorTable.LATERAL) { return false; } - Preconditions.checkState(!udtfCall.getOperands().isEmpty()); - RexNode operand = udtfCall.getOperands().get(0); + RexCall inlineCall = (RexCall) udtfCall.getOperands().get(0); + if (!FunctionRegistry.INLINE_FUNC_NAME.equalsIgnoreCase(inlineCall.getOperator().getName())) { + return false; + } + + Preconditions.checkState(!inlineCall.getOperands().isEmpty()); + RexNode operand = inlineCall.getOperands().get(0); if (!(operand instanceof RexCall)) { return false; } @@ -99,7 +106,8 @@ public boolean matches(RelOptRuleCall call) { public void onMatch(RelOptRuleCall call) { final HiveTableFunctionScan tfs = call.rel(0); RelNode inputRel = tfs.getInput(0); - RexCall inlineCall = (RexCall) tfs.getCall(); + RexCall lateralCall = (RexCall) tfs.getCall(); + RexCall inlineCall = (RexCall) lateralCall.getOperands().get(0); RexCall arrayCall = (RexCall) inlineCall.getOperands().get(0); RelOptCluster cluster = tfs.getCluster(); @@ -124,9 +132,12 @@ public void onMatch(RelOptRuleCall call) { RexNode newInlineCall = cluster.getRexBuilder().makeCall(tfs.getRowType(), inlineCall.op, newArrayCall); + // Use empty listfor columnMappings. The return row type of the RelNode now comprises of + // all the fields within the UDTF, so there is no mapping from the output fields + // directly to the input fields anymore. final RelNode newTableFunctionScanNode = tfs.copy(tfs.getTraitSet(), tfs.getInputs(), newInlineCall, tfs.getElementType(), tfs.getRowType(), - tfs.getColumnMappings()); + Collections.emptySet()); call.transformTo(newTableFunctionScanNode); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java index 515e25872785..dec4deddd7d7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -71,9 +72,11 @@ import org.apache.hadoop.hive.ql.QueryProperties; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil; import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveValues; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.jdbc.HiveJdbcConverter; @@ -383,8 +386,10 @@ private ASTNode convert() throws CalciteSemanticException { ASTBuilder sel = ASTBuilder.construct(HiveParser.TOK_SELEXPR, "TOK_SELEXPR"); ASTNode function = buildUDTFAST(call.getOperator().getName(), children); sel.add(function); - for (String alias : udtf.getRowType().getFieldNames()) { - sel.add(HiveParser.Identifier, alias); + + List fields = udtf.getRowType().getFieldNames(); + for (int i = 0; i < udtf.getRowType().getFieldCount(); ++i) { + sel.add(HiveParser.Identifier, fields.get(i)); } b.add(sel); hiveAST.select = b.node(); @@ -408,7 +413,7 @@ private void addRefToBuilder(ASTBuilder b, int i) { b.add(iRef.accept(new RexVisitor(schema, false, root.getCluster().getRexBuilder()))); } - private ASTNode buildUDTFAST(String functionName, List children) { + private static ASTNode buildUDTFAST(String functionName, List children) { ASTNode node = (ASTNode) ParseDriver.adaptor.create(HiveParser.TOK_FUNCTION, "TOK_FUNCTION"); node.addChild((ASTNode) ParseDriver.adaptor.create(HiveParser.Identifier, functionName)); for (ASTNode c : children) { @@ -577,6 +582,17 @@ private QueryBlockInfo convertSource(RelNode r) throws CalciteSemanticException ast = ASTBuilder.subQuery(left, sqAlias); s = new Schema((Union) r, sqAlias); } + } else if (isLateralView(r)) { + TableFunctionScan tfs = ((TableFunctionScan) r); + + // retrieve the base table source. + QueryBlockInfo tableFunctionSource = convertSource(tfs.getInput(0)); + String sqAlias = tableFunctionSource.schema.get(0).table; + // the schema will contain the base table source fields + s = new Schema(tfs, sqAlias); + + ast = createASTLateralView(tfs, s, tableFunctionSource, sqAlias); + } else { ASTConverter src = new ASTConverter(r, this.derivedTableCount, planMapper); ASTNode srcAST = src.convert(); @@ -621,6 +637,77 @@ private ASTNode pkFkHint(int fkTableIndex, boolean nonFkSideIsFiltered) { } } + private static ASTNode createASTLateralView(TableFunctionScan tfs, Schema s, + QueryBlockInfo tableFunctionSource, String sqAlias) { + // The structure of the AST LATERAL VIEW will be: + // + // TOK_LATERAL_VIEW + // TOK_SELECT + // TOK_SELEXPR + // TOK_FUNCTION + // + // ... + // + // TOK_TABALIAS + // + + // set up the select for the parameters of the UDTF + List children = new ArrayList<>(); + // The UDTF function call within the table function scan will be of the form: + // lateral(my_udtf_func(...), $0, $1, ...). For recreating the AST, we need + // the inner "my_udtf_func". + RexCall lateralCall = (RexCall) tfs.getCall(); + RexCall call = (RexCall) lateralCall.getOperands().get(0); + for (RexNode rn : call.getOperands()) { + ASTNode expr = rn.accept(new RexVisitor(s, rn instanceof RexLiteral, + tfs.getCluster().getRexBuilder())); + children.add(expr); + } + ASTNode function = buildUDTFAST(call.getOperator().getName(), children); + + // Add the function to the SELEXPR + ASTBuilder selexpr = ASTBuilder.construct(HiveParser.TOK_SELEXPR, "TOK_SELEXPR"); + selexpr.add(function); + + // Add only the table generated size columns to the select expr for the function, + // skipping over the base table columns from the input side of the join. + int i = 0; + for (ColumnInfo c : s) { + if (i++ < tableFunctionSource.schema.size()) { + continue; + } + selexpr.add(HiveParser.Identifier, c.column); + } + // add the table alias for the lateral view. + ASTBuilder tabAlias = ASTBuilder.construct(HiveParser.TOK_TABALIAS, "TOK_TABALIAS"); + tabAlias.add(HiveParser.Identifier, sqAlias); + + // add the table alias to the SEL_EXPR + selexpr.add(tabAlias.node()); + + // create the SELECT clause + ASTBuilder sel = ASTBuilder.construct(HiveParser.TOK_SELEXPR, "TOK_SELECT"); + sel.add(selexpr.node()); + + // place the SELECT clause under the LATERAL VIEW clause + ASTBuilder lateralview = ASTBuilder.construct(HiveParser.TOK_LATERAL_VIEW, "TOK_LATERAL_VIEW"); + lateralview.add(sel.node()); + + // finally, add the LATERAL VIEW clause under the left side source which is the base table. + lateralview.add(tableFunctionSource.ast); + + return lateralview.node(); + } + + private boolean isLateralView(RelNode relNode) { + if (!(relNode instanceof TableFunctionScan)) { + return false; + } + TableFunctionScan htfs = (TableFunctionScan) relNode; + RexCall call = (RexCall) htfs.getCall(); + return ((RexCall) htfs.getCall()).getOperator() == SqlStdOperatorTable.LATERAL; + } + class QBVisitor extends RelVisitor { public void handle(Filter filter) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java index 61eb3282eb87..7861cc20e951 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java @@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.jdbc.HiveJdbcConverter; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRelColumnsAlignment; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; @@ -171,6 +172,10 @@ private static void convertOpTree(RelNode rel, RelNode parent) { introduceDerivedTable(inputRel, setop); } } + } else if (rel instanceof HiveTableFunctionScan) { + if (!validTableFunctionScanChild((HiveTableFunctionScan)rel)) { + introduceDerivedTable(rel.getInput(0), rel); + } } else if (rel instanceof SingleRel) { if (rel instanceof HiveJdbcConverter) { introduceDerivedTable(rel, parent); @@ -382,6 +387,11 @@ private static boolean validExchangeChild(HiveSortExchange sortNode) { return sortNode.getInput() instanceof Project; } + private static boolean validTableFunctionScanChild(HiveTableFunctionScan htfsNode) { + return htfsNode.getInputs().size() == 1 && + (htfsNode.getInput(0) instanceof Project || htfsNode.getInput(0) instanceof HiveTableScan); + } + private static boolean validSetopParent(RelNode setop, RelNode parent) { boolean validChild = true; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java index 75d52d5edd0f..850f42e5259b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java @@ -142,7 +142,6 @@ import org.apache.hadoop.hive.ql.QueryProperties; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.Operator; @@ -223,6 +222,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSetOpTransposeRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSortPredicates; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSortTransposeRule; +import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterTableFunctionTransposeRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveInBetweenExpandRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveInsertExchange4JoinRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveIntersectMergeRule; @@ -311,9 +311,7 @@ import org.apache.hadoop.hive.ql.plan.mapper.StatsSource; import org.apache.hadoop.hive.ql.reexec.ReCompileException; import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFArray; import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFInline; import org.apache.hadoop.hive.ql.util.DirectionUtils; import org.apache.hadoop.hive.ql.util.NullOrdering; import org.apache.hadoop.hive.serde2.Deserializer; @@ -1024,30 +1022,8 @@ boolean isCBOExecuted() { @Override boolean isCBOSupportedLateralView(ASTNode lateralView) { - // Lateral view AST has the following shape: - // ^(TOK_LATERAL_VIEW - // ^(TOK_SELECT ^(TOK_SELEXPR ^(TOK_FUNCTION Identifier params) identifier* tableAlias))) - if (lateralView.getToken().getType() == HiveParser.TOK_LATERAL_VIEW_OUTER) { - // LATERAL VIEW OUTER not supported in CBO - return false; - } - // Only INLINE followed by ARRAY supported in CBO - ASTNode lvFunc = (ASTNode) lateralView.getChild(0).getChild(0).getChild(0); - String lvFuncName = lvFunc.getChild(0).getText(); - if (lvFuncName.compareToIgnoreCase( - GenericUDTFInline.class.getAnnotation(Description.class).name()) != 0) { - return false; - } - if (lvFunc.getChildCount() != 2) { - return false; - } - ASTNode innerFunc = (ASTNode) lvFunc.getChild(1); - if (innerFunc.getToken().getType() != HiveParser.TOK_FUNCTION || - innerFunc.getChild(0).getText().compareToIgnoreCase( - GenericUDFArray.class.getAnnotation(Description.class).name()) != 0) { - return false; - } - return true; + // LATERAL VIEW OUTER not supported in CBO + return lateralView.getToken().getType() != HiveParser.TOK_LATERAL_VIEW_OUTER; } @Override @@ -1838,6 +1814,7 @@ protected RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataPr } else { rules.add(HiveFilterProjectTransposeRule.DETERMINISTIC); } + rules.add(HiveFilterTableFunctionTransposeRule.INSTANCE); rules.add(HiveOptimizeInlineArrayTableFunctionRule.INSTANCE); rules.add(HiveFilterSetOpTransposeRule.INSTANCE); rules.add(HiveFilterSortTransposeRule.INSTANCE); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 30c24ddc746c..ed8983690312 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -1308,6 +1308,30 @@ private String processSubQuery(QB qb, ASTNode subq) throws SemanticException { return alias; } + private void processLateralViewSelect(ASTNode lateralViewSelect) throws SemanticException { + ASTNode selExprToken = (ASTNode) lateralViewSelect.getChild(0); + if (selExprToken.getToken().getType() != HiveParser.TOK_SELEXPR) { + throw new SemanticException(ASTErrorUtils.getMsg( + ErrorMsg.LATERAL_VIEW_INVALID_CHILD.getMsg(), selExprToken)); + } + for (Object o : selExprToken.getChildren()) { + ASTNode node = (ASTNode) o; + switch (node.getToken().getType()) { + case HiveParser.TOK_FUNCTION: + break; + case HiveParser.Identifier: + unparseTranslator.addIdentifierTranslation(node); + break; + case HiveParser.TOK_TABALIAS: + unparseTranslator.addIdentifierTranslation((ASTNode) node.getChild(0)); + break; + default: + throw new SemanticException(ASTErrorUtils.getMsg( + ErrorMsg.LATERAL_VIEW_INVALID_CHILD.getMsg(), selExprToken)); + } + } + } + /* * Phase1: hold onto any CTE definitions in aliasToCTE. * CTE definitions are global to the Query. @@ -1682,6 +1706,7 @@ private String processLateralView(QB qb, ASTNode lateralView) throw new SemanticException(ASTErrorUtils.getMsg( ErrorMsg.LATERAL_VIEW_INVALID_CHILD.getMsg(), lateralView)); } + processLateralViewSelect((ASTNode) lateralView.getChild(0)); alias = alias.toLowerCase(); qb.getParseInfo().addLateralViewForAlias(alias, lateralView); qb.addAlias(alias); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/relnodegen/LateralViewPlan.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/relnodegen/LateralViewPlan.java index 24421489e704..1e206405eaae 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/relnodegen/LateralViewPlan.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/relnodegen/LateralViewPlan.java @@ -19,13 +19,16 @@ import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.metadata.RelColumnMapping; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan; @@ -102,8 +105,9 @@ public LateralViewPlan(ASTNode lateralView, RelOptCluster cluster, RelNode input this.lateralTableAlias = getTableAliasFromASTNode(selExprAST); - // The RexCall for the udtf function (e.g. inline) - RexCall udtfCall = getUDTFFunction(functionAST, inputRR); + // The RexCall for the lateral function (e.g. lateral(inline(), $0, $1, ...)), where + // the inputrefs are all retrieved from the input RelNode. + RexCall udtfCall = getLateralFunction(functionAST, inputRR, inputRel); // Column aliases provided by the query. List columnAliases = getColumnAliasesFromASTNode(selExprAST, udtfCall); @@ -114,7 +118,7 @@ public LateralViewPlan(ASTNode lateralView, RelOptCluster cluster, RelNode input this.lateralViewRel = HiveTableFunctionScan.create(cluster, TraitsUtil.getDefaultTraitSet(cluster), ImmutableList.of(inputRel), udtfCall, - null, retType, null); + null, retType, createColumnMappings(inputRel)); } public static void validateLateralView(ASTNode lateralView) throws SemanticException { @@ -130,6 +134,18 @@ public static void validateLateralView(ASTNode lateralView) throws SemanticExcep } } + private RexCall getLateralFunction(ASTNode functionAST, RowResolver inputRR, RelNode inputRel) + throws SemanticException { + RexCall udtfCall = getUDTFFunction(functionAST, inputRR); + List operands = new ArrayList<>(); + operands.add(udtfCall); + for (int i = 0; i < inputRel.getRowType().getFieldCount(); ++i) { + RelDataType type = inputRel.getRowType().getFieldList().get(i).getType(); + operands.add(this.cluster.getRexBuilder().makeInputRef(type, i)); + } + return (RexCall) this.cluster.getRexBuilder().makeCall(SqlStdOperatorTable.LATERAL, operands); + } + private RexCall getUDTFFunction(ASTNode functionAST, RowResolver inputRR) throws SemanticException { @@ -246,8 +262,7 @@ private RelDataType getRetType(RelOptCluster cluster, RelNode inputRel, RexNode udtfCall, List columnAliases) { // initialize allDataTypes and allDataTypeNames from the fields in the inputRel - List allDataTypes = new ArrayList<>( - Lists.transform(inputRel.getRowType().getFieldList(), RelDataTypeField::getType)); + List allDataTypes = new ArrayList<>( Lists.transform(inputRel.getRowType().getFieldList(), RelDataTypeField::getType)); List allDataTypeNames = new ArrayList<>( Lists.transform(inputRel.getRowType().getFieldList(), RelDataTypeField::getName)); @@ -256,10 +271,20 @@ private RelDataType getRetType(RelOptCluster cluster, RelNode inputRel, Preconditions.checkState(retType.isStruct()); // Add the type names and values from the udtf into the lists that will make up the - // return type. + // return type. Names need to be unique so add the table prefix allDataTypes.addAll(Lists.transform(retType.getFieldList(), RelDataTypeField::getType)); - allDataTypeNames.addAll(columnAliases); + for (String s : columnAliases) { + allDataTypeNames.add(lateralTableAlias + "." + s); + } return cluster.getTypeFactory().createStructType(allDataTypes, allDataTypeNames); } + + private Set createColumnMappings(RelNode inputRel) { + Set colMappings = new HashSet<>(); + for (int i = 0; i < inputRel.getRowType().getFieldCount(); ++i) { + colMappings.add(new RelColumnMapping(i, 0, i, false)); + } + return colMappings; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java index 0c72d79a3190..fa7f5710f6fd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java @@ -385,6 +385,25 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, } + public static class LateralViewJoinerPPD extends JoinerPPD implements SemanticNodeProcessor { + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + Object o = super.process(nd, stack, procCtx, nodeOutputs); + OpWalkerInfo owi = (OpWalkerInfo) procCtx; + if (HiveConf.getBoolVar(owi.getParseContext().getConf(), + HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + // The lateral view join is allowed to have a filter pushed through it. + // We need to remove the filter candidate here once it has been applied. + // If we do not remove it here, the candidates will be cleared out through + // the getCandidateFilterOps().clear() method in another processor and the + // filter candidate would not be removed. + removeAllCandidates(owi); + } + return o; + } + } + public static class LateralViewForwardPPD extends DefaultPPD implements SemanticNodeProcessor { @Override @@ -504,12 +523,14 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // We try to push the full Filter predicate iff: // - the Filter is on top of a TableScan, or // - the Filter is on top of a PTF (between PTF and Filter, there might be Select operators) + // - the Filter is on top of a LateralViewJoinOperator (the filter can be pushed through one + // side of the join with the base table predicate, but not the UDTF side.) // Otherwise, we push only the synthetic join predicates // Note : pushing Filter on top of PTF is necessary so the LimitPushdownOptimizer for Rank - // functions gets enabled - boolean parentTableScan = filterOp.getParentOperators().get(0) instanceof TableScanOperator; - boolean ancestorPTF = false; - if (!parentTableScan) { + // functions gets enabled. + boolean onlySyntheticJoinPredicate = false; + if (!(filterOp.getParentOperators().get(0) instanceof TableScanOperator)) { + onlySyntheticJoinPredicate = true; Operator parent = filterOp; while (true) { assert parent.getParentOperators().size() == 1; @@ -517,14 +538,17 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, if (parent instanceof SelectOperator) { continue; } else if (parent instanceof PTFOperator) { - ancestorPTF = true; + onlySyntheticJoinPredicate = false; + break; + } else if (parent instanceof LateralViewJoinOperator) { + onlySyntheticJoinPredicate = false; break; } else { break; } } } - return process(nd, stack, procCtx, !parentTableScan && !ancestorPTF, nodeOutputs); + return process(nd, stack, procCtx, onlySyntheticJoinPredicate, nodeOutputs); } } @@ -1409,7 +1433,7 @@ public static SemanticNodeProcessor getUDTFProc() { } public static SemanticNodeProcessor getLVJProc() { - return new JoinerPPD(); + return new LateralViewJoinerPPD(); } public static SemanticNodeProcessor getRSProc() { diff --git a/ql/src/test/queries/clientpositive/annotate_stats_lateral_view_join.q b/ql/src/test/queries/clientpositive/annotate_stats_lateral_view_join.q index 45aca435ee69..03f0251c06c0 100644 --- a/ql/src/test/queries/clientpositive/annotate_stats_lateral_view_join.q +++ b/ql/src/test/queries/clientpositive/annotate_stats_lateral_view_join.q @@ -1,4 +1,5 @@ set hive.fetch.task.conversion=none; +set hive.cbo.fallback.strategy=NEVER; -- setting up a table with multiple rows drop table if exists annotate_stats_lateral_view_join_test; diff --git a/ql/src/test/queries/clientpositive/annotate_stats_udtf.q b/ql/src/test/queries/clientpositive/annotate_stats_udtf.q index 74e6ebf3541f..90468a0e3a87 100644 --- a/ql/src/test/queries/clientpositive/annotate_stats_udtf.q +++ b/ql/src/test/queries/clientpositive/annotate_stats_udtf.q @@ -1,3 +1,4 @@ +set hive.cbo.fallback.strategy=NEVER; -- setting up a table with multiple rows drop table if exists HIVE_20262; create table HIVE_20262 (a array); @@ -17,7 +18,6 @@ explain select explode(a) from HIVE_20262; explain select 1, r from HIVE_20262 lateral view explode(a) t as r ; - -- Default behaviour tests: -- 1 is the default value diff --git a/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q b/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q index 9a1afefea215..a3673cf287b3 100644 --- a/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q +++ b/ql/src/test/queries/clientpositive/cbo_rp_windowing_2.q @@ -234,11 +234,14 @@ set hive.cbo.returnpath.hiveop=true ; select * from mfgr_brand_price_view_n1; -- 24. testLateralViews +-- HIVE-27470: implement returnpath for lateral views +set hive.cbo.returnpath.hiveop=false; select p_mfgr, p_name, lv_col, p_size, sum(p_size) over w1 as s from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p lateral view explode(arr) part_lv as lv_col window w1 as (distribute by p_mfgr sort by p_size, lv_col rows between 2 preceding and current row); +set hive.cbo.returnpath.hiveop=true; -- 25. testMultipleInserts3SWQs CREATE TABLE part_1_n1( diff --git a/ql/src/test/queries/clientpositive/lateral_view.q b/ql/src/test/queries/clientpositive/lateral_view.q index 8454179d90e1..ab05b5c031c1 100644 --- a/ql/src/test/queries/clientpositive/lateral_view.q +++ b/ql/src/test/queries/clientpositive/lateral_view.q @@ -1,5 +1,5 @@ --! qt:dataset:src - +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE tmp_pyang_lv (inputs string) STORED AS RCFILE; @@ -10,6 +10,7 @@ EXPLAIN SELECT myTable.* FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS EXPLAIN SELECT myTable.myCol, myTable2.myCol2 FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array('a', 'b', 'c')) myTable2 AS myCol2 LIMIT 9; EXPLAIN SELECT myTable2.* FROM src LATERAL VIEW explode(array(array(1,2,3))) myTable AS myCol LATERAL VIEW explode(myTable.myCol) myTable2 AS myCol2 LIMIT 3; + -- Verify that * selects columns from both tables SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol SORT BY key ASC, myCol ASC LIMIT 1; -- TABLE.* should be supported diff --git a/ql/src/test/queries/clientpositive/lateral_view_cbo.q b/ql/src/test/queries/clientpositive/lateral_view_cbo.q new file mode 100644 index 000000000000..efd203391fb0 --- /dev/null +++ b/ql/src/test/queries/clientpositive/lateral_view_cbo.q @@ -0,0 +1,25 @@ + +CREATE TABLE simple_table (col string, array_col array); + +EXPLAIN CBO SELECT myTable.myCol FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol; + +EXPLAIN CBO SELECT myTable.myCol, myTable2.myCol2 FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2; + +EXPLAIN CBO SELECT tf.col1, tf.col2, tf.col3 +FROM simple_table + LATERAL TABLE(VALUES('A', 10, simple_table.col),('B', 20, simple_table.col)) AS tf(col1, col2, col3); + +EXPLAIN CBO SELECT myTable.myCol FROM simple_table +LATERAL VIEW explode(simple_table.array_col) myTable AS myCol; + +EXPLAIN CBO SELECT myCol FROM +(SELECT * FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE col='0'; + +EXPLAIN CBO SELECT myCol FROM simple_table +LATERAL VIEW explode(simple_table.array_col) myTable AS myCol where myCol = 1; + diff --git a/ql/src/test/queries/clientpositive/lateral_view_cp.q b/ql/src/test/queries/clientpositive/lateral_view_cp.q index f50bb84f446e..b4b4739f9446 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_cp.q +++ b/ql/src/test/queries/clientpositive/lateral_view_cp.q @@ -1,4 +1,5 @@ --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; --HIVE 3226 drop table array_valued_src; create table array_valued_src (key string, value array); diff --git a/ql/src/test/queries/clientpositive/lateral_view_explode2.q b/ql/src/test/queries/clientpositive/lateral_view_explode2.q index edcfd9fbfd3d..7fadf8930358 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_explode2.q +++ b/ql/src/test/queries/clientpositive/lateral_view_explode2.q @@ -1,4 +1,5 @@ --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar; CREATE TEMPORARY FUNCTION explode2 AS 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFExplode2'; diff --git a/ql/src/test/queries/clientpositive/lateral_view_multi.q b/ql/src/test/queries/clientpositive/lateral_view_multi.q index 08fa5856374f..560b1bd5d915 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_multi.q +++ b/ql/src/test/queries/clientpositive/lateral_view_multi.q @@ -1,3 +1,4 @@ +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE customer(orders array); INSERT INTO customer VALUES (ARRAY('0', '1')); @@ -114,4 +115,4 @@ lateral view explode(orders) v as c47 lateral view explode(orders) v as c48 lateral view explode(orders) v as c49 lateral view explode(orders) v as c50 -LIMIT 1024; \ No newline at end of file +LIMIT 1024; diff --git a/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/ql/src/test/queries/clientpositive/lateral_view_noalias.q index ae80ee34f13f..555dba3135d8 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_noalias.q +++ b/ql/src/test/queries/clientpositive/lateral_view_noalias.q @@ -1,6 +1,7 @@ --! qt:dataset:src --! qt:dataset:part set hive.fetch.task.conversion=more; +set hive.cbo.fallback.strategy=NEVER; --HIVE-2608 Do not require AS a,b,c part in LATERAL VIEW EXPLAIN SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; @@ -13,4 +14,4 @@ SELECT explode(map('key1', 100, 'key2', 200)) from src limit 2; create view lv_noalias as SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; explain select * from lv_noalias a join lv_noalias b on a.key=b.key; -select * from lv_noalias a join lv_noalias b on a.key=b.key; \ No newline at end of file +select * from lv_noalias a join lv_noalias b on a.key=b.key; diff --git a/ql/src/test/queries/clientpositive/lateral_view_onview.q b/ql/src/test/queries/clientpositive/lateral_view_onview.q index 4d23634106e2..76f31b99fcd9 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_onview.q +++ b/ql/src/test/queries/clientpositive/lateral_view_onview.q @@ -1,4 +1,5 @@ --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE lv_table_n0( c1 STRING, c2 ARRAY, c3 INT, c4 CHAR(1)); INSERT OVERWRITE TABLE lv_table_n0 SELECT 'abc ', array(1,2,3), 100, 't' FROM src; @@ -28,4 +29,4 @@ CREATE TABLE lv_table2( c1 STRING, c2 ARRAY); INSERT OVERWRITE TABLE lv_table1_n0 SELECT 'abc ', 100, 't', 'test', 'test', 'test', 'test', 'test', 'test', 'test', 'test', 'test' FROM src; INSERT OVERWRITE TABLE lv_table2 SELECT 'abc ', array(1,2,3) FROM src; EXPLAIN WITH lv_view1 AS (SELECT lv_table1_n0.*, c2 FROM lv_table1_n0 JOIN lv_table2 ON lv_table1_n0.c1 = lv_table2.c1), lv_view2 AS (SELECT * FROM lv_view1 LATERAL VIEW explode(c2) myTable AS myCol) SELECT * FROM lv_view2 SORT BY c1 ASC, myCol ASC LIMIT 1; -WITH lv_view1 AS (SELECT lv_table1_n0.*, c2 FROM lv_table1_n0 JOIN lv_table2 ON lv_table1_n0.c1 = lv_table2.c1), lv_view2 AS (SELECT * FROM lv_view1 LATERAL VIEW explode(c2) myTable AS myCol) SELECT * FROM lv_view2 SORT BY c1 ASC, myCol ASC LIMIT 1; \ No newline at end of file +WITH lv_view1 AS (SELECT lv_table1_n0.*, c2 FROM lv_table1_n0 JOIN lv_table2 ON lv_table1_n0.c1 = lv_table2.c1), lv_view2 AS (SELECT * FROM lv_view1 LATERAL VIEW explode(c2) myTable AS myCol) SELECT * FROM lv_view2 SORT BY c1 ASC, myCol ASC LIMIT 1; diff --git a/ql/src/test/queries/clientpositive/lateral_view_onview2.q b/ql/src/test/queries/clientpositive/lateral_view_onview2.q index 46e2330308f2..27a184083295 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_onview2.q +++ b/ql/src/test/queries/clientpositive/lateral_view_onview2.q @@ -1,4 +1,5 @@ --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE lv_table_n1( c1 STRING, c2 ARRAY, c3 INT, c4 CHAR(1)); INSERT OVERWRITE TABLE lv_table_n1 SELECT 'abc ', array(1,2,3), 100, 't' FROM src; diff --git a/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/ql/src/test/queries/clientpositive/lateral_view_ppd.q index 6b76a7b0c03f..e4ce0937c740 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_ppd.q +++ b/ql/src/test/queries/clientpositive/lateral_view_ppd.q @@ -1,6 +1,7 @@ --! qt:dataset:srcpart --! qt:dataset:src set hive.optimize.ppd=true; +set hive.cbo.fallback.strategy=NEVER; EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0'; SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0'; @@ -16,4 +17,4 @@ SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) m -- HIVE-4293 Predicates following UDTF operator are removed by PPD EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; -SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; \ No newline at end of file +SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; diff --git a/ql/src/test/queries/clientpositive/lateral_view_unionall.q b/ql/src/test/queries/clientpositive/lateral_view_unionall.q index d4591e8fdcd5..7c089ae20518 100644 --- a/ql/src/test/queries/clientpositive/lateral_view_unionall.q +++ b/ql/src/test/queries/clientpositive/lateral_view_unionall.q @@ -1,3 +1,4 @@ +set hive.cbo.fallback.strategy=NEVER; create table tez_test_t1(md_exper string); insert into tez_test_t1 values('tez_test_t1-md_expr'); diff --git a/ql/src/test/queries/clientpositive/lvj_mapjoin.q b/ql/src/test/queries/clientpositive/lvj_mapjoin.q index 808d59adc89f..5f9dbb8c3599 100644 --- a/ql/src/test/queries/clientpositive/lvj_mapjoin.q +++ b/ql/src/test/queries/clientpositive/lvj_mapjoin.q @@ -1,5 +1,6 @@ --! qt:dataset:alltypesorc set hive.explain.user=false; +set hive.cbo.fallback.strategy=NEVER; -- SORT_QUERY_RESULTS drop table sour1; @@ -45,4 +46,4 @@ insert into table tmp_lateral_view from alltypesorc limit 10; select arst.name, arst.age from tmp_lateral_view - lateral view inline(arst) arst; \ No newline at end of file + lateral view inline(arst) arst; diff --git a/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q b/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q index 3442e8246b2c..ed27634c8619 100644 --- a/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q +++ b/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q @@ -1,5 +1,6 @@ --! qt:dataset:src set hive.stats.dbclass=fs; +set hive.cbo.fallback.strategy=NEVER; -- SORT_QUERY_RESULTS create table src_10_n0 as select * from src limit 10; diff --git a/ql/src/test/queries/clientpositive/reduce_deduplicate_null_keys.q b/ql/src/test/queries/clientpositive/reduce_deduplicate_null_keys.q index 035052f2dd3a..8ebf9741b288 100644 --- a/ql/src/test/queries/clientpositive/reduce_deduplicate_null_keys.q +++ b/ql/src/test/queries/clientpositive/reduce_deduplicate_null_keys.q @@ -1,6 +1,7 @@ -- HIVE-24104: NPE due to null key columns in ReduceSink after deduplication -- The query in this test case is not very meaningful but corresponds to a reduced and anonymized version of a query -- used in production. +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE TA(id int); INSERT INTO TA VALUES(10); @@ -22,6 +23,7 @@ FROM TC) SELECT C FROM TD ORDER BY C; + -- Execution fails before HIVE-24104 WITH TC AS diff --git a/ql/src/test/queries/clientpositive/select_column_pruning.q b/ql/src/test/queries/clientpositive/select_column_pruning.q index b7083d872cf1..940a54f98095 100644 --- a/ql/src/test/queries/clientpositive/select_column_pruning.q +++ b/ql/src/test/queries/clientpositive/select_column_pruning.q @@ -1,5 +1,6 @@ --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE lv_table1( c1 STRING, c2 ARRAY, c3 INT, c4 CHAR(1), c5 STRING, c6 STRING, c7 STRING, c8 STRING, c9 STRING, c10 STRING, c11 STRING); INSERT OVERWRITE TABLE lv_table1 SELECT 'abc ', array(1,2,3), 100, 't', 'test', 'test', 'test', 'test', 'test', 'test', 'test' FROM src; EXPLAIN SELECT * FROM lv_table1 LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE c3 = 100 SORT BY c1 ASC, myCol ASC LIMIT 1; -SELECT * FROM lv_table1 LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE c3 = 100 SORT BY c1 ASC, myCol ASC LIMIT 1; \ No newline at end of file +SELECT * FROM lv_table1 LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE c3 = 100 SORT BY c1 ASC, myCol ASC LIMIT 1; diff --git a/ql/src/test/queries/clientpositive/skewjoin_mapjoin6.q b/ql/src/test/queries/clientpositive/skewjoin_mapjoin6.q index 9923c801bc2f..f3e3436c4975 100644 --- a/ql/src/test/queries/clientpositive/skewjoin_mapjoin6.q +++ b/ql/src/test/queries/clientpositive/skewjoin_mapjoin6.q @@ -1,6 +1,7 @@ set hive.mapred.mode=nonstrict; set hive.optimize.skewjoin.compiletime = true; set hive.auto.convert.join=true; +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE T1_n39(key STRING, value STRING) STORED AS TEXTFILE; diff --git a/ql/src/test/queries/clientpositive/skewjoinopt10.q b/ql/src/test/queries/clientpositive/skewjoinopt10.q index 3695d17ec41b..8459bafd4571 100644 --- a/ql/src/test/queries/clientpositive/skewjoinopt10.q +++ b/ql/src/test/queries/clientpositive/skewjoinopt10.q @@ -1,5 +1,6 @@ set hive.mapred.mode=nonstrict; set hive.optimize.skewjoin.compiletime = true; +set hive.cbo.fallback.strategy=NEVER; CREATE TABLE T1_n88(key STRING, value STRING) STORED AS TEXTFILE; diff --git a/ql/src/test/queries/clientpositive/tablevalues.q b/ql/src/test/queries/clientpositive/tablevalues.q index 39f77c3e5fab..910f6454a81f 100644 --- a/ql/src/test/queries/clientpositive/tablevalues.q +++ b/ql/src/test/queries/clientpositive/tablevalues.q @@ -2,6 +2,8 @@ -- VALUES -> array(struct(),struct()) -- TABLE -> LATERAL VIEW INLINE -- SORT_QUERY_RESULTS +set hive.cbo.fallback.strategy=NEVER; + CREATE TABLE mytbl_n1 AS SELECT key, value @@ -25,13 +27,6 @@ FROM LATERAL VIEW INLINE(array(struct('A', 10, t.key),struct('B', 20, t.key))) tf AS col1, col2, col3; -EXPLAIN CBO -SELECT tf.col1, tf.col2, tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t - LATERAL VIEW - INLINE(array(struct('A', 10, t.key),struct('B', 20, t.key))) tf AS col1, col2, col3; - SELECT tf.col1, tf.col2, tf.col3 FROM (SELECT key, value FROM mytbl_n1) t @@ -41,9 +36,6 @@ FROM EXPLAIN SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3); -EXPLAIN CBO -SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3); - SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3); EXPLAIN @@ -100,12 +92,6 @@ FROM (SELECT key, value FROM mytbl_n1) t, LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3); -EXPLAIN CBO -SELECT tf.col1, tf.col2, tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3); - SELECT tf.col1, tf.col2, tf.col3 FROM (SELECT key, value FROM mytbl_n1) t, @@ -117,12 +103,6 @@ FROM (SELECT key, value FROM mytbl_n1) t, LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf; -EXPLAIN CBO -SELECT t.key -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf; - SELECT t.key FROM (SELECT key, value FROM mytbl_n1) t, @@ -134,12 +114,6 @@ FROM (SELECT key, value FROM mytbl_n1) t, LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3); -EXPLAIN CBO -SELECT tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3); - SELECT tf.col3 FROM (SELECT key, value FROM mytbl_n1) t, @@ -151,12 +125,6 @@ FROM (SELECT row_number() over (order by key desc) as r FROM mytbl_n1) t, LATERAL TABLE(VALUES('A', 10, t.r),('B', 20, t.r)) AS tf(col1, col2, col3); -EXPLAIN CBO -SELECT tf.col3 -FROM - (SELECT row_number() over (order by key desc) as r FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.r),('B', 20, t.r)) AS tf(col1, col2, col3); - SELECT tf.col3 FROM (SELECT row_number() over (order by key desc) as r FROM mytbl_n1) t, diff --git a/ql/src/test/queries/clientpositive/tez_union_udtf.q b/ql/src/test/queries/clientpositive/tez_union_udtf.q index ed58cfd55083..bd7f020da662 100644 --- a/ql/src/test/queries/clientpositive/tez_union_udtf.q +++ b/ql/src/test/queries/clientpositive/tez_union_udtf.q @@ -1,6 +1,7 @@ --! qt:dataset:src1 --! qt:dataset:src set hive.merge.tezfiles=true; +set hive.cbo.fallback.strategy=NEVER; -- SORT_BEFORE_DIFF EXPLAIN diff --git a/ql/src/test/queries/clientpositive/udtf_explode.q b/ql/src/test/queries/clientpositive/udtf_explode.q index 39d429d59116..358fb9cc0bb3 100644 --- a/ql/src/test/queries/clientpositive/udtf_explode.q +++ b/ql/src/test/queries/clientpositive/udtf_explode.q @@ -1,6 +1,7 @@ --! qt:dataset:src set hive.mapred.mode=nonstrict; set hive.fetch.task.conversion=more; +set hive.cbo.fallback.strategy=NEVER; DESCRIBE FUNCTION explode; DESCRIBE FUNCTION EXTENDED explode; diff --git a/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q index 66aab527789a..9aa51220b34d 100644 --- a/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q +++ b/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q @@ -2,6 +2,7 @@ -- SORT_QUERY_RESULTS set hive.mapred.mode=nonstrict; +set hive.cbo.fallback.strategy=NEVER; create table url_t (key string, fullurl string); insert overwrite table url_t diff --git a/ql/src/test/queries/clientpositive/udtf_stack.q b/ql/src/test/queries/clientpositive/udtf_stack.q index ba2b6e354fb3..8aa7a8cfe80e 100644 --- a/ql/src/test/queries/clientpositive/udtf_stack.q +++ b/ql/src/test/queries/clientpositive/udtf_stack.q @@ -1,4 +1,5 @@ --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; DESCRIBE FUNCTION stack; EXPLAIN SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z') a AS x, y LIMIT 2; @@ -9,4 +10,4 @@ SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z', array(4)) a AS x, EXPLAIN SELECT stack(1, "en", "dbpedia", NULL ); -SELECT stack(1, "en", "dbpedia", NULL ); \ No newline at end of file +SELECT stack(1, "en", "dbpedia", NULL ); diff --git a/ql/src/test/queries/clientpositive/union26.q b/ql/src/test/queries/clientpositive/union26.q index 1a236d61771b..4fb30db8e45b 100644 --- a/ql/src/test/queries/clientpositive/union26.q +++ b/ql/src/test/queries/clientpositive/union26.q @@ -1,5 +1,6 @@ --! qt:dataset:srcpart -- SORT_QUERY_RESULTS +set hive.cbo.fallback.strategy=NEVER; EXPLAIN SELECT diff --git a/ql/src/test/queries/clientpositive/union_lateralview.q b/ql/src/test/queries/clientpositive/union_lateralview.q index 68b6f380af5e..63d7b84f3101 100644 --- a/ql/src/test/queries/clientpositive/union_lateralview.q +++ b/ql/src/test/queries/clientpositive/union_lateralview.q @@ -1,6 +1,7 @@ --! qt:dataset:srcpart --! qt:dataset:src +set hive.cbo.fallback.strategy=NEVER; create table test_union_lateral_view(key int, arr_ele int, value string); EXPLAIN diff --git a/ql/src/test/results/clientnegative/lateral_view_explain_v0_limit.q.out b/ql/src/test/results/clientnegative/lateral_view_explain_v0_limit.q.out index 31ca06e361d9..16471a592eae 100644 --- a/ql/src/test/results/clientnegative/lateral_view_explain_v0_limit.q.out +++ b/ql/src/test/results/clientnegative/lateral_view_explain_v0_limit.q.out @@ -20,4 +20,4 @@ lateral view explode(orders) v as c9 PREHOOK: type: QUERY PREHOOK: Input: default@customer #### A masked pattern was here #### -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExplainTask. LVJ[45] reached hive.explain.node.visit.limit(256) +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExplainTask. LVJ[53] reached hive.explain.node.visit.limit(256) diff --git a/ql/src/test/results/clientnegative/lateral_view_explain_v1_limit.q.out b/ql/src/test/results/clientnegative/lateral_view_explain_v1_limit.q.out index 31ca06e361d9..16471a592eae 100644 --- a/ql/src/test/results/clientnegative/lateral_view_explain_v1_limit.q.out +++ b/ql/src/test/results/clientnegative/lateral_view_explain_v1_limit.q.out @@ -20,4 +20,4 @@ lateral view explode(orders) v as c9 PREHOOK: type: QUERY PREHOOK: Input: default@customer #### A masked pattern was here #### -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExplainTask. LVJ[45] reached hive.explain.node.visit.limit(256) +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExplainTask. LVJ[53] reached hive.explain.node.visit.limit(256) diff --git a/ql/src/test/results/clientnegative/lateral_view_explain_v2_limit.q.out b/ql/src/test/results/clientnegative/lateral_view_explain_v2_limit.q.out index 4f6e2fe78994..47d85268c5d0 100644 --- a/ql/src/test/results/clientnegative/lateral_view_explain_v2_limit.q.out +++ b/ql/src/test/results/clientnegative/lateral_view_explain_v2_limit.q.out @@ -20,4 +20,4 @@ lateral view explode(orders) v as c9 PREHOOK: type: QUERY PREHOOK: Input: default@customer #### A masked pattern was here #### -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExplainTask. LVJ[45] reached hive.explain.node.visit.limit(256) +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExplainTask. LVJ[53] reached hive.explain.node.visit.limit(256) diff --git a/ql/src/test/results/clientnegative/udf_assert_true2.q.out b/ql/src/test/results/clientnegative/udf_assert_true2.q.out index 42f6c9959bfc..beff2a6a30ad 100644 --- a/ql/src/test/results/clientnegative/udf_assert_true2.q.out +++ b/ql/src/test/results/clientnegative/udf_assert_true2.q.out @@ -23,7 +23,7 @@ STAGE PLANS: Limit Number of rows: 2 Select Operator - expressions: (1 + assert_true((_col6 < 2))) (type: int) + expressions: (1 + UDFToInteger(assert_true((_col6 < 2)))) (type: int) outputColumnNames: _col0 ListSink Select Operator @@ -36,7 +36,7 @@ STAGE PLANS: Limit Number of rows: 2 Select Operator - expressions: (1 + assert_true((_col6 < 2))) (type: int) + expressions: (1 + UDFToInteger(assert_true((_col6 < 2)))) (type: int) outputColumnNames: _col0 ListSink diff --git a/ql/src/test/results/clientpositive/llap/annotate_stats_lateral_view_join.q.out b/ql/src/test/results/clientpositive/llap/annotate_stats_lateral_view_join.q.out index c63c06121c38..f8b3838ce51d 100644 --- a/ql/src/test/results/clientpositive/llap/annotate_stats_lateral_view_join.q.out +++ b/ql/src/test/results/clientpositive/llap/annotate_stats_lateral_view_join.q.out @@ -80,10 +80,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 30 Data size: 4920 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 30 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 30 Data size: 4920 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 30 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -101,10 +101,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 30 Data size: 4920 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 30 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 30 Data size: 4920 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 30 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -233,10 +233,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -254,10 +254,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -284,10 +284,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -305,10 +305,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 300 Data size: 46800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 300 Data size: 45600 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -374,10 +374,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -395,10 +395,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -425,10 +425,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -446,10 +446,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -485,10 +485,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -506,10 +506,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -536,10 +536,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -557,10 +557,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3000 Data size: 600000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -608,10 +608,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -629,10 +629,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -761,10 +761,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -782,10 +782,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -812,10 +812,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -833,10 +833,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 132 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -902,10 +902,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -923,10 +923,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -953,10 +953,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -974,10 +974,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1013,10 +1013,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1034,10 +1034,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1064,10 +1064,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1085,10 +1085,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1136,10 +1136,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 3 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 324 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 324 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1157,10 +1157,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 3 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 324 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 324 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1289,10 +1289,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1310,10 +1310,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1340,10 +1340,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1361,10 +1361,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 468 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1430,10 +1430,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1451,10 +1451,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1481,10 +1481,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1502,10 +1502,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1541,10 +1541,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1562,10 +1562,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1592,10 +1592,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -1613,10 +1613,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 456 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git a/ql/src/test/results/clientpositive/llap/create_view.q.out b/ql/src/test/results/clientpositive/llap/create_view.q.out index 395cc5c273b0..1338152a9d77 100644 --- a/ql/src/test/results/clientpositive/llap/create_view.q.out +++ b/ql/src/test/results/clientpositive/llap/create_view.q.out @@ -1076,9 +1076,9 @@ POSTHOOK: type: CREATEVIEW POSTHOOK: Input: default@src POSTHOOK: Output: database:default POSTHOOK: Output: default@view12 -POSTHOOK: Lineage: view12.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: view12.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: view12.mycol SCRIPT [] -POSTHOOK: Lineage: view12.value EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: view12.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: DESCRIBE EXTENDED view12 PREHOOK: type: DESCTABLE PREHOOK: Input: default@view12 diff --git a/ql/src/test/results/clientpositive/llap/lateral_view.q.out b/ql/src/test/results/clientpositive/llap/lateral_view.q.out index e603fb54d2e7..7ad9255515c1 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view.q.out @@ -673,11 +673,11 @@ STAGE PLANS: alias: tmp_pyang_src_rcfile Statistics: Num rows: 20 Data size: 42080 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: array) - outputColumnNames: key, value + expressions: value[0] (type: string), key (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 20 Data size: 42080 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: value[0] (type: string), key (type: string) + keys: _col0 (type: string), _col1 (type: string) minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_cbo.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_cbo.q.out new file mode 100644 index 000000000000..4584ef06b90a --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/lateral_view_cbo.q.out @@ -0,0 +1,113 @@ +PREHOOK: query: CREATE TABLE simple_table (col string, array_col array) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@simple_table +POSTHOOK: query: CREATE TABLE simple_table (col string, array_col array) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@simple_table +PREHOOK: query: EXPLAIN CBO SELECT myTable.myCol FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +PREHOOK: type: QUERY +PREHOOK: Input: default@simple_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT myTable.myCol FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +POSTHOOK: type: QUERY +POSTHOOK: Input: default@simple_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(mycol=[$6]) + HiveTableFunctionScan(invocation=[LATERAL(explode(ARRAY(1, 2, 3)), $0, $1, $2, $3, $4, $5)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, INTEGER mytable.mycol)]) + HiveTableScan(table=[[default, simple_table]], table:alias=[simple_table]) + +PREHOOK: query: EXPLAIN CBO SELECT myTable.myCol, myTable2.myCol2 FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2 +PREHOOK: type: QUERY +PREHOOK: Input: default@simple_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT myTable.myCol, myTable2.myCol2 FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@simple_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(mycol=[$6], mycol2=[$7]) + HiveTableFunctionScan(invocation=[LATERAL(explode(ARRAY(1, 2, 3)), $0, $1, $2, $3, $4, $5, $6)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, INTEGER mytable.mycol, INTEGER mytable2.mycol2)]) + HiveProject(col=[$0], array_col=[$1], BLOCK__OFFSET__INSIDE__FILE=[$2], INPUT__FILE__NAME=[$3], ROW__ID=[$4], ROW__IS__DELETED=[$5], mytable.mycol=[$6]) + HiveTableFunctionScan(invocation=[LATERAL(explode(ARRAY(1, 2, 3)), $0, $1, $2, $3, $4, $5)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, INTEGER mytable.mycol)]) + HiveTableScan(table=[[default, simple_table]], table:alias=[simple_table]) + +PREHOOK: query: EXPLAIN CBO SELECT tf.col1, tf.col2, tf.col3 +FROM simple_table + LATERAL TABLE(VALUES('A', 10, simple_table.col),('B', 20, simple_table.col)) AS tf(col1, col2, col3) +PREHOOK: type: QUERY +PREHOOK: Input: default@simple_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT tf.col1, tf.col2, tf.col3 +FROM simple_table + LATERAL TABLE(VALUES('A', 10, simple_table.col),('B', 20, simple_table.col)) AS tf(col1, col2, col3) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@simple_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(col1=[$6], col2=[$7], col3=[$8]) + HiveTableFunctionScan(invocation=[inline(ARRAY(ROW($0, $1, $2, $3, $4, $5, _UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, $0), ROW($0, $1, $2, $3, $4, $5, _UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, $0)))], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, VARCHAR(2147483647) tf.col1, INTEGER tf.col2, VARCHAR(2147483647) tf.col3)]) + HiveTableScan(table=[[default, simple_table]], table:alias=[simple_table]) + +PREHOOK: query: EXPLAIN CBO SELECT myTable.myCol FROM simple_table +LATERAL VIEW explode(simple_table.array_col) myTable AS myCol +PREHOOK: type: QUERY +PREHOOK: Input: default@simple_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT myTable.myCol FROM simple_table +LATERAL VIEW explode(simple_table.array_col) myTable AS myCol +POSTHOOK: type: QUERY +POSTHOOK: Input: default@simple_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(mycol=[$6]) + HiveTableFunctionScan(invocation=[LATERAL(explode($1), $0, $1, $2, $3, $4, $5)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, VARCHAR(2147483647) mytable.mycol)]) + HiveTableScan(table=[[default, simple_table]], table:alias=[simple_table]) + +PREHOOK: query: EXPLAIN CBO SELECT myCol FROM +(SELECT * FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE col='0' +PREHOOK: type: QUERY +PREHOOK: Input: default@simple_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT myCol FROM +(SELECT * FROM simple_table +LATERAL VIEW explode(array(1,2,3)) myTable AS myCol +LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE col='0' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@simple_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(mycol=[$6]) + HiveTableFunctionScan(invocation=[LATERAL(explode(ARRAY(1, 2, 3)), $0, $1, $2, $3, $4, $5, $6)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, INTEGER mytable.mycol, INTEGER mytable2.mycol2)]) + HiveProject(col=[$0], array_col=[$1], BLOCK__OFFSET__INSIDE__FILE=[$2], INPUT__FILE__NAME=[$3], ROW__ID=[$4], ROW__IS__DELETED=[$5], mytable.mycol=[$6]) + HiveTableFunctionScan(invocation=[LATERAL(explode(ARRAY(1, 2, 3)), $0, $1, $2, $3, $4, $5)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, INTEGER mytable.mycol)]) + HiveProject(col=[$0], array_col=[$1], BLOCK__OFFSET__INSIDE__FILE=[$2], INPUT__FILE__NAME=[$3], ROW__ID=[$4], ROW__IS__DELETED=[$5]) + HiveFilter(condition=[=($0, _UTF-16LE'0')]) + HiveTableScan(table=[[default, simple_table]], table:alias=[simple_table]) + +PREHOOK: query: EXPLAIN CBO SELECT myCol FROM simple_table +LATERAL VIEW explode(simple_table.array_col) myTable AS myCol where myCol = 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@simple_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT myCol FROM simple_table +LATERAL VIEW explode(simple_table.array_col) myTable AS myCol where myCol = 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@simple_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(mycol=[$6]) + HiveFilter(condition=[=(CAST($6):DOUBLE, 1)]) + HiveTableFunctionScan(invocation=[LATERAL(explode($1), $0, $1, $2, $3, $4, $5)], rowType=[RecordType(VARCHAR(2147483647) col, VARCHAR(2147483647) ARRAY array_col, BIGINT BLOCK__OFFSET__INSIDE__FILE, VARCHAR(2147483647) INPUT__FILE__NAME, RecordType(BIGINT writeid, INTEGER bucketid, BIGINT rowid) ROW__ID, BOOLEAN ROW__IS__DELETED, VARCHAR(2147483647) mytable.mycol)]) + HiveTableScan(table=[[default, simple_table]], table:alias=[simple_table]) + diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_cp.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_cp.q.out index 6ce945526a39..a626da0e7940 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view_cp.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view_cp.q.out @@ -54,12 +54,16 @@ STAGE PLANS: Filter Operator predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs Map 4 @@ -71,13 +75,17 @@ STAGE PLANS: Filter Operator predicate: key is not null (type: boolean) Statistics: Num rows: 475 Data size: 951428 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string), value (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 475 Data size: 951428 Basic stats: COMPLETE Column stats: NONE - value expressions: value (type: array) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 475 Data size: 951428 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: array) Execution mode: vectorized, llap LLAP IO: all inputs Reducer 2 @@ -87,12 +95,12 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 key (type: string) - 1 key (type: string) - outputColumnNames: _col7 + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col2 Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col7 (type: array) + expressions: _col2 (type: array) outputColumnNames: _col1 Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE Lateral View Forward diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_multi.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_multi.q.out index f06834a0ad49..9bc38cba2ff9 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view_multi.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view_multi.q.out @@ -126,7 +126,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@customer #### A masked pattern was here #### LOGICAL PLAN: -customer +$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:$hdt$_0:customer TableScan (TS_0) alias: customer Lateral View Forward (LVF_1) @@ -135,646 +135,790 @@ customer outputColumnNames: orders Lateral View Join Operator (LVJ_5) outputColumnNames: _col0, _col5 - Lateral View Forward (LVF_6) - Select Operator (SEL_7) + Lateral View Forward (LVF_7) + Select Operator (SEL_8) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_10) + Lateral View Join Operator (LVJ_11) outputColumnNames: _col0, _col5, _col6 - Lateral View Forward (LVF_11) - Select Operator (SEL_12) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_15) - outputColumnNames: _col0, _col5, _col7 - Lateral View Forward (LVF_16) - Select Operator (SEL_17) + Select Operator (SEL_12) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_13) + Select Operator (SEL_14) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_17) + outputColumnNames: _col0, _col5, _col7 + Select Operator (SEL_18) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_20) - outputColumnNames: _col0, _col5, _col8 - Lateral View Forward (LVF_21) - Select Operator (SEL_22) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_25) - outputColumnNames: _col0, _col5, _col9 - Lateral View Forward (LVF_26) - Select Operator (SEL_27) + Lateral View Forward (LVF_19) + Select Operator (SEL_20) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_23) + outputColumnNames: _col0, _col5, _col8 + Select Operator (SEL_24) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_25) + Select Operator (SEL_26) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_30) - outputColumnNames: _col0, _col5, _col10 - Lateral View Forward (LVF_31) - Select Operator (SEL_32) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_35) - outputColumnNames: _col0, _col5, _col11 - Lateral View Forward (LVF_36) - Select Operator (SEL_37) + Lateral View Join Operator (LVJ_29) + outputColumnNames: _col0, _col5, _col9 + Select Operator (SEL_30) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_31) + Select Operator (SEL_32) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_35) + outputColumnNames: _col0, _col5, _col10 + Select Operator (SEL_36) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_40) - outputColumnNames: _col0, _col5, _col12 - Lateral View Forward (LVF_41) - Select Operator (SEL_42) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_45) - outputColumnNames: _col0, _col5, _col13 - Lateral View Forward (LVF_46) - Select Operator (SEL_47) + Lateral View Forward (LVF_37) + Select Operator (SEL_38) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_41) + outputColumnNames: _col0, _col5, _col11 + Select Operator (SEL_42) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_43) + Select Operator (SEL_44) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_50) - outputColumnNames: _col0, _col5, _col14 - Lateral View Forward (LVF_51) - Select Operator (SEL_52) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_55) - outputColumnNames: _col0, _col5, _col15 - Lateral View Forward (LVF_56) - Select Operator (SEL_57) + Lateral View Join Operator (LVJ_47) + outputColumnNames: _col0, _col5, _col12 + Select Operator (SEL_48) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_49) + Select Operator (SEL_50) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_53) + outputColumnNames: _col0, _col5, _col13 + Select Operator (SEL_54) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_60) - outputColumnNames: _col0, _col5, _col16 - Lateral View Forward (LVF_61) - Select Operator (SEL_62) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_65) - outputColumnNames: _col0, _col5, _col17 - Lateral View Forward (LVF_66) - Select Operator (SEL_67) + Lateral View Forward (LVF_55) + Select Operator (SEL_56) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_59) + outputColumnNames: _col0, _col5, _col14 + Select Operator (SEL_60) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_61) + Select Operator (SEL_62) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_70) - outputColumnNames: _col0, _col5, _col18 - Lateral View Forward (LVF_71) - Select Operator (SEL_72) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_75) - outputColumnNames: _col0, _col5, _col19 - Lateral View Forward (LVF_76) - Select Operator (SEL_77) + Lateral View Join Operator (LVJ_65) + outputColumnNames: _col0, _col5, _col15 + Select Operator (SEL_66) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_67) + Select Operator (SEL_68) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_71) + outputColumnNames: _col0, _col5, _col16 + Select Operator (SEL_72) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_80) - outputColumnNames: _col0, _col5, _col20 - Lateral View Forward (LVF_81) - Select Operator (SEL_82) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_85) - outputColumnNames: _col0, _col5, _col21 - Lateral View Forward (LVF_86) - Select Operator (SEL_87) + Lateral View Forward (LVF_73) + Select Operator (SEL_74) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_77) + outputColumnNames: _col0, _col5, _col17 + Select Operator (SEL_78) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_79) + Select Operator (SEL_80) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_90) - outputColumnNames: _col0, _col5, _col22 - Lateral View Forward (LVF_91) - Select Operator (SEL_92) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_95) - outputColumnNames: _col0, _col5, _col23 - Lateral View Forward (LVF_96) - Select Operator (SEL_97) + Lateral View Join Operator (LVJ_83) + outputColumnNames: _col0, _col5, _col18 + Select Operator (SEL_84) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_85) + Select Operator (SEL_86) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_89) + outputColumnNames: _col0, _col5, _col19 + Select Operator (SEL_90) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_100) - outputColumnNames: _col0, _col5, _col24 - Lateral View Forward (LVF_101) - Select Operator (SEL_102) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_105) - outputColumnNames: _col0, _col5, _col25 - Lateral View Forward (LVF_106) - Select Operator (SEL_107) + Lateral View Forward (LVF_91) + Select Operator (SEL_92) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_95) + outputColumnNames: _col0, _col5, _col20 + Select Operator (SEL_96) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_97) + Select Operator (SEL_98) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_110) - outputColumnNames: _col0, _col5, _col26 - Lateral View Forward (LVF_111) - Select Operator (SEL_112) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_115) - outputColumnNames: _col0, _col5, _col27 - Lateral View Forward (LVF_116) - Select Operator (SEL_117) + Lateral View Join Operator (LVJ_101) + outputColumnNames: _col0, _col5, _col21 + Select Operator (SEL_102) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_103) + Select Operator (SEL_104) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_107) + outputColumnNames: _col0, _col5, _col22 + Select Operator (SEL_108) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_120) - outputColumnNames: _col0, _col5, _col28 - Lateral View Forward (LVF_121) - Select Operator (SEL_122) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_125) - outputColumnNames: _col0, _col5, _col29 - Lateral View Forward (LVF_126) - Select Operator (SEL_127) + Lateral View Forward (LVF_109) + Select Operator (SEL_110) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_113) + outputColumnNames: _col0, _col5, _col23 + Select Operator (SEL_114) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_115) + Select Operator (SEL_116) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_130) - outputColumnNames: _col0, _col5, _col30 - Lateral View Forward (LVF_131) - Select Operator (SEL_132) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_135) - outputColumnNames: _col0, _col5, _col31 - Lateral View Forward (LVF_136) - Select Operator (SEL_137) + Lateral View Join Operator (LVJ_119) + outputColumnNames: _col0, _col5, _col24 + Select Operator (SEL_120) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_121) + Select Operator (SEL_122) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_125) + outputColumnNames: _col0, _col5, _col25 + Select Operator (SEL_126) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_140) - outputColumnNames: _col0, _col5, _col32 - Lateral View Forward (LVF_141) - Select Operator (SEL_142) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_145) - outputColumnNames: _col0, _col5, _col33 - Lateral View Forward (LVF_146) - Select Operator (SEL_147) + Lateral View Forward (LVF_127) + Select Operator (SEL_128) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_131) + outputColumnNames: _col0, _col5, _col26 + Select Operator (SEL_132) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_133) + Select Operator (SEL_134) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_150) - outputColumnNames: _col0, _col5, _col34 - Lateral View Forward (LVF_151) - Select Operator (SEL_152) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_155) - outputColumnNames: _col0, _col5, _col35 - Lateral View Forward (LVF_156) - Select Operator (SEL_157) + Lateral View Join Operator (LVJ_137) + outputColumnNames: _col0, _col5, _col27 + Select Operator (SEL_138) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_139) + Select Operator (SEL_140) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_143) + outputColumnNames: _col0, _col5, _col28 + Select Operator (SEL_144) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_160) - outputColumnNames: _col0, _col5, _col36 - Lateral View Forward (LVF_161) - Select Operator (SEL_162) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_165) - outputColumnNames: _col0, _col5, _col37 - Lateral View Forward (LVF_166) - Select Operator (SEL_167) + Lateral View Forward (LVF_145) + Select Operator (SEL_146) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_149) + outputColumnNames: _col0, _col5, _col29 + Select Operator (SEL_150) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_151) + Select Operator (SEL_152) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_170) - outputColumnNames: _col0, _col5, _col38 - Lateral View Forward (LVF_171) - Select Operator (SEL_172) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_175) - outputColumnNames: _col0, _col5, _col39 - Lateral View Forward (LVF_176) - Select Operator (SEL_177) + Lateral View Join Operator (LVJ_155) + outputColumnNames: _col0, _col5, _col30 + Select Operator (SEL_156) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_157) + Select Operator (SEL_158) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_161) + outputColumnNames: _col0, _col5, _col31 + Select Operator (SEL_162) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_180) - outputColumnNames: _col0, _col5, _col40 - Lateral View Forward (LVF_181) - Select Operator (SEL_182) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_185) - outputColumnNames: _col0, _col5, _col41 - Lateral View Forward (LVF_186) - Select Operator (SEL_187) + Lateral View Forward (LVF_163) + Select Operator (SEL_164) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_167) + outputColumnNames: _col0, _col5, _col32 + Select Operator (SEL_168) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_169) + Select Operator (SEL_170) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_190) - outputColumnNames: _col0, _col5, _col42 - Lateral View Forward (LVF_191) - Select Operator (SEL_192) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_195) - outputColumnNames: _col0, _col5, _col43 - Lateral View Forward (LVF_196) - Select Operator (SEL_197) + Lateral View Join Operator (LVJ_173) + outputColumnNames: _col0, _col5, _col33 + Select Operator (SEL_174) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_175) + Select Operator (SEL_176) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_179) + outputColumnNames: _col0, _col5, _col34 + Select Operator (SEL_180) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_200) - outputColumnNames: _col0, _col5, _col44 - Lateral View Forward (LVF_201) - Select Operator (SEL_202) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_205) - outputColumnNames: _col0, _col5, _col45 - Lateral View Forward (LVF_206) - Select Operator (SEL_207) + Lateral View Forward (LVF_181) + Select Operator (SEL_182) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_185) + outputColumnNames: _col0, _col5, _col35 + Select Operator (SEL_186) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_187) + Select Operator (SEL_188) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_210) - outputColumnNames: _col0, _col5, _col46 - Lateral View Forward (LVF_211) - Select Operator (SEL_212) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_215) - outputColumnNames: _col0, _col5, _col47 - Lateral View Forward (LVF_216) - Select Operator (SEL_217) + Lateral View Join Operator (LVJ_191) + outputColumnNames: _col0, _col5, _col36 + Select Operator (SEL_192) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_193) + Select Operator (SEL_194) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_197) + outputColumnNames: _col0, _col5, _col37 + Select Operator (SEL_198) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_220) - outputColumnNames: _col0, _col5, _col48 - Lateral View Forward (LVF_221) - Select Operator (SEL_222) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_225) - outputColumnNames: _col0, _col5, _col49 - Lateral View Forward (LVF_226) - Select Operator (SEL_227) + Lateral View Forward (LVF_199) + Select Operator (SEL_200) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_203) + outputColumnNames: _col0, _col5, _col38 + Select Operator (SEL_204) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_205) + Select Operator (SEL_206) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_230) - outputColumnNames: _col0, _col5, _col50 - Lateral View Forward (LVF_231) - Select Operator (SEL_232) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_235) - outputColumnNames: _col0, _col5, _col51 - Lateral View Forward (LVF_236) - Select Operator (SEL_237) + Lateral View Join Operator (LVJ_209) + outputColumnNames: _col0, _col5, _col39 + Select Operator (SEL_210) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_211) + Select Operator (SEL_212) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_215) + outputColumnNames: _col0, _col5, _col40 + Select Operator (SEL_216) expressions: _col0 (type: array), _col5 (type: string) outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_240) - outputColumnNames: _col0, _col5, _col52 - Lateral View Forward (LVF_241) - Select Operator (SEL_242) - expressions: _col0 (type: array), _col5 (type: string) - outputColumnNames: _col0, _col5 - Lateral View Join Operator (LVJ_245) - outputColumnNames: _col0, _col5, _col53 - Lateral View Forward (LVF_246) - Select Operator (SEL_247) - expressions: _col5 (type: string) - outputColumnNames: _col5 - Lateral View Join Operator (LVJ_250) - outputColumnNames: _col5, _col54 - Select Operator (SEL_251) - expressions: _col5 (type: string) - outputColumnNames: _col0 - ListSink (LIST_SINK_253) - Select Operator (SEL_248) + Lateral View Forward (LVF_217) + Select Operator (SEL_218) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_221) + outputColumnNames: _col0, _col5, _col41 + Select Operator (SEL_222) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_223) + Select Operator (SEL_224) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_227) + outputColumnNames: _col0, _col5, _col42 + Select Operator (SEL_228) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_229) + Select Operator (SEL_230) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_233) + outputColumnNames: _col0, _col5, _col43 + Select Operator (SEL_234) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_235) + Select Operator (SEL_236) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_239) + outputColumnNames: _col0, _col5, _col44 + Select Operator (SEL_240) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_241) + Select Operator (SEL_242) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_245) + outputColumnNames: _col0, _col5, _col45 + Select Operator (SEL_246) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_247) + Select Operator (SEL_248) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_251) + outputColumnNames: _col0, _col5, _col46 + Select Operator (SEL_252) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_253) + Select Operator (SEL_254) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_257) + outputColumnNames: _col0, _col5, _col47 + Select Operator (SEL_258) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_259) + Select Operator (SEL_260) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_263) + outputColumnNames: _col0, _col5, _col48 + Select Operator (SEL_264) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_265) + Select Operator (SEL_266) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_269) + outputColumnNames: _col0, _col5, _col49 + Select Operator (SEL_270) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_271) + Select Operator (SEL_272) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_275) + outputColumnNames: _col0, _col5, _col50 + Select Operator (SEL_276) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_277) + Select Operator (SEL_278) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_281) + outputColumnNames: _col0, _col5, _col51 + Select Operator (SEL_282) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_283) + Select Operator (SEL_284) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_287) + outputColumnNames: _col0, _col5, _col52 + Select Operator (SEL_288) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_289) + Select Operator (SEL_290) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Join Operator (LVJ_293) + outputColumnNames: _col0, _col5, _col53 + Select Operator (SEL_294) + expressions: _col0 (type: array), _col5 (type: string) + outputColumnNames: _col0, _col5 + Lateral View Forward (LVF_295) + Select Operator (SEL_296) + expressions: _col5 (type: string) + outputColumnNames: _col5 + Lateral View Join Operator (LVJ_299) + outputColumnNames: _col5, _col54 + Select Operator (SEL_300) + expressions: _col5 (type: string) + outputColumnNames: _col0 + ListSink (LIST_SINK_302) + Select Operator (SEL_297) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_298) + function name: explode + Lateral View Join Operator (LVJ_299) + outputColumnNames: _col5, _col54 + Select Operator (SEL_291) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_292) + function name: explode + Lateral View Join Operator (LVJ_293) + outputColumnNames: _col0, _col5, _col53 + Select Operator (SEL_285) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_286) + function name: explode + Lateral View Join Operator (LVJ_287) + outputColumnNames: _col0, _col5, _col52 + Select Operator (SEL_279) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_280) + function name: explode + Lateral View Join Operator (LVJ_281) + outputColumnNames: _col0, _col5, _col51 + Select Operator (SEL_273) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_274) + function name: explode + Lateral View Join Operator (LVJ_275) + outputColumnNames: _col0, _col5, _col50 + Select Operator (SEL_267) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_268) + function name: explode + Lateral View Join Operator (LVJ_269) + outputColumnNames: _col0, _col5, _col49 + Select Operator (SEL_261) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_262) + function name: explode + Lateral View Join Operator (LVJ_263) + outputColumnNames: _col0, _col5, _col48 + Select Operator (SEL_255) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_256) + function name: explode + Lateral View Join Operator (LVJ_257) + outputColumnNames: _col0, _col5, _col47 + Select Operator (SEL_249) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_250) + function name: explode + Lateral View Join Operator (LVJ_251) + outputColumnNames: _col0, _col5, _col46 + Select Operator (SEL_243) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_244) + function name: explode + Lateral View Join Operator (LVJ_245) + outputColumnNames: _col0, _col5, _col45 + Select Operator (SEL_237) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_238) + function name: explode + Lateral View Join Operator (LVJ_239) + outputColumnNames: _col0, _col5, _col44 + Select Operator (SEL_231) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_232) + function name: explode + Lateral View Join Operator (LVJ_233) + outputColumnNames: _col0, _col5, _col43 + Select Operator (SEL_225) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_249) + UDTF Operator (UDTF_226) function name: explode - Lateral View Join Operator (LVJ_250) - outputColumnNames: _col5, _col54 - Select Operator (SEL_243) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_244) - function name: explode - Lateral View Join Operator (LVJ_245) - outputColumnNames: _col0, _col5, _col53 - Select Operator (SEL_238) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_239) - function name: explode - Lateral View Join Operator (LVJ_240) - outputColumnNames: _col0, _col5, _col52 - Select Operator (SEL_233) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_234) - function name: explode - Lateral View Join Operator (LVJ_235) - outputColumnNames: _col0, _col5, _col51 - Select Operator (SEL_228) + Lateral View Join Operator (LVJ_227) + outputColumnNames: _col0, _col5, _col42 + Select Operator (SEL_219) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_220) + function name: explode + Lateral View Join Operator (LVJ_221) + outputColumnNames: _col0, _col5, _col41 + Select Operator (SEL_213) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_214) + function name: explode + Lateral View Join Operator (LVJ_215) + outputColumnNames: _col0, _col5, _col40 + Select Operator (SEL_207) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_229) + UDTF Operator (UDTF_208) function name: explode - Lateral View Join Operator (LVJ_230) - outputColumnNames: _col0, _col5, _col50 - Select Operator (SEL_223) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_224) - function name: explode - Lateral View Join Operator (LVJ_225) - outputColumnNames: _col0, _col5, _col49 - Select Operator (SEL_218) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_219) - function name: explode - Lateral View Join Operator (LVJ_220) - outputColumnNames: _col0, _col5, _col48 - Select Operator (SEL_213) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_214) - function name: explode - Lateral View Join Operator (LVJ_215) - outputColumnNames: _col0, _col5, _col47 - Select Operator (SEL_208) + Lateral View Join Operator (LVJ_209) + outputColumnNames: _col0, _col5, _col39 + Select Operator (SEL_201) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_202) + function name: explode + Lateral View Join Operator (LVJ_203) + outputColumnNames: _col0, _col5, _col38 + Select Operator (SEL_195) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_196) + function name: explode + Lateral View Join Operator (LVJ_197) + outputColumnNames: _col0, _col5, _col37 + Select Operator (SEL_189) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_209) + UDTF Operator (UDTF_190) function name: explode - Lateral View Join Operator (LVJ_210) - outputColumnNames: _col0, _col5, _col46 - Select Operator (SEL_203) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_204) - function name: explode - Lateral View Join Operator (LVJ_205) - outputColumnNames: _col0, _col5, _col45 - Select Operator (SEL_198) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_199) - function name: explode - Lateral View Join Operator (LVJ_200) - outputColumnNames: _col0, _col5, _col44 - Select Operator (SEL_193) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_194) - function name: explode - Lateral View Join Operator (LVJ_195) - outputColumnNames: _col0, _col5, _col43 - Select Operator (SEL_188) + Lateral View Join Operator (LVJ_191) + outputColumnNames: _col0, _col5, _col36 + Select Operator (SEL_183) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_184) + function name: explode + Lateral View Join Operator (LVJ_185) + outputColumnNames: _col0, _col5, _col35 + Select Operator (SEL_177) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_178) + function name: explode + Lateral View Join Operator (LVJ_179) + outputColumnNames: _col0, _col5, _col34 + Select Operator (SEL_171) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_189) + UDTF Operator (UDTF_172) function name: explode - Lateral View Join Operator (LVJ_190) - outputColumnNames: _col0, _col5, _col42 - Select Operator (SEL_183) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_184) - function name: explode - Lateral View Join Operator (LVJ_185) - outputColumnNames: _col0, _col5, _col41 - Select Operator (SEL_178) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_179) - function name: explode - Lateral View Join Operator (LVJ_180) - outputColumnNames: _col0, _col5, _col40 - Select Operator (SEL_173) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_174) - function name: explode - Lateral View Join Operator (LVJ_175) - outputColumnNames: _col0, _col5, _col39 - Select Operator (SEL_168) + Lateral View Join Operator (LVJ_173) + outputColumnNames: _col0, _col5, _col33 + Select Operator (SEL_165) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_166) + function name: explode + Lateral View Join Operator (LVJ_167) + outputColumnNames: _col0, _col5, _col32 + Select Operator (SEL_159) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_160) + function name: explode + Lateral View Join Operator (LVJ_161) + outputColumnNames: _col0, _col5, _col31 + Select Operator (SEL_153) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_169) + UDTF Operator (UDTF_154) function name: explode - Lateral View Join Operator (LVJ_170) - outputColumnNames: _col0, _col5, _col38 - Select Operator (SEL_163) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_164) - function name: explode - Lateral View Join Operator (LVJ_165) - outputColumnNames: _col0, _col5, _col37 - Select Operator (SEL_158) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_159) - function name: explode - Lateral View Join Operator (LVJ_160) - outputColumnNames: _col0, _col5, _col36 - Select Operator (SEL_153) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_154) - function name: explode - Lateral View Join Operator (LVJ_155) - outputColumnNames: _col0, _col5, _col35 - Select Operator (SEL_148) + Lateral View Join Operator (LVJ_155) + outputColumnNames: _col0, _col5, _col30 + Select Operator (SEL_147) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_148) + function name: explode + Lateral View Join Operator (LVJ_149) + outputColumnNames: _col0, _col5, _col29 + Select Operator (SEL_141) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_142) + function name: explode + Lateral View Join Operator (LVJ_143) + outputColumnNames: _col0, _col5, _col28 + Select Operator (SEL_135) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_149) + UDTF Operator (UDTF_136) function name: explode - Lateral View Join Operator (LVJ_150) - outputColumnNames: _col0, _col5, _col34 - Select Operator (SEL_143) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_144) - function name: explode - Lateral View Join Operator (LVJ_145) - outputColumnNames: _col0, _col5, _col33 - Select Operator (SEL_138) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_139) - function name: explode - Lateral View Join Operator (LVJ_140) - outputColumnNames: _col0, _col5, _col32 - Select Operator (SEL_133) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_134) - function name: explode - Lateral View Join Operator (LVJ_135) - outputColumnNames: _col0, _col5, _col31 - Select Operator (SEL_128) + Lateral View Join Operator (LVJ_137) + outputColumnNames: _col0, _col5, _col27 + Select Operator (SEL_129) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_130) + function name: explode + Lateral View Join Operator (LVJ_131) + outputColumnNames: _col0, _col5, _col26 + Select Operator (SEL_123) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_124) + function name: explode + Lateral View Join Operator (LVJ_125) + outputColumnNames: _col0, _col5, _col25 + Select Operator (SEL_117) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_129) + UDTF Operator (UDTF_118) function name: explode - Lateral View Join Operator (LVJ_130) - outputColumnNames: _col0, _col5, _col30 - Select Operator (SEL_123) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_124) - function name: explode - Lateral View Join Operator (LVJ_125) - outputColumnNames: _col0, _col5, _col29 - Select Operator (SEL_118) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_119) - function name: explode - Lateral View Join Operator (LVJ_120) - outputColumnNames: _col0, _col5, _col28 - Select Operator (SEL_113) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_114) - function name: explode - Lateral View Join Operator (LVJ_115) - outputColumnNames: _col0, _col5, _col27 - Select Operator (SEL_108) + Lateral View Join Operator (LVJ_119) + outputColumnNames: _col0, _col5, _col24 + Select Operator (SEL_111) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_112) + function name: explode + Lateral View Join Operator (LVJ_113) + outputColumnNames: _col0, _col5, _col23 + Select Operator (SEL_105) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_106) + function name: explode + Lateral View Join Operator (LVJ_107) + outputColumnNames: _col0, _col5, _col22 + Select Operator (SEL_99) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_109) + UDTF Operator (UDTF_100) function name: explode - Lateral View Join Operator (LVJ_110) - outputColumnNames: _col0, _col5, _col26 - Select Operator (SEL_103) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_104) - function name: explode - Lateral View Join Operator (LVJ_105) - outputColumnNames: _col0, _col5, _col25 - Select Operator (SEL_98) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_99) - function name: explode - Lateral View Join Operator (LVJ_100) - outputColumnNames: _col0, _col5, _col24 - Select Operator (SEL_93) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_94) - function name: explode - Lateral View Join Operator (LVJ_95) - outputColumnNames: _col0, _col5, _col23 - Select Operator (SEL_88) + Lateral View Join Operator (LVJ_101) + outputColumnNames: _col0, _col5, _col21 + Select Operator (SEL_93) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_94) + function name: explode + Lateral View Join Operator (LVJ_95) + outputColumnNames: _col0, _col5, _col20 + Select Operator (SEL_87) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_88) + function name: explode + Lateral View Join Operator (LVJ_89) + outputColumnNames: _col0, _col5, _col19 + Select Operator (SEL_81) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_89) + UDTF Operator (UDTF_82) function name: explode - Lateral View Join Operator (LVJ_90) - outputColumnNames: _col0, _col5, _col22 - Select Operator (SEL_83) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_84) - function name: explode - Lateral View Join Operator (LVJ_85) - outputColumnNames: _col0, _col5, _col21 - Select Operator (SEL_78) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_79) - function name: explode - Lateral View Join Operator (LVJ_80) - outputColumnNames: _col0, _col5, _col20 - Select Operator (SEL_73) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_74) - function name: explode - Lateral View Join Operator (LVJ_75) - outputColumnNames: _col0, _col5, _col19 - Select Operator (SEL_68) + Lateral View Join Operator (LVJ_83) + outputColumnNames: _col0, _col5, _col18 + Select Operator (SEL_75) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_76) + function name: explode + Lateral View Join Operator (LVJ_77) + outputColumnNames: _col0, _col5, _col17 + Select Operator (SEL_69) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_70) + function name: explode + Lateral View Join Operator (LVJ_71) + outputColumnNames: _col0, _col5, _col16 + Select Operator (SEL_63) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_69) + UDTF Operator (UDTF_64) function name: explode - Lateral View Join Operator (LVJ_70) - outputColumnNames: _col0, _col5, _col18 - Select Operator (SEL_63) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_64) - function name: explode - Lateral View Join Operator (LVJ_65) - outputColumnNames: _col0, _col5, _col17 - Select Operator (SEL_58) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_59) - function name: explode - Lateral View Join Operator (LVJ_60) - outputColumnNames: _col0, _col5, _col16 - Select Operator (SEL_53) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_54) - function name: explode - Lateral View Join Operator (LVJ_55) - outputColumnNames: _col0, _col5, _col15 - Select Operator (SEL_48) + Lateral View Join Operator (LVJ_65) + outputColumnNames: _col0, _col5, _col15 + Select Operator (SEL_57) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_58) + function name: explode + Lateral View Join Operator (LVJ_59) + outputColumnNames: _col0, _col5, _col14 + Select Operator (SEL_51) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_52) + function name: explode + Lateral View Join Operator (LVJ_53) + outputColumnNames: _col0, _col5, _col13 + Select Operator (SEL_45) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_49) + UDTF Operator (UDTF_46) function name: explode - Lateral View Join Operator (LVJ_50) - outputColumnNames: _col0, _col5, _col14 - Select Operator (SEL_43) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_44) - function name: explode - Lateral View Join Operator (LVJ_45) - outputColumnNames: _col0, _col5, _col13 - Select Operator (SEL_38) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_39) - function name: explode - Lateral View Join Operator (LVJ_40) - outputColumnNames: _col0, _col5, _col12 - Select Operator (SEL_33) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_34) - function name: explode - Lateral View Join Operator (LVJ_35) - outputColumnNames: _col0, _col5, _col11 - Select Operator (SEL_28) + Lateral View Join Operator (LVJ_47) + outputColumnNames: _col0, _col5, _col12 + Select Operator (SEL_39) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_40) + function name: explode + Lateral View Join Operator (LVJ_41) + outputColumnNames: _col0, _col5, _col11 + Select Operator (SEL_33) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_34) + function name: explode + Lateral View Join Operator (LVJ_35) + outputColumnNames: _col0, _col5, _col10 + Select Operator (SEL_27) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_29) + UDTF Operator (UDTF_28) function name: explode - Lateral View Join Operator (LVJ_30) - outputColumnNames: _col0, _col5, _col10 - Select Operator (SEL_23) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_24) - function name: explode - Lateral View Join Operator (LVJ_25) - outputColumnNames: _col0, _col5, _col9 - Select Operator (SEL_18) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_19) - function name: explode - Lateral View Join Operator (LVJ_20) - outputColumnNames: _col0, _col5, _col8 - Select Operator (SEL_13) - expressions: _col0 (type: array) - outputColumnNames: _col0 - UDTF Operator (UDTF_14) - function name: explode - Lateral View Join Operator (LVJ_15) - outputColumnNames: _col0, _col5, _col7 - Select Operator (SEL_8) + Lateral View Join Operator (LVJ_29) + outputColumnNames: _col0, _col5, _col9 + Select Operator (SEL_21) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_22) + function name: explode + Lateral View Join Operator (LVJ_23) + outputColumnNames: _col0, _col5, _col8 + Select Operator (SEL_15) + expressions: _col0 (type: array) + outputColumnNames: _col0 + UDTF Operator (UDTF_16) + function name: explode + Lateral View Join Operator (LVJ_17) + outputColumnNames: _col0, _col5, _col7 + Select Operator (SEL_9) expressions: _col0 (type: array) outputColumnNames: _col0 - UDTF Operator (UDTF_9) + UDTF Operator (UDTF_10) function name: explode - Lateral View Join Operator (LVJ_10) + Lateral View Join Operator (LVJ_11) outputColumnNames: _col0, _col5, _col6 Select Operator (SEL_3) expressions: orders (type: array) diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_noalias.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_noalias.q.out index b08b4b7bd746..a06607e78bf0 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view_noalias.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view_noalias.q.out @@ -130,6 +130,8 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src + properties: + insideView TRUE Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Lateral View Forward Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_onview.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_onview.q.out index 9e6bdbbbc7a4..224739b1c940 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view_onview.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view_onview.q.out @@ -175,6 +175,8 @@ STAGE PLANS: Processor Tree: TableScan alias: lv_table_n0 + properties: + insideView TRUE Select Operator Lateral View Forward Select Operator @@ -220,6 +222,8 @@ STAGE PLANS: Processor Tree: TableScan alias: lv_table_n0 + properties: + insideView TRUE Select Operator Lateral View Forward Select Operator @@ -303,6 +307,8 @@ STAGE PLANS: Processor Tree: TableScan alias: lv_table_n0 + properties: + insideView TRUE Select Operator Lateral View Forward Select Operator @@ -440,6 +446,8 @@ STAGE PLANS: Processor Tree: TableScan alias: lv_table_n0 + properties: + insideView TRUE Select Operator expressions: c1 (type: string), c2 (type: array), c3 (type: int), c4 (type: char(1)) outputColumnNames: _col0, _col1, _col2, _col3 diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_onview2.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_onview2.q.out index 527ab85d79c1..f5b124e34716 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view_onview2.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view_onview2.q.out @@ -52,6 +52,8 @@ STAGE PLANS: Processor Tree: TableScan alias: lv_table_n1 + properties: + insideView TRUE Select Operator Lateral View Forward Select Operator diff --git a/ql/src/test/results/clientpositive/llap/lateral_view_ppd.q.out b/ql/src/test/results/clientpositive/llap/lateral_view_ppd.q.out index 930a8e6f43de..3497b6918ef3 100644 --- a/ql/src/test/results/clientpositive/llap/lateral_view_ppd.q.out +++ b/ql/src/test/results/clientpositive/llap/lateral_view_ppd.q.out @@ -19,27 +19,30 @@ STAGE PLANS: filterExpr: (key = '0') (type: boolean) Filter Operator predicate: (key = '0') (type: boolean) - Lateral View Forward - Select Operator - expressions: value (type: string) - outputColumnNames: value - Lateral View Join Operator - outputColumnNames: _col1, _col6 - Select Operator - expressions: _col1 (type: string), _col6 (type: int) - outputColumnNames: _col0, _col1 - ListSink - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - UDTF Operator - function name: explode + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Lateral View Forward + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 Lateral View Join Operator outputColumnNames: _col1, _col6 Select Operator expressions: _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1 ListSink + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col1, _col6 + Select Operator + expressions: _col1 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1 + ListSink PREHOOK: query: SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0' PREHOOK: type: QUERY @@ -79,29 +82,32 @@ STAGE PLANS: filterExpr: (key = '0') (type: boolean) Filter Operator predicate: (key = '0') (type: boolean) - Lateral View Forward - Select Operator - expressions: value (type: string) - outputColumnNames: value - Lateral View Join Operator - outputColumnNames: _col1, _col6 - Select Operator - expressions: _col1 (type: string), 1 (type: int) - outputColumnNames: _col0, _col1 - ListSink - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - UDTF Operator - function name: explode - Filter Operator - predicate: (col = 1) (type: boolean) - Lateral View Join Operator - outputColumnNames: _col1, _col6 - Select Operator - expressions: _col1 (type: string), 1 (type: int) - outputColumnNames: _col0, _col1 - ListSink + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Lateral View Forward + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 + Lateral View Join Operator + outputColumnNames: _col1, _col6 + Select Operator + expressions: _col1 (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + ListSink + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Filter Operator + predicate: (col = 1) (type: boolean) + Lateral View Join Operator + outputColumnNames: _col1, _col6 + Select Operator + expressions: _col1 (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + ListSink PREHOOK: query: SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE key='0' AND myCol=1 PREHOOK: type: QUERY @@ -135,23 +141,13 @@ STAGE PLANS: TableScan alias: srcpart filterExpr: ((ds = '2008-04-08') and (hr = '12')) (type: boolean) - Lateral View Forward - Select Operator - expressions: value (type: string) - outputColumnNames: value - Lateral View Join Operator - outputColumnNames: _col1, _col8 - Limit - Number of rows: 12 - Select Operator - expressions: _col1 (type: string), _col8 (type: int) - outputColumnNames: _col0, _col1 - ListSink - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - UDTF Operator - function name: explode + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Lateral View Forward + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 Lateral View Join Operator outputColumnNames: _col1, _col8 Limit @@ -160,6 +156,19 @@ STAGE PLANS: expressions: _col1 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1 ListSink + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col1, _col8 + Limit + Number of rows: 12 + Select Operator + expressions: _col1 (type: string), _col8 (type: int) + outputColumnNames: _col0, _col1 + ListSink PREHOOK: query: SELECT value, myCol FROM (SELECT * FROM srcpart LATERAL VIEW explode(array(1,2,3)) myTable AS myCol) a WHERE ds='2008-04-08' AND hr="12" LIMIT 12 PREHOOK: type: QUERY @@ -204,38 +213,13 @@ STAGE PLANS: filterExpr: (key = '0') (type: boolean) Filter Operator predicate: (key = '0') (type: boolean) - Lateral View Forward - Select Operator - expressions: value (type: string) - outputColumnNames: value - Lateral View Join Operator - outputColumnNames: _col1, _col6 - Lateral View Forward - Select Operator - expressions: _col1 (type: string), _col6 (type: int) - outputColumnNames: _col1, _col6 - Lateral View Join Operator - outputColumnNames: _col1, _col6, _col7 - Select Operator - expressions: _col1 (type: string), _col6 (type: int) - outputColumnNames: _col0, _col1 - ListSink - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - UDTF Operator - function name: explode - Lateral View Join Operator - outputColumnNames: _col1, _col6, _col7 - Select Operator - expressions: _col1 (type: string), _col6 (type: int) - outputColumnNames: _col0, _col1 - ListSink - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - UDTF Operator - function name: explode + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Lateral View Forward + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 Lateral View Join Operator outputColumnNames: _col1, _col6 Lateral View Forward @@ -259,6 +243,34 @@ STAGE PLANS: expressions: _col1 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1 ListSink + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col1, _col6 + Lateral View Forward + Select Operator + expressions: _col1 (type: string), _col6 (type: int) + outputColumnNames: _col1, _col6 + Lateral View Join Operator + outputColumnNames: _col1, _col6, _col7 + Select Operator + expressions: _col1 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1 + ListSink + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Lateral View Join Operator + outputColumnNames: _col1, _col6, _col7 + Select Operator + expressions: _col1 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1 + ListSink PREHOOK: query: SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0' PREHOOK: type: QUERY @@ -316,29 +328,32 @@ STAGE PLANS: filterExpr: (key = '0') (type: boolean) Filter Operator predicate: (key = '0') (type: boolean) - Lateral View Forward - Select Operator - expressions: value (type: string) - outputColumnNames: value - Lateral View Join Operator - outputColumnNames: _col1, _col6 - Select Operator - expressions: _col1 (type: string), _col6 (type: int) - outputColumnNames: _col0, _col1 - ListSink - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - UDTF Operator - function name: explode - Filter Operator - predicate: (col > 1) (type: boolean) - Lateral View Join Operator - outputColumnNames: _col1, _col6 - Select Operator - expressions: _col1 (type: string), _col6 (type: int) - outputColumnNames: _col0, _col1 - ListSink + Select Operator + expressions: value (type: string) + outputColumnNames: _col1 + Lateral View Forward + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col1 + Lateral View Join Operator + outputColumnNames: _col1, _col6 + Select Operator + expressions: _col1 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1 + ListSink + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + UDTF Operator + function name: explode + Filter Operator + predicate: (col > 1) (type: boolean) + Lateral View Join Operator + outputColumnNames: _col1, _col6 + Select Operator + expressions: _col1 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1 + ListSink PREHOOK: query: SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0' PREHOOK: type: QUERY diff --git a/ql/src/test/results/clientpositive/llap/lineage2.q.out b/ql/src/test/results/clientpositive/llap/lineage2.q.out index 8b5b272d157b..441918757744 100644 --- a/ql/src/test/results/clientpositive/llap/lineage2.q.out +++ b/ql/src/test/results/clientpositive/llap/lineage2.q.out @@ -724,7 +724,7 @@ PREHOOK: query: select identity, ep1_id from relations PREHOOK: type: QUERY PREHOOK: Input: default@relations #### A masked pattern was here #### -{"version":"1.0","engine":"tez","database":"default","hash":"436a649a0d9540e8f093f8353d86813a","queryText":"select identity, ep1_id from relations\n lateral view explode(ep1_ids) nav_rel as ep1_id","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"nav_rel._col12","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"identity"},{"id":1,"vertexType":"COLUMN","vertexId":"ep1_id"},{"id":2,"vertexType":"COLUMN","vertexId":"default.relations.identity"},{"id":3,"vertexType":"COLUMN","vertexId":"default.relations.ep1_ids"}]} +{"version":"1.0","engine":"tez","database":"default","hash":"436a649a0d9540e8f093f8353d86813a","queryText":"select identity, ep1_id from relations\n lateral view explode(ep1_ids) nav_rel as ep1_id","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"relations._col12","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"identity"},{"id":1,"vertexType":"COLUMN","vertexId":"ep1_id"},{"id":2,"vertexType":"COLUMN","vertexId":"default.relations.identity"},{"id":3,"vertexType":"COLUMN","vertexId":"default.relations.ep1_ids"}]} PREHOOK: query: insert into rels_exploded select identity, type, ep1_src_type, ep1_type, ep2_src_type, ep2_type, ep1_id, ep2_id from relations lateral view explode(ep1_ids) rel1 as ep1_id @@ -733,4 +733,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@relations PREHOOK: Output: default@rels_exploded Result schema has 8 fields, but we don't get as many dependencies -{"version":"1.0","engine":"tez","database":"default","hash":"56b2b197f394a30537ce1acf835ff8e1","queryText":"insert into rels_exploded select identity, type,\n ep1_src_type, ep1_type, ep2_src_type, ep2_type, ep1_id, ep2_id\nfrom relations lateral view explode(ep1_ids) rel1 as ep1_id\n lateral view explode (ep2_ids) rel2 as ep2_id","edges":[{"sources":[8],"targets":[0],"edgeType":"PROJECTION"},{"sources":[9],"targets":[1],"edgeType":"PROJECTION"},{"sources":[10],"targets":[2],"edgeType":"PROJECTION"},{"sources":[11],"targets":[3],"edgeType":"PROJECTION"},{"sources":[12],"targets":[4],"edgeType":"PROJECTION"},{"sources":[13],"targets":[5],"edgeType":"PROJECTION"},{"sources":[14],"targets":[6],"expression":"CAST( rel1._col12 AS CHAR(32))","edgeType":"PROJECTION"},{"sources":[15],"targets":[7],"expression":"CAST( rel2._col13 AS CHAR(32))","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.rels_exploded.identity"},{"id":1,"vertexType":"COLUMN","vertexId":"default.rels_exploded.type"},{"id":2,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_src_type"},{"id":3,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_type"},{"id":4,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_src_type"},{"id":5,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_type"},{"id":6,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_id"},{"id":7,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_id"},{"id":8,"vertexType":"COLUMN","vertexId":"default.relations.identity"},{"id":9,"vertexType":"COLUMN","vertexId":"default.relations.type"},{"id":10,"vertexType":"COLUMN","vertexId":"default.relations.ep1_src_type"},{"id":11,"vertexType":"COLUMN","vertexId":"default.relations.ep1_type"},{"id":12,"vertexType":"COLUMN","vertexId":"default.relations.ep2_src_type"},{"id":13,"vertexType":"COLUMN","vertexId":"default.relations.ep2_type"},{"id":14,"vertexType":"COLUMN","vertexId":"default.relations.ep1_ids"},{"id":15,"vertexType":"COLUMN","vertexId":"default.relations.ep2_ids"}]} +{"version":"1.0","engine":"tez","database":"default","hash":"56b2b197f394a30537ce1acf835ff8e1","queryText":"insert into rels_exploded select identity, type,\n ep1_src_type, ep1_type, ep2_src_type, ep2_type, ep1_id, ep2_id\nfrom relations lateral view explode(ep1_ids) rel1 as ep1_id\n lateral view explode (ep2_ids) rel2 as ep2_id","edges":[{"sources":[8],"targets":[0],"edgeType":"PROJECTION"},{"sources":[9],"targets":[1],"edgeType":"PROJECTION"},{"sources":[10],"targets":[2],"edgeType":"PROJECTION"},{"sources":[11],"targets":[3],"edgeType":"PROJECTION"},{"sources":[12],"targets":[4],"edgeType":"PROJECTION"},{"sources":[13],"targets":[5],"edgeType":"PROJECTION"},{"sources":[14],"targets":[6],"expression":"CAST( rel1.ep1_id AS CHAR(32))","edgeType":"PROJECTION"},{"sources":[15],"targets":[7],"expression":"CAST( rel2.ep2_id AS CHAR(32))","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.rels_exploded.identity"},{"id":1,"vertexType":"COLUMN","vertexId":"default.rels_exploded.type"},{"id":2,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_src_type"},{"id":3,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_type"},{"id":4,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_src_type"},{"id":5,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_type"},{"id":6,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep1_id"},{"id":7,"vertexType":"COLUMN","vertexId":"default.rels_exploded.ep2_id"},{"id":8,"vertexType":"COLUMN","vertexId":"default.relations.identity"},{"id":9,"vertexType":"COLUMN","vertexId":"default.relations.type"},{"id":10,"vertexType":"COLUMN","vertexId":"default.relations.ep1_src_type"},{"id":11,"vertexType":"COLUMN","vertexId":"default.relations.ep1_type"},{"id":12,"vertexType":"COLUMN","vertexId":"default.relations.ep2_src_type"},{"id":13,"vertexType":"COLUMN","vertexId":"default.relations.ep2_type"},{"id":14,"vertexType":"COLUMN","vertexId":"default.relations.ep1_ids"},{"id":15,"vertexType":"COLUMN","vertexId":"default.relations.ep2_ids"}]} diff --git a/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out index 9bbd5d4462e4..d12554dd530d 100644 --- a/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out +++ b/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out @@ -133,50 +133,16 @@ STAGE PLANS: Filter Operator predicate: aid is not null (type: boolean) Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - Lateral View Forward + Select Operator + expressions: aid (type: int), av (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: aid (type: int) - outputColumnNames: aid + Lateral View Forward Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - Lateral View Join Operator - outputColumnNames: _col0, _col6 - Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col6 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3 - input vertices: - 1 Map 2 - Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (_col0 = _col2) (type: boolean) - Statistics: Num rows: 1 Data size: 4232 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 4232 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 4232 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Select Operator - expressions: av (type: array) - outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - UDTF Operator + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - function name: explode Lateral View Join Operator outputColumnNames: _col0, _col6 Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE @@ -190,20 +156,52 @@ STAGE PLANS: keys: 0 _col0 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col3 input vertices: 1 Map 2 Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (_col0 = _col2) (type: boolean) - Statistics: Num rows: 1 Data size: 4232 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Select Operator + expressions: _col1 (type: array) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE + UDTF Operator + Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col6 + Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col6 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + input vertices: + 1 Map 2 + Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 4232 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 4232 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 12698 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -219,33 +217,16 @@ STAGE PLANS: Filter Operator predicate: bid is not null (type: boolean) Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - Lateral View Forward + Select Operator + expressions: bid (type: int), bv (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: bid (type: int) - outputColumnNames: bid - Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - Lateral View Join Operator - outputColumnNames: _col0, _col6 - Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), _col6 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) - Select Operator - expressions: bv (type: array) - outputColumnNames: _col0 + Lateral View Forward Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - UDTF Operator + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE - function name: explode Lateral View Join Operator outputColumnNames: _col0, _col6 Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE @@ -260,6 +241,27 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) + Select Operator + expressions: _col1 (type: array) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE + UDTF Operator + Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col6 + Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col6 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 11544 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Execution mode: llap LLAP IO: all inputs diff --git a/ql/src/test/results/clientpositive/llap/ppd_field_garbage.q.out b/ql/src/test/results/clientpositive/llap/ppd_field_garbage.q.out index 494db05e7413..3b9498cbcb0a 100644 --- a/ql/src/test/results/clientpositive/llap/ppd_field_garbage.q.out +++ b/ql/src/test/results/clientpositive/llap/ppd_field_garbage.q.out @@ -17,7 +17,7 @@ POSTHOOK: Input: default@test_issue POSTHOOK: Output: database:default POSTHOOK: Output: default@v_test_issue POSTHOOK: Lineage: v_test_issue.age EXPRESSION [(test_issue)test_issue.FieldSchema(name:test_c, type:struct>, comment:null), ] -POSTHOOK: Lineage: v_test_issue.fileid EXPRESSION [(test_issue)test_issue.FieldSchema(name:fileid, type:int, comment:null), ] +POSTHOOK: Lineage: v_test_issue.fileid SIMPLE [(test_issue)test_issue.FieldSchema(name:fileid, type:int, comment:null), ] POSTHOOK: Lineage: v_test_issue.u SCRIPT [(test_issue)test_issue.FieldSchema(name:infos, type:array>, comment:null), ] PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_issue PREHOOK: type: LOAD diff --git a/ql/src/test/results/clientpositive/llap/reduce_deduplicate_null_keys.q.out b/ql/src/test/results/clientpositive/llap/reduce_deduplicate_null_keys.q.out index 0eb4c37330b8..1532755cd819 100644 --- a/ql/src/test/results/clientpositive/llap/reduce_deduplicate_null_keys.q.out +++ b/ql/src/test/results/clientpositive/llap/reduce_deduplicate_null_keys.q.out @@ -82,7 +82,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: CASE WHEN ((_col0 = _col1)) THEN (1) ELSE (2) END (type: int) + key expressions: if((_col0 = _col1), 1, 2) (type: int) null sort order: z sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE @@ -101,7 +101,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: CASE WHEN ((_col0 = _col1)) THEN (1) ELSE (2) END (type: int) + key expressions: if((_col0 = _col1), 1, 2) (type: int) null sort order: z sort order: + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE diff --git a/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin6.q.out b/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin6.q.out index b47793f9fb9c..845ada00f123 100644 --- a/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin6.q.out +++ b/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin6.q.out @@ -71,12 +71,16 @@ STAGE PLANS: Filter Operator predicate: key is not null (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs Map 2 @@ -84,48 +88,35 @@ STAGE PLANS: TableScan alias: b filterExpr: key is not null (type: boolean) - probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:0, keyRatio:6.0 + probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_31_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.0 Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 key (type: string) - 1 key (type: string) - outputColumnNames: _col0, _col7 - input vertices: - 0 Map 1 - Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col7 (type: array) - outputColumnNames: _col0, _col1 + Select Operator + expressions: key (type: string), value (type: array) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2 + input vertices: + 0 Map 1 Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE - Lateral View Forward + Select Operator + expressions: _col0 (type: string), _col2 (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: array) - outputColumnNames: _col0, _col1 + Lateral View Forward Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE - Lateral View Join Operator - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 27772 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 6 Data size: 27772 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Select Operator - expressions: _col1 (type: array) - outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE - UDTF Operator + Select Operator + expressions: _col0 (type: string), _col1 (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE - function name: explode Lateral View Join Operator outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 27772 Basic stats: COMPLETE Column stats: NONE @@ -136,6 +127,23 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Select Operator + expressions: _col1 (type: array) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE + UDTF Operator + Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 27772 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 6 Data size: 27772 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe Execution mode: llap LLAP IO: all inputs diff --git a/ql/src/test/results/clientpositive/llap/tablevalues.q.out b/ql/src/test/results/clientpositive/llap/tablevalues.q.out index de228e14bc04..3aa1a3639c33 100644 --- a/ql/src/test/results/clientpositive/llap/tablevalues.q.out +++ b/ql/src/test/results/clientpositive/llap/tablevalues.q.out @@ -184,30 +184,6 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 ListSink -PREHOOK: query: EXPLAIN CBO -SELECT tf.col1, tf.col2, tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t - LATERAL VIEW - INLINE(array(struct('A', 10, t.key),struct('B', 20, t.key))) tf AS col1, col2, col3 -PREHOOK: type: QUERY -PREHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -POSTHOOK: query: EXPLAIN CBO -SELECT tf.col1, tf.col2, tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t - LATERAL VIEW - INLINE(array(struct('A', 10, t.key),struct('B', 20, t.key))) tf AS col1, col2, col3 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -CBO PLAN: -HiveProject(col1=[$2], col2=[$3], col3=[$4]) - HiveTableFunctionScan(invocation=[inline(ARRAY(ROW($0, $1, _UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, $0), ROW($0, $1, _UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, $0)))], rowType=[RecordType(VARCHAR(2147483647) key, VARCHAR(2147483647) value, VARCHAR(2147483647) col1, INTEGER col2, VARCHAR(2147483647) col3)]) - HiveProject(key=[$0], value=[$1]) - HiveTableScan(table=[[default, mytbl_n1]], table:alias=[mytbl_n1]) - PREHOOK: query: SELECT tf.col1, tf.col2, tf.col3 FROM (SELECT key, value FROM mytbl_n1) t @@ -265,20 +241,6 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 ListSink -PREHOOK: query: EXPLAIN CBO -SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3) -PREHOOK: type: QUERY -PREHOOK: Input: _dummy_database@_dummy_table -#### A masked pattern was here #### -POSTHOOK: query: EXPLAIN CBO -SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3) -POSTHOOK: type: QUERY -POSTHOOK: Input: _dummy_database@_dummy_table -#### A masked pattern was here #### -CBO PLAN: -HiveTableFunctionScan(invocation=[inline(ARRAY(ROW(_UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, 30), ROW(_UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, 30)))], rowType=[RecordType(VARCHAR(2147483647) col1, INTEGER col2, INTEGER col3)]) - HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) - PREHOOK: query: SELECT INLINE(array(struct('A', 10, 30),struct('B', 20, 30))) AS (col1, col2, col3) PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table @@ -724,28 +686,6 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 ListSink -PREHOOK: query: EXPLAIN CBO -SELECT tf.col1, tf.col2, tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3) -PREHOOK: type: QUERY -PREHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -POSTHOOK: query: EXPLAIN CBO -SELECT tf.col1, tf.col2, tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3) -POSTHOOK: type: QUERY -POSTHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -CBO PLAN: -HiveProject(col1=[$2], col2=[$3], col3=[$4]) - HiveTableFunctionScan(invocation=[inline(ARRAY(ROW($0, $1, _UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, $0), ROW($0, $1, _UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, $0)))], rowType=[RecordType(VARCHAR(2147483647) key, VARCHAR(2147483647) value, VARCHAR(2147483647) col1, INTEGER col2, VARCHAR(2147483647) col3)]) - HiveProject(key=[$0], value=[$1]) - HiveTableScan(table=[[default, mytbl_n1]], table:alias=[mytbl_n1]) - PREHOOK: query: SELECT tf.col1, tf.col2, tf.col3 FROM (SELECT key, value FROM mytbl_n1) t, @@ -806,28 +746,6 @@ STAGE PLANS: outputColumnNames: _col0 ListSink -PREHOOK: query: EXPLAIN CBO -SELECT t.key -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf -PREHOOK: type: QUERY -PREHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -POSTHOOK: query: EXPLAIN CBO -SELECT t.key -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf -POSTHOOK: type: QUERY -POSTHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -CBO PLAN: -HiveProject(key=[$0]) - HiveTableFunctionScan(invocation=[inline(ARRAY(ROW($0, $1, _UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, $0), ROW($0, $1, _UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, $0)))], rowType=[RecordType(VARCHAR(2147483647) key, VARCHAR(2147483647) value, VARCHAR(2147483647) col1, INTEGER col2, VARCHAR(2147483647) col3)]) - HiveProject(key=[$0], value=[$1]) - HiveTableScan(table=[[default, mytbl_n1]], table:alias=[mytbl_n1]) - PREHOOK: query: SELECT t.key FROM (SELECT key, value FROM mytbl_n1) t, @@ -888,28 +806,6 @@ STAGE PLANS: outputColumnNames: _col0 ListSink -PREHOOK: query: EXPLAIN CBO -SELECT tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3) -PREHOOK: type: QUERY -PREHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -POSTHOOK: query: EXPLAIN CBO -SELECT tf.col3 -FROM - (SELECT key, value FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.key),('B', 20, t.key)) AS tf(col1, col2, col3) -POSTHOOK: type: QUERY -POSTHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -CBO PLAN: -HiveProject(col3=[$4]) - HiveTableFunctionScan(invocation=[inline(ARRAY(ROW($0, $1, _UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, $0), ROW($0, $1, _UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, $0)))], rowType=[RecordType(VARCHAR(2147483647) key, VARCHAR(2147483647) value, VARCHAR(2147483647) col1, INTEGER col2, VARCHAR(2147483647) col3)]) - HiveProject(key=[$0], value=[$1]) - HiveTableScan(table=[[default, mytbl_n1]], table:alias=[mytbl_n1]) - PREHOOK: query: SELECT tf.col3 FROM (SELECT key, value FROM mytbl_n1) t, @@ -1027,28 +923,6 @@ STAGE PLANS: Processor Tree: ListSink -PREHOOK: query: EXPLAIN CBO -SELECT tf.col3 -FROM - (SELECT row_number() over (order by key desc) as r FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.r),('B', 20, t.r)) AS tf(col1, col2, col3) -PREHOOK: type: QUERY -PREHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -POSTHOOK: query: EXPLAIN CBO -SELECT tf.col3 -FROM - (SELECT row_number() over (order by key desc) as r FROM mytbl_n1) t, - LATERAL TABLE(VALUES('A', 10, t.r),('B', 20, t.r)) AS tf(col1, col2, col3) -POSTHOOK: type: QUERY -POSTHOOK: Input: default@mytbl_n1 -#### A masked pattern was here #### -CBO PLAN: -HiveProject(col3=[$3]) - HiveTableFunctionScan(invocation=[inline(ARRAY(ROW($0, _UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 10, $0), ROW($0, _UTF-16LE'B':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", 20, $0)))], rowType=[RecordType(INTEGER r, VARCHAR(2147483647) col1, INTEGER col2, INTEGER col3)]) - HiveProject(r=[row_number() OVER (PARTITION BY 0 ORDER BY $0 DESC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)]) - HiveTableScan(table=[[default, mytbl_n1]], table:alias=[mytbl_n1]) - PREHOOK: query: SELECT tf.col3 FROM (SELECT row_number() over (order by key desc) as r FROM mytbl_n1) t, diff --git a/ql/src/test/results/clientpositive/llap/union26.q.out b/ql/src/test/results/clientpositive/llap/union26.q.out index 37b44beff86d..234d738e16aa 100644 --- a/ql/src/test/results/clientpositive/llap/union26.q.out +++ b/ql/src/test/results/clientpositive/llap/union26.q.out @@ -73,13 +73,17 @@ STAGE PLANS: Filter Operator predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: value (type: string) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) Execution mode: vectorized, llap LLAP IO: all inputs Map 5 @@ -91,12 +95,16 @@ STAGE PLANS: Filter Operator predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs Map 6 @@ -105,61 +113,65 @@ STAGE PLANS: alias: srcpart filterExpr: ((ds = '2008-04-08') and (hr = '11')) (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Lateral View Forward + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: key, value + Lateral View Forward Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Lateral View Join Operator - outputColumnNames: _col0, _col1, _col8 - Statistics: Num rows: 500 Data size: 117000 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(1) - keys: _col0 (type: string), _col1 (type: string) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: bigint) - Select Operator - expressions: array(1,2,3) (type: array) - outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE - UDTF Operator - Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE - function name: explode + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Lateral View Join Operator outputColumnNames: _col0, _col1, _col8 Statistics: Num rows: 500 Data size: 117000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 500 Data size: 115500 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count(1) + aggregations: count() keys: _col0 (type: string), _col1 (type: string) minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: bigint) + Select Operator + expressions: array(1,2,3) (type: array) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE + UDTF Operator + Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col1, _col8 + Statistics: Num rows: 500 Data size: 117000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 115500 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col0 (type: string), _col1 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: bigint) Execution mode: llap LLAP IO: all inputs Reducer 2 @@ -169,23 +181,23 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 key (type: string) - 1 key (type: string) + 0 _col0 (type: string) + 1 _col0 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator - aggregations: count(1) + aggregations: count() keys: _col0 (type: string), _col1 (type: string) minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: bigint) Reducer 4 Execution mode: vectorized, llap @@ -195,14 +207,14 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col2 (type: bigint), _col0 (type: string), _col1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 645 Data size: 119970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 645 Data size: 154155 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git a/ql/src/test/results/clientpositive/llap/union_lateralview.q.out b/ql/src/test/results/clientpositive/llap/union_lateralview.q.out index b01aea9cddf3..3f3a89cda01b 100644 --- a/ql/src/test/results/clientpositive/llap/union_lateralview.q.out +++ b/ql/src/test/results/clientpositive/llap/union_lateralview.q.out @@ -75,11 +75,11 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: key (type: string), value (type: string), array(1,2,3) (type: array) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 500 Data size: 117000 Basic stats: COMPLETE Column stats: COMPLETE + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Lateral View Forward - Statistics: Num rows: 1000 Data size: 234000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: string) outputColumnNames: _col0, _col1 @@ -99,7 +99,7 @@ STAGE PLANS: Statistics: Num rows: 1000 Data size: 231000 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col2 (type: string) Select Operator - expressions: _col2 (type: array) + expressions: Const array [1, 2, 3] (type: array) outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 56000 Basic stats: COMPLETE Column stats: COMPLETE UDTF Operator @@ -128,11 +128,11 @@ STAGE PLANS: filterExpr: ((ds = '2008-04-08') and (hr = '12')) (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: key (type: string), value (type: string), array(1,2,3) (type: array) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 500 Data size: 117000 Basic stats: COMPLETE Column stats: COMPLETE + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE Lateral View Forward - Statistics: Num rows: 1000 Data size: 234000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: string) outputColumnNames: _col0, _col1 @@ -152,7 +152,7 @@ STAGE PLANS: Statistics: Num rows: 1000 Data size: 231000 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col2 (type: string) Select Operator - expressions: _col2 (type: array) + expressions: Const array [1, 2, 3] (type: array) outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 56000 Basic stats: COMPLETE Column stats: COMPLETE UDTF Operator @@ -178,13 +178,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b + filterExpr: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Filter Operator + predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs Reducer 3 @@ -195,16 +203,16 @@ STAGE PLANS: Left Outer Join 0 to 1 keys: 0 _col1 (type: string) - 1 key (type: string) + 1 _col0 (type: string) outputColumnNames: _col0, _col2, _col3 - Statistics: Num rows: 1552 Data size: 234527 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1582 Data size: 287924 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: UDFToInteger(_col3) (type: int), _col0 (type: int), _col2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1552 Data size: 147440 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1582 Data size: 150290 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1552 Data size: 147440 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1582 Data size: 150290 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -213,7 +221,7 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) outputColumnNames: key, arr_ele, value - Statistics: Num rows: 1552 Data size: 147440 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1582 Data size: 150290 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(arr_ele), max(arr_ele), count(arr_ele), compute_bit_vector_hll(arr_ele), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value) minReductionHashAggr: 0.99 diff --git a/ql/src/test/results/clientpositive/llap/unionall_lateralview.q.out b/ql/src/test/results/clientpositive/llap/unionall_lateralview.q.out index 4b246f42d138..d70e28e09bfe 100644 --- a/ql/src/test/results/clientpositive/llap/unionall_lateralview.q.out +++ b/ql/src/test/results/clientpositive/llap/unionall_lateralview.q.out @@ -35,7 +35,7 @@ POSTHOOK: query: CREATE TABLE unionall_lateralview2(col1 INT) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@unionall_lateralview2 -Warning: Shuffle Join MERGEJOIN[38][tables = [x1, expdobj]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product PREHOOK: query: INSERT INTO unionall_lateralview2 SELECT 1 AS `col1` FROM unionall_lateralview1 diff --git a/ql/src/test/results/clientpositive/nonmr_fetch.q.out b/ql/src/test/results/clientpositive/nonmr_fetch.q.out index b14280320b46..907025078c17 100644 --- a/ql/src/test/results/clientpositive/nonmr_fetch.q.out +++ b/ql/src/test/results/clientpositive/nonmr_fetch.q.out @@ -1066,41 +1066,45 @@ STAGE PLANS: alias: srcpart filterExpr: ((ds = '2008-04-08') and (hr = '11')) (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Lateral View Forward + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string) - outputColumnNames: key - Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Lateral View Join Operator - outputColumnNames: _col0, _col8 - Statistics: Num rows: 500 Data size: 1003500 Basic stats: COMPLETE Column stats: COMPLETE - Limit - Number of rows: 20 - Statistics: Num rows: 20 Data size: 1740 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: string), _col8 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 20 Data size: 1740 Basic stats: COMPLETE Column stats: COMPLETE - ListSink - Select Operator - expressions: array(key,value) (type: array) - outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 960000 Basic stats: COMPLETE Column stats: COMPLETE - UDTF Operator - Statistics: Num rows: 500 Data size: 960000 Basic stats: COMPLETE Column stats: COMPLETE - function name: explode + Lateral View Forward + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE Lateral View Join Operator outputColumnNames: _col0, _col8 Statistics: Num rows: 500 Data size: 1003500 Basic stats: COMPLETE Column stats: COMPLETE Limit Number of rows: 20 - Statistics: Num rows: 20 Data size: 1740 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 20 Data size: 40080 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col8 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 20 Data size: 1740 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 20 Data size: 40080 Basic stats: COMPLETE Column stats: COMPLETE ListSink + Select Operator + expressions: array(_col0,_col1) (type: array) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 960000 Basic stats: COMPLETE Column stats: COMPLETE + UDTF Operator + Statistics: Num rows: 500 Data size: 960000 Basic stats: COMPLETE Column stats: COMPLETE + function name: explode + Lateral View Join Operator + outputColumnNames: _col0, _col8 + Statistics: Num rows: 500 Data size: 1003500 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 20 + Statistics: Num rows: 20 Data size: 40080 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col8 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 20 Data size: 40080 Basic stats: COMPLETE Column stats: COMPLETE + ListSink PREHOOK: query: select key,X from srcpart lateral view explode(array(key,value)) L as x where (ds='2008-04-08' AND hr='11') limit 20 PREHOOK: type: QUERY diff --git a/ql/src/test/results/clientpositive/skewjoinopt10.q.out b/ql/src/test/results/clientpositive/skewjoinopt10.q.out index 6cb47d0268fd..93d63a3c5b3e 100644 --- a/ql/src/test/results/clientpositive/skewjoinopt10.q.out +++ b/ql/src/test/results/clientpositive/skewjoinopt10.q.out @@ -67,12 +67,16 @@ STAGE PLANS: Filter Operator predicate: (key is not null and (key = '8')) (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE TableScan alias: b filterExpr: key is not null (type: boolean) @@ -80,24 +84,28 @@ STAGE PLANS: Filter Operator predicate: (key is not null and (key = '8')) (type: boolean) Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string), value (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE - value expressions: value (type: array) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: array) Reduce Operator Tree: Join Operator condition map: Inner Join 0 to 1 keys: - 0 key (type: string) - 1 key (type: string) - outputColumnNames: _col0, _col7 + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2 Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), _col7 (type: array) + expressions: _col0 (type: string), _col2 (type: array) outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 13886 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -192,36 +200,44 @@ STAGE PLANS: Filter Operator predicate: (key is not null and (not (key = '8'))) (type: boolean) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE TableScan alias: b Statistics: Num rows: 6 Data size: 12624 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (not (key = '8'))) (type: boolean) Statistics: Num rows: 1 Data size: 2104 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: key (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: key (type: string) + Select Operator + expressions: key (type: string), value (type: array) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 2104 Basic stats: COMPLETE Column stats: NONE - value expressions: value (type: array) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 2104 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: array) Reduce Operator Tree: Join Operator condition map: Inner Join 0 to 1 keys: - 0 key (type: string) - 1 key (type: string) - outputColumnNames: _col0, _col7 + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 202 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), _col7 (type: array) + expressions: _col0 (type: string), _col2 (type: array) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 202 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out index 726d0d3b1628..fda3188cacb9 100644 --- a/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out +++ b/ql/src/test/results/clientpositive/tez/explainanalyze_3.q.out @@ -34,7 +34,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 POSTHOOK: Output: hdfs://### HDFS PATH ### -Plan not optimized by CBO because the statement has lateral views +Plan optimized by CBO. Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out index b5783c259207..cc2adc2fc676 100644 --- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out +++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out @@ -72,7 +72,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 POSTHOOK: Output: hdfs://### HDFS PATH ### -Plan not optimized by CBO because the statement has lateral views +Plan optimized by CBO. Stage-0 Fetch Operator diff --git a/ql/src/test/results/clientpositive/tez/tez_union_udtf.q.out b/ql/src/test/results/clientpositive/tez/tez_union_udtf.q.out index 8d72ef232627..1a1c7fabbf1a 100644 --- a/ql/src/test/results/clientpositive/tez/tez_union_udtf.q.out +++ b/ql/src/test/results/clientpositive/tez/tez_union_udtf.q.out @@ -22,7 +22,7 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@src1 POSTHOOK: Output: database:default POSTHOOK: Output: default@x -Plan not optimized by CBO because the statement has lateral views +Plan optimized by CBO. Vertex dependency in root stage Map 1 <- Union 2 (CONTAINS) @@ -41,40 +41,39 @@ Stage-3 Conditional Operator Stage-1 Reducer 3 vectorized - File Output Operator [FS_60] - Select Operator [SEL_59] (rows=1 width=530) + File Output Operator [FS_59] + Select Operator [SEL_58] (rows=1 width=530) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] - Group By Operator [GBY_58] (rows=1 width=332) + Group By Operator [GBY_57] (rows=1 width=332) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] <-Union 2 [CUSTOM_SIMPLE_EDGE] <-Map 1 [CONTAINS] vectorized - File Output Operator [FS_54] + File Output Operator [FS_53] table:{"name:":"default.x"} - Select Operator [SEL_53] (rows=4 width=91) + Select Operator [SEL_52] (rows=2 width=91) Output:["_col0","_col1"] - Select Operator [SEL_52] (rows=2 width=4) - Output:["_col1"] - Filter Operator [FIL_51] (rows=2 width=87) - predicate:(key = '238') - TableScan [TS_26] (rows=500 width=87) - default@src,src,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] - Reduce Output Operator [RS_57] - Group By Operator [GBY_56] (rows=1 width=400) + Filter Operator [FIL_51] (rows=2 width=87) + predicate:(key = '238') + TableScan [TS_27] (rows=500 width=87) + default@src,src,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] + Reduce Output Operator [RS_56] + Group By Operator [GBY_55] (rows=1 width=400) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["max(length(col1))","avg(COALESCE(length(col1),0))","count(1)","count(col1)","compute_bit_vector_hll(col1)","min(col2)","max(col2)","count(col2)","compute_bit_vector_hll(col2)"] - Select Operator [SEL_55] (rows=4 width=91) + Select Operator [SEL_54] (rows=4 width=136) Output:["col1","col2"] - Please refer to the previous Select Operator [SEL_53] + Please refer to the previous Select Operator [SEL_52] <-Map 4 [CONTAINS] File Output Operator [FS_45] table:{"name:":"default.x"} - Select Operator [SEL_44] (rows=4 width=91) + Select Operator [SEL_43] (rows=2 width=132) Output:["_col0","_col1"] - Select Operator [SEL_42] (rows=2 width=8) - Output:["_col1"] - Lateral View Join Operator [LVJ_40] (rows=2 width=56) - Output:["_col6"] - Select Operator [SEL_38] (rows=2 width=8) - Lateral View Forward [LVF_37] (rows=2 width=86) + Lateral View Join Operator [LVJ_41] (rows=2 width=134) + Output:["_col0","_col6"] + Select Operator [SEL_39] (rows=2 width=86) + Output:["_col0"] + Lateral View Forward [LVF_38] (rows=2 width=86) + Select Operator [SEL_37] (rows=2 width=86) + Output:["_col0"] Filter Operator [FIL_36] (rows=2 width=86) predicate:(key = '238') TableScan [TS_35] (rows=25 width=86) @@ -82,28 +81,26 @@ Stage-3 Reduce Output Operator [RS_48] Group By Operator [GBY_47] (rows=1 width=400) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["max(length(col1))","avg(COALESCE(length(col1),0))","count(1)","count(col1)","compute_bit_vector_hll(col1)","min(col2)","max(col2)","count(col2)","compute_bit_vector_hll(col2)"] - Select Operator [SEL_46] (rows=4 width=91) + Select Operator [SEL_46] (rows=4 width=136) Output:["col1","col2"] - Please refer to the previous Select Operator [SEL_44] + Please refer to the previous Select Operator [SEL_43] File Output Operator [FS_45] table:{"name:":"default.x"} - Select Operator [SEL_44] (rows=4 width=91) + Select Operator [SEL_43] (rows=2 width=132) Output:["_col0","_col1"] - Select Operator [SEL_42] (rows=2 width=8) - Output:["_col1"] - Lateral View Join Operator [LVJ_40] (rows=2 width=56) - Output:["_col6"] - UDTF Operator [UDTF_41] (rows=2 width=48) - function name:explode - Select Operator [SEL_39] (rows=2 width=48) - Output:["_col0"] - Please refer to the previous Lateral View Forward [LVF_37] + Lateral View Join Operator [LVJ_41] (rows=2 width=134) + Output:["_col0","_col6"] + UDTF Operator [UDTF_42] (rows=2 width=48) + function name:explode + Select Operator [SEL_40] (rows=2 width=48) + Output:["_col0"] + Please refer to the previous Lateral View Forward [LVF_38] Reduce Output Operator [RS_48] Group By Operator [GBY_47] (rows=1 width=400) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["max(length(col1))","avg(COALESCE(length(col1),0))","count(1)","count(col1)","compute_bit_vector_hll(col1)","min(col2)","max(col2)","count(col2)","compute_bit_vector_hll(col2)"] - Select Operator [SEL_46] (rows=4 width=91) + Select Operator [SEL_46] (rows=4 width=136) Output:["col1","col2"] - Please refer to the previous Select Operator [SEL_44] + Please refer to the previous Select Operator [SEL_43] Stage-4(CONDITIONAL) File Merge Please refer to the previous Stage-8(CONDITIONAL CHILD TASKS: Stage-5, Stage-4, Stage-6) @@ -140,7 +137,7 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@src1 POSTHOOK: Output: database:default POSTHOOK: Output: default@x -POSTHOOK: Lineage: x.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), (src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: x.key EXPRESSION [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: x.tag EXPRESSION [] PREHOOK: query: SELECT * FROM x PREHOOK: type: QUERY From f5d089138ee3eec762111265b5377223019a0f89 Mon Sep 17 00:00:00 2001 From: rkirtir <111496669+rkirtir@users.noreply.github.com> Date: Mon, 9 Oct 2023 13:23:29 +0530 Subject: [PATCH 002/179] HIVE-23680 : TestDbNotificationListener is unstable (Kirti Ruge, reviewed by Laszlo Vegh) --- hcatalog/webhcat/java-client/pom.xml | 5 + itests/hcatalog-unit/pom.xml | 1 + .../listener/TestDbNotificationCleanup.java | 189 +++++++++++++++ .../listener/TestDbNotificationListener.java | 218 +++--------------- ...stTransactionalDbNotificationListener.java | 169 ++++++++++++++ 5 files changed, 391 insertions(+), 191 deletions(-) create mode 100644 itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationCleanup.java create mode 100644 itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java diff --git a/hcatalog/webhcat/java-client/pom.xml b/hcatalog/webhcat/java-client/pom.xml index 7fbc2244a1e9..48488d7e361e 100644 --- a/hcatalog/webhcat/java-client/pom.xml +++ b/hcatalog/webhcat/java-client/pom.xml @@ -110,6 +110,11 @@ tests test + + org.mockito + mockito-core + ${mockito-core.version} + ${basedir}/src/main/java diff --git a/itests/hcatalog-unit/pom.xml b/itests/hcatalog-unit/pom.xml index 5183a2ba8026..4a049697b712 100644 --- a/itests/hcatalog-unit/pom.xml +++ b/itests/hcatalog-unit/pom.xml @@ -89,6 +89,7 @@ org.apache.hive hive-standalone-metastore-server + tests test diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationCleanup.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationCleanup.java new file mode 100644 index 000000000000..e73a57ea58f5 --- /dev/null +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationCleanup.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hive.hcatalog.listener; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import java.util.concurrent.TimeUnit; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient.NotificationFilter; +import org.apache.hadoop.hive.metastore.api.NotificationEventRequest; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer; +import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageEncoder; +import org.apache.hadoop.hive.ql.DriverFactory; +import org.apache.hadoop.hive.ql.IDriver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.AfterClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.EVENT_DB_LISTENER_CLEAN_STARTUP_WAIT_INTERVAL; + + + +public class TestDbNotificationCleanup { + private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationCleanup.class + .getName()); + private static final int EVENTS_TTL = 30; + private static final int CLEANUP_SLEEP_TIME = 10; + private static Map emptyParameters = new HashMap(); + private static IMetaStoreClient msClient; + private static IDriver driver; + private static MessageDeserializer md; + private static HiveConf conf; + + private long firstEventId; + private final String testTempDir = Paths.get(System.getProperty("java.io.tmpdir"), "testDbNotif").toString(); + @SuppressWarnings("rawtypes") + @BeforeClass + public static void connectToMetastore() throws Exception { + conf = new HiveConf(); + + MetastoreConf.setVar(conf,MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS, + "org.apache.hive.hcatalog.listener.DbNotificationListener"); + conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); + conf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true); + conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL, DummyRawStoreFailEvent.class.getName()); + MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EVENT_MESSAGE_FACTORY, JSONMessageEncoder.class.getName()); + MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.EVENT_DB_LISTENER_CLEAN_INTERVAL, CLEANUP_SLEEP_TIME, TimeUnit.SECONDS); + MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.EVENT_DB_LISTENER_TTL, EVENTS_TTL, TimeUnit.SECONDS); + MetastoreConf.setTimeVar(conf, EVENT_DB_LISTENER_CLEAN_STARTUP_WAIT_INTERVAL, 20, TimeUnit.SECONDS); + conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, + "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); + SessionState.start(new CliSessionState(conf)); + msClient = new HiveMetaStoreClient(conf); + driver = DriverFactory.newDriver(conf); + + } + + @Before + public void setup() throws Exception { + firstEventId = msClient.getCurrentNotificationEventId().getEventId(); + DummyRawStoreFailEvent.setEventSucceed(true); + } + + @AfterClass + public static void tearDownAfterClass() { + + if (msClient != null) { + msClient.close(); + } + if (driver != null) { + driver.close(); + } + conf = null; + } + + + @Test + public void cleanupNotifs() throws Exception { + Database db = new Database("cleanup1", "no description", testTempDir, emptyParameters); + msClient.createDatabase(db); + msClient.dropDatabase("cleanup1"); + + LOG.info("Pulling events immediately after createDatabase/dropDatabase"); + NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(2, rsp.getEventsSize()); + + // sleep for expiry time, and then fetch again + // sleep twice the TTL interval - things should have been cleaned by then. + Thread.sleep(EVENTS_TTL * 2 * 1000); + + LOG.info("Pulling events again after cleanup"); + NotificationEventResponse rsp2 = msClient.getNextNotification(firstEventId, 0, null); + LOG.info("second trigger done"); + assertEquals(0, rsp2.getEventsSize()); + } + + /** + * Test makes sure that if you use the API {@link HiveMetaStoreClient#getNextNotification(NotificationEventRequest, boolean, NotificationFilter)} + * does not error out if the events are cleanedup. + */ + @Test + public void skipCleanedUpEvents() throws Exception { + Database db = new Database("cleanup1", "no description", testTempDir, emptyParameters); + msClient.createDatabase(db); + msClient.dropDatabase("cleanup1"); + + // sleep for expiry time, and then fetch again + // sleep twice the TTL interval - things should have been cleaned by then. + Thread.sleep(EVENTS_TTL * 2 * 1000); + + db = new Database("cleanup2", "no description", testTempDir, emptyParameters); + msClient.createDatabase(db); + msClient.dropDatabase("cleanup2"); + + // the firstEventId is before the cleanup happened, so we should just receive the + // events which remaining after cleanup. + NotificationEventRequest request = new NotificationEventRequest(); + request.setLastEvent(firstEventId); + request.setMaxEvents(-1); + NotificationEventResponse rsp2 = msClient.getNextNotification(request, true, null); + assertEquals(2, rsp2.getEventsSize()); + // when we pass the allowGapsInEvents as false the API should error out + Exception ex = null; + try { + NotificationEventResponse rsp = msClient.getNextNotification(request, false, null); + } catch (Exception e) { + ex = e; + } + assertNotNull(ex); + } + + @Test + public void cleanupNotificationWithError() throws Exception { + Database db = new Database("cleanup1", "no description", testTempDir, emptyParameters); + msClient.createDatabase(db); + msClient.dropDatabase("cleanup1"); + + LOG.info("Pulling events immediately after createDatabase/dropDatabase"); + NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(2, rsp.getEventsSize()); + //this simulates that cleaning thread will error out while cleaning the notifications + DummyRawStoreFailEvent.setEventSucceed(false); + // sleep for expiry time, and then fetch again + // sleep twice the TTL interval - things should have been cleaned by then. + Thread.sleep(EVENTS_TTL * 2 * 1000); + + LOG.info("Pulling events again after failing to cleanup"); + NotificationEventResponse rsp2 = msClient.getNextNotification(firstEventId, 0, null); + LOG.info("second trigger done"); + assertEquals(2, rsp2.getEventsSize()); + DummyRawStoreFailEvent.setEventSucceed(true); + Thread.sleep(EVENTS_TTL * 2 * 1000); + + LOG.info("Pulling events again after cleanup"); + rsp2 = msClient.getNextNotification(firstEventId, 0, null); + LOG.info("third trigger done"); + assertEquals(0, rsp2.getEventsSize()); + } +} + + diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java index 100ee24e1fa5..46f4f2da8335 100644 --- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java @@ -19,7 +19,6 @@ package org.apache.hive.hcatalog.listener; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; @@ -29,7 +28,6 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -43,7 +41,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient.NotificationFilter; import org.apache.hadoop.hive.metastore.MetaStoreEventListener; import org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants; import org.apache.hadoop.hive.metastore.TableType; @@ -57,7 +54,6 @@ import org.apache.hadoop.hive.metastore.api.InsertEventRequestData; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.api.NotificationEventRequest; import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest; import org.apache.hadoop.hive.metastore.api.Partition; @@ -67,12 +63,11 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TxnType; -import org.apache.hadoop.hive.metastore.client.builder.TableBuilder; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.events.AddPartitionEvent; import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent; import org.apache.hadoop.hive.metastore.events.AlterTableEvent; +import org.apache.hadoop.hive.metastore.events.AlterDatabaseEvent; import org.apache.hadoop.hive.metastore.events.BatchAcidWriteEvent; import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent; import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent; @@ -82,9 +77,6 @@ import org.apache.hadoop.hive.metastore.events.DropPartitionEvent; import org.apache.hadoop.hive.metastore.events.DropTableEvent; import org.apache.hadoop.hive.metastore.events.InsertEvent; -import org.apache.hadoop.hive.metastore.events.OpenTxnEvent; -import org.apache.hadoop.hive.metastore.events.CommitTxnEvent; -import org.apache.hadoop.hive.metastore.events.AbortTxnEvent; import org.apache.hadoop.hive.metastore.events.ListenerEvent; import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent; import org.apache.hadoop.hive.metastore.events.AcidWriteEvent; @@ -110,6 +102,7 @@ import org.apache.hive.hcatalog.api.repl.ReplicationV1CompatRule; import org.apache.hive.hcatalog.data.Pair; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -118,14 +111,14 @@ import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.junit.Ignore; + /** * Tests DbNotificationListener when used as a transactional event listener * (hive.metastore.transactional.event.listeners) */ -@org.junit.Ignore("TestDbNotificationListener is unstable HIVE-23680") -public class TestDbNotificationListener { +public class TestDbNotificationListener +{ private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationListener.class .getName()); private static final int EVENTS_TTL = 30; @@ -211,6 +204,11 @@ public void onAlterTable (AlterTableEvent tableEvent) throws MetaException { pushEventId(EventType.ALTER_TABLE, tableEvent); } + @Override + public void onAlterDatabase(AlterDatabaseEvent dbEvent) throws MetaException { + pushEventId(EventType.ALTER_DATABASE, dbEvent); + } + @Override public void onAddPartition (AddPartitionEvent partitionEvent) throws MetaException { pushEventId(EventType.ADD_PARTITION, partitionEvent); @@ -251,18 +249,6 @@ public void onInsert(InsertEvent insertEvent) throws MetaException { pushEventId(EventType.INSERT, insertEvent); } - public void onOpenTxn(OpenTxnEvent openTxnEvent) throws MetaException { - pushEventId(EventType.OPEN_TXN, openTxnEvent); - } - - public void onCommitTxn(CommitTxnEvent commitTxnEvent) throws MetaException { - pushEventId(EventType.COMMIT_TXN, commitTxnEvent); - } - - public void onAbortTxn(AbortTxnEvent abortTxnEvent) throws MetaException { - pushEventId(EventType.ABORT_TXN, abortTxnEvent); - } - public void onAllocWriteId(AllocWriteIdEvent allocWriteIdEvent) throws MetaException { pushEventId(EventType.ALLOC_WRITE_ID, allocWriteIdEvent); } @@ -312,6 +298,18 @@ public void setup() throws Exception { DummyRawStoreFailEvent.setEventSucceed(true); } + @AfterClass + public static void tearDownAfterClass() { + + if (msClient != null) { + msClient.close(); + } + if (driver != null) { + driver.close(); + } + + } + @After public void tearDown() { MockMetaStoreEventListener.clearEvents(); @@ -392,6 +390,8 @@ public void alterDatabase() throws Exception { String newDesc = "test database"; Database dbAfter = dbBefore.deepCopy(); dbAfter.setDescription(newDesc); + dbAfter.setOwnerName("test2"); + dbAfter.setOwnerType(PrincipalType.USER); msClient.alterDatabase(dbName, dbAfter); dbAfter = msClient.getDatabase(dbName); @@ -1071,88 +1071,6 @@ public void dropFunction() throws Exception { testEventCounts(defaultDbName, firstEventId, null, null, 3); } - @Test - public void openTxn() throws Exception { - msClient.openTxn("me", TxnType.READ_ONLY); - NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(0, rsp.getEventsSize()); - - msClient.openTxn("me", TxnType.DEFAULT); - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); - - NotificationEvent event = rsp.getEvents().get(0); - assertEquals(firstEventId + 1, event.getEventId()); - assertTrue(event.getEventTime() >= startTime); - assertEquals(EventType.OPEN_TXN.toString(), event.getEventType()); - } - - @Test - public void abortTxn() throws Exception { - long txnId1 = msClient.openTxn("me", TxnType.READ_ONLY); - long txnId2 = msClient.openTxn("me", TxnType.DEFAULT); - - NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); - - msClient.abortTxns(Collections.singletonList(txnId1)); - rsp = msClient.getNextNotification(firstEventId + 1, 0, null); - assertEquals(0, rsp.getEventsSize()); - - msClient.abortTxns(Collections.singletonList(txnId2)); - rsp = msClient.getNextNotification(firstEventId + 1, 0, null); - assertEquals(1, rsp.getEventsSize()); - - NotificationEvent event = rsp.getEvents().get(0); - assertEquals(firstEventId + 2, event.getEventId()); - assertTrue(event.getEventTime() >= startTime); - assertEquals(EventType.ABORT_TXN.toString(), event.getEventType()); - } - - @Test - public void rollbackTxn() throws Exception { - long txnId1 = msClient.openTxn("me", TxnType.READ_ONLY); - long txnId2 = msClient.openTxn("me", TxnType.DEFAULT); - - NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); - - msClient.rollbackTxn(txnId1); - rsp = msClient.getNextNotification(firstEventId + 1, 0, null); - assertEquals(0, rsp.getEventsSize()); - - msClient.rollbackTxn(txnId2); - rsp = msClient.getNextNotification(firstEventId + 1, 0, null); - assertEquals(1, rsp.getEventsSize()); - - NotificationEvent event = rsp.getEvents().get(0); - assertEquals(firstEventId + 2, event.getEventId()); - assertTrue(event.getEventTime() >= startTime); - assertEquals(EventType.ABORT_TXN.toString(), event.getEventType()); - } - - @Test - public void commitTxn() throws Exception { - long txnId1 = msClient.openTxn("me", TxnType.READ_ONLY); - long txnId2 = msClient.openTxn("me", TxnType.DEFAULT); - - NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); - - msClient.commitTxn(txnId1); - rsp = msClient.getNextNotification(firstEventId + 1, 0, null); - assertEquals(0, rsp.getEventsSize()); - - msClient.commitTxn(txnId2); - rsp = msClient.getNextNotification(firstEventId + 1, 0, null); - assertEquals(1, rsp.getEventsSize()); - - NotificationEvent event = rsp.getEvents().get(0); - assertEquals(firstEventId + 2, event.getEventId()); - assertTrue(event.getEventTime() >= startTime); - assertEquals(EventType.COMMIT_TXN.toString(), event.getEventType()); - } - @Test public void insertTable() throws Exception { String defaultDbName = "default"; @@ -1420,7 +1338,6 @@ public boolean accept(NotificationEvent event) { } @Test - @Ignore("HIVE-23401") public void sqlInsertTable() throws Exception { String defaultDbName = "default"; String tblName = "sqlins"; @@ -1532,7 +1449,7 @@ public void sqlInsertPartition() throws Exception { // Event 5, 6, 7 driver.run("insert into table " + tblName + " partition (ds = 'today') values (2)"); // Event 8, 9, 10 - driver.run("insert into table " + tblName + " partition (ds) values (3, 'today')"); + driver.run("insert into table " + tblName + " partition (ds = 'today') values (3)"); // Event 9, 10 driver.run("alter table " + tblName + " add partition (ds = 'yesterday')"); @@ -1545,9 +1462,9 @@ public void sqlInsertPartition() throws Exception { // Event 10, 11, 12 driver.run("insert into table " + tblName + " partition (ds = 'yesterday') values (2)"); // Event 12, 13, 14 - driver.run("insert into table " + tblName + " partition (ds) values (3, 'yesterday')"); + driver.run("insert into table " + tblName + " partition (ds = 'yesterday') values (3)"); // Event 15, 16, 17 - driver.run("insert into table " + tblName + " partition (ds) values (3, 'tomorrow')"); + driver.run("insert into table " + tblName + " partition (ds = 'tomorrow') values (2)"); // Event 18 driver.run("alter table " + tblName + " drop partition (ds = 'tomorrow')"); // Event 19, 20, 21 @@ -1664,86 +1581,5 @@ private void verifyInsert(NotificationEvent event, String dbName, String tblName assertTrue(files.hasNext()); } - @Test - public void cleanupNotifs() throws Exception { - Database db = new Database("cleanup1", "no description", testTempDir, emptyParameters); - msClient.createDatabase(db); - msClient.dropDatabase("cleanup1"); - LOG.info("Pulling events immediately after createDatabase/dropDatabase"); - NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(2, rsp.getEventsSize()); - - // sleep for expiry time, and then fetch again - // sleep twice the TTL interval - things should have been cleaned by then. - Thread.sleep(EVENTS_TTL * 2 * 1000); - - LOG.info("Pulling events again after cleanup"); - NotificationEventResponse rsp2 = msClient.getNextNotification(firstEventId, 0, null); - LOG.info("second trigger done"); - assertEquals(0, rsp2.getEventsSize()); - } - - /** - * Test makes sure that if you use the API {@link HiveMetaStoreClient#getNextNotification(NotificationEventRequest, boolean, NotificationFilter)} - * does not error out if the events are cleanedup. - */ - @Test - public void skipCleanedUpEvents() throws Exception { - Database db = new Database("cleanup1", "no description", testTempDir, emptyParameters); - msClient.createDatabase(db); - msClient.dropDatabase("cleanup1"); - - // sleep for expiry time, and then fetch again - // sleep twice the TTL interval - things should have been cleaned by then. - Thread.sleep(EVENTS_TTL * 2 * 1000); - - db = new Database("cleanup2", "no description", testTempDir, emptyParameters); - msClient.createDatabase(db); - msClient.dropDatabase("cleanup2"); - - // the firstEventId is before the cleanup happened, so we should just receive the - // events which remaining after cleanup. - NotificationEventRequest request = new NotificationEventRequest(); - request.setLastEvent(firstEventId); - request.setMaxEvents(-1); - NotificationEventResponse rsp2 = msClient.getNextNotification(request, true, null); - assertEquals(2, rsp2.getEventsSize()); - // when we pass the allowGapsInEvents as false the API should error out - Exception ex = null; - try { - NotificationEventResponse rsp = msClient.getNextNotification(request, false, null); - } catch (Exception e) { - ex = e; - } - assertNotNull(ex); - } - - @Test - public void cleanupNotificationWithError() throws Exception { - Database db = new Database("cleanup1", "no description", testTempDir, emptyParameters); - msClient.createDatabase(db); - msClient.dropDatabase("cleanup1"); - - LOG.info("Pulling events immediately after createDatabase/dropDatabase"); - NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(2, rsp.getEventsSize()); - //this simulates that cleaning thread will error out while cleaning the notifications - DummyRawStoreFailEvent.setEventSucceed(false); - // sleep for expiry time, and then fetch again - // sleep twice the TTL interval - things should have been cleaned by then. - Thread.sleep(EVENTS_TTL * 2 * 1000); - - LOG.info("Pulling events again after failing to cleanup"); - NotificationEventResponse rsp2 = msClient.getNextNotification(firstEventId, 0, null); - LOG.info("second trigger done"); - assertEquals(2, rsp2.getEventsSize()); - DummyRawStoreFailEvent.setEventSucceed(true); - Thread.sleep(EVENTS_TTL * 2 * 1000); - - LOG.info("Pulling events again after cleanup"); - rsp2 = msClient.getNextNotification(firstEventId, 0, null); - LOG.info("third trigger done"); - assertEquals(0, rsp2.getEventsSize()); - } } diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java new file mode 100644 index 000000000000..3b9853684a49 --- /dev/null +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hive.hcatalog.listener; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import java.util.Collections; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; +import org.apache.hadoop.hive.metastore.api.TxnType; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType; +import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageEncoder; +import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; +import org.apache.hadoop.hive.ql.DriverFactory; +import org.apache.hadoop.hive.ql.IDriver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + + + +public class TestTransactionalDbNotificationListener { + private static IMetaStoreClient msClient; + private static IDriver driver; + + private int startTime; + private long firstEventId; + + + @SuppressWarnings("rawtypes") + @BeforeClass + public static void connectToMetastore() throws Exception { + HiveConf conf = new HiveConf(); + conf.setVar(HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS, + "org.apache.hive.hcatalog.listener.DbNotificationListener"); + conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); + conf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true); + conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL, DummyRawStoreFailEvent.class.getName()); + MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EVENT_MESSAGE_FACTORY, JSONMessageEncoder.class.getName()); + conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, + "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); + SessionState.start(new CliSessionState(conf)); + TestTxnDbUtil.setConfValues(conf); + TestTxnDbUtil.prepDb(conf); + msClient = new HiveMetaStoreClient(conf); + driver = DriverFactory.newDriver(conf); + + } + + @Before + public void setup() throws Exception { + long now = System.currentTimeMillis() / 1000; + startTime = 0; + if (now > Integer.MAX_VALUE) { + fail("Bummer, time has fallen over the edge"); + } else { + startTime = (int) now; + } + firstEventId = msClient.getCurrentNotificationEventId().getEventId(); + DummyRawStoreFailEvent.setEventSucceed(true); + } + + @Test + public void openTxn() throws Exception { + msClient.openTxn("me", TxnType.READ_ONLY); + NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(0, rsp.getEventsSize()); + + msClient.openTxn("me", TxnType.DEFAULT); + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); + + NotificationEvent event = rsp.getEvents().get(0); + assertEquals(firstEventId + 1, event.getEventId()); + assertTrue(event.getEventTime() >= startTime); + assertEquals(EventType.OPEN_TXN.toString(), event.getEventType()); + } + + @Test + public void abortTxn() throws Exception { + + long txnId1 = msClient.openTxn("me", TxnType.READ_ONLY); + long txnId2 = msClient.openTxn("me", TxnType.DEFAULT); + + NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); + + msClient.abortTxns(Collections.singletonList(txnId1)); + rsp = msClient.getNextNotification(firstEventId + 1, 0, null); + assertEquals(0, rsp.getEventsSize()); + + msClient.abortTxns(Collections.singletonList(txnId2)); + rsp = msClient.getNextNotification(firstEventId + 1, 0, null); + assertEquals(1, rsp.getEventsSize()); + + NotificationEvent event = rsp.getEvents().get(0); + assertEquals(firstEventId + 2, event.getEventId()); + assertTrue(event.getEventTime() >= startTime); + assertEquals(EventType.ABORT_TXN.toString(), event.getEventType()); + } + + @Test + public void rollbackTxn() throws Exception { + long txnId1 = msClient.openTxn("me", TxnType.READ_ONLY); + long txnId2 = msClient.openTxn("me", TxnType.DEFAULT); + + NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); + + msClient.rollbackTxn(txnId1); + rsp = msClient.getNextNotification(firstEventId + 1, 0, null); + assertEquals(0, rsp.getEventsSize()); + + msClient.rollbackTxn(txnId2); + rsp = msClient.getNextNotification(firstEventId + 1, 0, null); + assertEquals(1, rsp.getEventsSize()); + + NotificationEvent event = rsp.getEvents().get(0); + assertEquals(firstEventId + 2, event.getEventId()); + assertTrue(event.getEventTime() >= startTime); + assertEquals(EventType.ABORT_TXN.toString(), event.getEventType()); + } + + @Test + public void commitTxn() throws Exception { + long txnId1 = msClient.openTxn("me", TxnType.READ_ONLY); + long txnId2 = msClient.openTxn("me", TxnType.DEFAULT); + + NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); + + msClient.commitTxn(txnId1); + rsp = msClient.getNextNotification(firstEventId + 1, 0, null); + assertEquals(0, rsp.getEventsSize()); + + msClient.commitTxn(txnId2); + rsp = msClient.getNextNotification(firstEventId + 1, 0, null); + assertEquals(1, rsp.getEventsSize()); + + NotificationEvent event = rsp.getEvents().get(0); + assertEquals(firstEventId + 2, event.getEventId()); + assertTrue(event.getEventTime() >= startTime); + assertEquals(EventType.COMMIT_TXN.toString(), event.getEventType()); + } + +} \ No newline at end of file From 20be17d19c19a1482dc3b1b753ce1f342b940e36 Mon Sep 17 00:00:00 2001 From: InvisibleProgrammer Date: Mon, 9 Oct 2023 09:54:26 +0200 Subject: [PATCH 003/179] Remove Powermock - finishing touches (Zsolt Miskolczi, reviewed by Laszlo Vegh) --- beeline/pom.xml | 2 +- itests/hive-jmh/pom.xml | 2 +- jdbc-handler/pom.xml | 2 +- llap-client/pom.xml | 2 +- pom.xml | 2 +- ql/pom.xml | 2 +- service/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/beeline/pom.xml b/beeline/pom.xml index 91ddc52aa50a..71da41a32da8 100644 --- a/beeline/pom.xml +++ b/beeline/pom.xml @@ -183,7 +183,7 @@ org.mockito mockito-inline - 4.11.0 + ${mockito-inline.version} test diff --git a/itests/hive-jmh/pom.xml b/itests/hive-jmh/pom.xml index e3908d8e410f..defef3de685a 100644 --- a/itests/hive-jmh/pom.xml +++ b/itests/hive-jmh/pom.xml @@ -81,7 +81,7 @@ org.mockito mockito-inline - 4.11.0 + ${mockito-inline.version} diff --git a/jdbc-handler/pom.xml b/jdbc-handler/pom.xml index 6869503e9e89..20705c1d69bf 100644 --- a/jdbc-handler/pom.xml +++ b/jdbc-handler/pom.xml @@ -114,7 +114,7 @@ org.mockito mockito-inline - 4.11.0 + ${mockito-inline.version} test diff --git a/llap-client/pom.xml b/llap-client/pom.xml index 6a801e64309b..65e8fa22c7b8 100644 --- a/llap-client/pom.xml +++ b/llap-client/pom.xml @@ -133,7 +133,7 @@ org.mockito mockito-inline - 4.11.0 + ${mockito-inline.version} test diff --git a/pom.xml b/pom.xml index 2e124968f1a2..d7e5f90f2c24 100644 --- a/pom.xml +++ b/pom.xml @@ -184,7 +184,7 @@ 2.3 1.8.3 3.4.4 - 2.0.2 + 4.11.0 2.0.0-M5 4.1.77.Final 3.10.5.Final diff --git a/ql/pom.xml b/ql/pom.xml index 8878ce57ba16..30bd3e310359 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -809,7 +809,7 @@ org.mockito mockito-inline - 4.11.0 + ${mockito-inline.version} test diff --git a/service/pom.xml b/service/pom.xml index 2800983e6799..9d0a9d2f3f47 100644 --- a/service/pom.xml +++ b/service/pom.xml @@ -334,7 +334,7 @@ org.mockito mockito-inline - 4.11.0 + ${mockito-inline.version} test From dec006e81e657bc6edb0d57d31076f04e1f088c5 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 9 Oct 2023 16:48:11 +0530 Subject: [PATCH 004/179] HIVE-27771: Iceberg: Allow expire snapshot by time range. (#4776). (Ayush Saxena, reviewed by Denys Kuzmenko) --- .../mr/hive/HiveIcebergStorageHandler.java | 22 ++++++++++++++++- .../hive/TestHiveIcebergExpireSnapshots.java | 17 +++++++++++++ .../hadoop/hive/ql/parse/AlterClauseParser.g | 2 ++ .../execute/AlterTableExecuteAnalyzer.java | 24 +++++++++++++------ .../hive/ql/parse/AlterTableExecuteSpec.java | 19 ++++++++++++++- 5 files changed, 75 insertions(+), 9 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index bb9356df2517..4e451403b028 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -834,7 +834,10 @@ private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnap LOG.info("Executing expire snapshots on iceberg table {} with {} threads", icebergTable.name(), numThreads); deleteExecutorService = getDeleteExecutorService(icebergTable.name(), numThreads); } - if (expireSnapshotsSpec.isExpireByIds()) { + if (expireSnapshotsSpec.isExpireByTimestampRange()) { + expireSnapshotByTimestampRange(icebergTable, expireSnapshotsSpec.getFromTimestampMillis(), + expireSnapshotsSpec.getTimestampMillis(), deleteExecutorService); + } else if (expireSnapshotsSpec.isExpireByIds()) { expireSnapshotByIds(icebergTable, expireSnapshotsSpec.getIdsToExpire(), deleteExecutorService); } else { expireSnapshotOlderThanTimestamp(icebergTable, expireSnapshotsSpec.getTimestampMillis(), deleteExecutorService); @@ -846,6 +849,23 @@ private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnap } } + private void expireSnapshotByTimestampRange(Table icebergTable, Long fromTimestamp, Long toTimestamp, + ExecutorService deleteExecutorService) { + ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots(); + for (Snapshot snapshot : icebergTable.snapshots()) { + if (snapshot.timestampMillis() >= fromTimestamp && snapshot.timestampMillis() <= toTimestamp) { + expireSnapshots.expireSnapshotId(snapshot.snapshotId()); + LOG.debug("Expiring snapshot on {} with id: {} and timestamp: {}", icebergTable.name(), snapshot.snapshotId(), + snapshot.timestampMillis()); + } + } + LOG.info("Expiring snapshot on {} within time range {} -> {}", icebergTable.name(), fromTimestamp, toTimestamp); + if (deleteExecutorService != null) { + expireSnapshots.executeDeleteWith(deleteExecutorService); + } + expireSnapshots.commit(); + } + private void expireSnapshotOlderThanTimestamp(Table icebergTable, Long timestamp, ExecutorService deleteExecutorService) { ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots().expireOlderThan(timestamp); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java index 08277fb7892e..a851578ee6c6 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java @@ -20,6 +20,8 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; import org.apache.commons.collections4.IterableUtils; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -65,4 +67,19 @@ public void testExpireSnapshotsWithSnapshotId() throws IOException, InterruptedE table.refresh(); Assert.assertEquals(7, IterableUtils.size(table.snapshots())); } + + @Test + public void testExpireSnapshotsWithTimestampRange() throws IOException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "source"); + Table table = testTables.createTableWithVersions(shell, identifier.name(), + HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 10); + SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS000000"); + String fromTime = simpleDateFormat.format(new Date(table.history().get(5).timestampMillis())); + String toTime = simpleDateFormat.format(new Date(table.history().get(8).timestampMillis())); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE EXPIRE_SNAPSHOTS BETWEEN" + + " '" + fromTime + "' AND '" + toTime + "'"); + table.refresh(); + Assert.assertEquals(6, IterableUtils.size(table.snapshots())); + } } diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g index 48a2aed96173..46a00fe5c873 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g @@ -483,6 +483,8 @@ alterStatementSuffixExecute -> ^(TOK_ALTERTABLE_EXECUTE KW_FAST_FORWARD $sourceBranch $targetBranch?) | KW_EXECUTE KW_CHERRY_PICK snapshotId=Number -> ^(TOK_ALTERTABLE_EXECUTE KW_CHERRY_PICK $snapshotId) + | KW_EXECUTE KW_EXPIRE_SNAPSHOTS KW_BETWEEN (fromTimestamp=StringLiteral) KW_AND (toTimestamp=StringLiteral) + -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS $fromTimestamp $toTimestamp) ; alterStatementSuffixDropBranch diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java index 81ed88849df6..ddc12935700d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.ddl.table.AlterTableType; import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.ASTNode; import org.apache.hadoop.hive.ql.parse.HiveParser; @@ -42,6 +43,7 @@ import org.apache.hadoop.hive.ql.session.SessionState; import java.time.ZoneId; +import java.util.List; import java.util.Map; import java.util.regex.Pattern; @@ -79,7 +81,8 @@ protected void analyzeCommand(TableName tableName, Map partition desc = getRollbackDesc(tableName, partitionSpec, (ASTNode) command.getChild(1)); break; case HiveParser.KW_EXPIRE_SNAPSHOTS: - desc = getExpireSnapshotDesc(tableName, partitionSpec, (ASTNode) command.getChild(1)); + desc = getExpireSnapshotDesc(tableName, partitionSpec, command.getChildren()); + break; case HiveParser.KW_SET_CURRENT_SNAPSHOT: desc = getSetCurrentSnapshotDesc(tableName, partitionSpec, (ASTNode) command.getChild(1)); @@ -130,18 +133,25 @@ private static AlterTableExecuteDesc getSetCurrentSnapshotDesc(TableName tableNa } private static AlterTableExecuteDesc getExpireSnapshotDesc(TableName tableName, Map partitionSpec, - ASTNode childNode) throws SemanticException { + List children) throws SemanticException { AlterTableExecuteSpec spec; - // the second child must be the rollback parameter ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() : SessionState.get().getConf().getLocalTimeZone(); - String childText = PlanUtils.stripQuotes(childNode.getText().trim()); - if (EXPIRE_SNAPSHOT_BY_ID_REGEX.matcher(childText).matches()) { - spec = new AlterTableExecuteSpec(EXPIRE_SNAPSHOT, new ExpireSnapshotsSpec(childText)); + ASTNode firstNode = (ASTNode) children.get(1); + String firstNodeText = PlanUtils.stripQuotes(firstNode.getText().trim()); + if (children.size() == 3) { + ASTNode secondNode = (ASTNode) children.get(2); + String secondNodeText = PlanUtils.stripQuotes(secondNode.getText().trim()); + TimestampTZ fromTime = TimestampTZUtil.parse(firstNodeText, timeZone); + TimestampTZ toTime = TimestampTZUtil.parse(secondNodeText, timeZone); + spec = new AlterTableExecuteSpec(EXPIRE_SNAPSHOT, + new ExpireSnapshotsSpec(fromTime.toEpochMilli(), toTime.toEpochMilli())); + } else if (EXPIRE_SNAPSHOT_BY_ID_REGEX.matcher(firstNodeText).matches()) { + spec = new AlterTableExecuteSpec(EXPIRE_SNAPSHOT, new ExpireSnapshotsSpec(firstNodeText)); } else { - TimestampTZ time = TimestampTZUtil.parse(childText, timeZone); + TimestampTZ time = TimestampTZUtil.parse(firstNodeText, timeZone); spec = new AlterTableExecuteSpec(EXPIRE_SNAPSHOT, new ExpireSnapshotsSpec(time.toEpochMilli())); } return new AlterTableExecuteDesc(tableName, partitionSpec, spec); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java index fefeb267c290..c469b24415f6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java @@ -109,6 +109,8 @@ public static class ExpireSnapshotsSpec { private long timestampMillis = -1L; private String[] idsToExpire = null; + private long fromTimestampMillis = -1L; + public ExpireSnapshotsSpec(long timestampMillis) { this.timestampMillis = timestampMillis; } @@ -117,10 +119,19 @@ public ExpireSnapshotsSpec(String ids) { this.idsToExpire = ids.split(","); } + public ExpireSnapshotsSpec(long fromTimestampMillis, long toTimestampMillis) { + this.fromTimestampMillis = fromTimestampMillis; + this.timestampMillis = toTimestampMillis; + } + public Long getTimestampMillis() { return timestampMillis; } + public Long getFromTimestampMillis() { + return fromTimestampMillis; + } + public String[] getIdsToExpire() { return idsToExpire; } @@ -129,10 +140,16 @@ public boolean isExpireByIds() { return idsToExpire != null; } + public boolean isExpireByTimestampRange() { + return timestampMillis != -1 && fromTimestampMillis != -1; + } + @Override public String toString() { MoreObjects.ToStringHelper stringHelper = MoreObjects.toStringHelper(this); - if (isExpireByIds()) { + if (isExpireByTimestampRange()) { + stringHelper.add("fromTimestampMillis", fromTimestampMillis).add("toTimestampMillis", timestampMillis); + } else if (isExpireByIds()) { stringHelper.add("idsToExpire", Arrays.toString(idsToExpire)); } else { stringHelper.add("timestampMillis", timestampMillis); From 076f0ac5d514f6ca414399e216fd40cb0d0919f5 Mon Sep 17 00:00:00 2001 From: Naveen Gangam Date: Mon, 9 Oct 2023 11:27:36 -0400 Subject: [PATCH 005/179] =?UTF-8?q?HIVE-27597:=20Implement=20data=20connec?= =?UTF-8?q?tor=20for=20Hive=20to=20Hive=20federation=20over=E2=80=A6=20(#4?= =?UTF-8?q?720)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * HIVE-27597: Implement data connector for Hive to Hive federation over JDBC (Naveen Gangam) --- .../AbstractDataConnectorProvider.java | 3 + .../DataConnectorProviderFactory.java | 2 + .../dataconnector/IDataConnectorProvider.java | 1 + .../JDBCConnectorProviderFactory.java | 5 ++ .../jdbc/AbstractJDBCConnectorProvider.java | 58 ++++++++++----- .../jdbc/DerbySQLConnectorProvider.java | 17 ----- .../jdbc/HiveJDBCConnectorProvider.java | 71 +++++++++++++++++++ .../jdbc/MSSQLConnectorProvider.java | 22 ------ .../jdbc/MySQLConnectorProvider.java | 16 ----- .../jdbc/OracleConnectorProvider.java | 22 ------ .../jdbc/PostgreSQLConnectorProvider.java | 15 ---- 11 files changed, 124 insertions(+), 108 deletions(-) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/HiveJDBCConnectorProvider.java diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/AbstractDataConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/AbstractDataConnectorProvider.java index cbb832f128c9..c39ccb42b7a4 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/AbstractDataConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/AbstractDataConnectorProvider.java @@ -148,4 +148,7 @@ protected Table buildTableFromColsList(String tableName, List cols) abstract protected String getOutputClass(); abstract protected String getTableLocation(String tblName); + + abstract protected String getDatasourceType(); + } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/DataConnectorProviderFactory.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/DataConnectorProviderFactory.java index 6a24fb0f85dc..44b54d5f2c6b 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/DataConnectorProviderFactory.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/DataConnectorProviderFactory.java @@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.hive.metastore.dataconnector.IDataConnectorProvider.DERBY_TYPE; +import static org.apache.hadoop.hive.metastore.dataconnector.IDataConnectorProvider.HIVE_JDBC_TYPE; import static org.apache.hadoop.hive.metastore.dataconnector.IDataConnectorProvider.MSSQL_TYPE; import static org.apache.hadoop.hive.metastore.dataconnector.IDataConnectorProvider.MYSQL_TYPE; import static org.apache.hadoop.hive.metastore.dataconnector.IDataConnectorProvider.ORACLE_TYPE; @@ -98,6 +99,7 @@ public static synchronized IDataConnectorProvider getDataConnectorProvider(Datab String type = connector.getType(); switch (type) { case DERBY_TYPE: + case HIVE_JDBC_TYPE: case MSSQL_TYPE: case MYSQL_TYPE: case ORACLE_TYPE: diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/IDataConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/IDataConnectorProvider.java index c4b6eb1edc0c..ade7fb1805c5 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/IDataConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/IDataConnectorProvider.java @@ -41,6 +41,7 @@ public interface IDataConnectorProvider { public static final String ORACLE_TYPE = "oracle"; public static final String MSSQL_TYPE = "mssql"; public static final String DERBY_TYPE = "derby"; + public static final String HIVE_JDBC_TYPE = "hivejdbc"; DataConnector connector = null; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/JDBCConnectorProviderFactory.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/JDBCConnectorProviderFactory.java index 60589e518441..5fea19adb207 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/JDBCConnectorProviderFactory.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/JDBCConnectorProviderFactory.java @@ -20,6 +20,7 @@ import org.apache.hadoop.hive.metastore.api.DataConnector; import org.apache.hadoop.hive.metastore.dataconnector.jdbc.DerbySQLConnectorProvider; +import org.apache.hadoop.hive.metastore.dataconnector.jdbc.HiveJDBCConnectorProvider; import org.apache.hadoop.hive.metastore.dataconnector.jdbc.MySQLConnectorProvider; import org.apache.hadoop.hive.metastore.dataconnector.jdbc.PostgreSQLConnectorProvider; import org.apache.hadoop.hive.metastore.dataconnector.jdbc.OracleConnectorProvider; @@ -51,6 +52,10 @@ public static IDataConnectorProvider get(String dbName, DataConnector connector) provider = new MSSQLConnectorProvider(dbName, connector); break; + case HIVE_JDBC_TYPE: + provider = new HiveJDBCConnectorProvider(dbName, connector); + break; + default: throw new RuntimeException("Unsupported JDBC type"); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/AbstractJDBCConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/AbstractJDBCConnectorProvider.java index 5a46fc963124..c30d1eddad1f 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/AbstractJDBCConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/AbstractJDBCConnectorProvider.java @@ -75,7 +75,7 @@ public abstract class AbstractJDBCConnectorProvider extends AbstractDataConnecto private static final String JDBC_INPUTFORMAT_CLASS = "org.apache.hive.storage.jdbc.JdbcInputFormat".intern(); private static final String JDBC_OUTPUTFORMAT_CLASS = "org.apache.hive.storage.jdbc.JdbcOutputFormat".intern(); - String type = null; // MYSQL, POSTGRES, ORACLE, DERBY, MSSQL, DB2 etc. + String type = null; // MYSQL, POSTGRES, ORACLE, DERBY, MSSQL, DB2, HIVEJDBC etc. String jdbcUrl = null; String username = null; String password = null; // TODO convert to byte array @@ -111,7 +111,7 @@ public AbstractJDBCConnectorProvider(String dbName, DataConnector dataConn, Stri try { Class.forName(driverClassName); } catch (ClassNotFoundException cnfe) { - LOG.warn("Driver class not found in classpath:" + driverClassName); + LOG.warn("Driver class not found in classpath: {}" + driverClassName); throw new RuntimeException("Driver class not found:" + driverClass.getClass().getName(), cnfe); } } @@ -121,7 +121,7 @@ public AbstractJDBCConnectorProvider(String dbName, DataConnector dataConn, Stri close(); handle = DriverManager.getDriver(jdbcUrl).connect(jdbcUrl, getConnectionProperties()); } catch (SQLException sqle) { - LOG.warn("Could not connect to remote data source at " + jdbcUrl); + LOG.warn("Could not connect to remote data source at {}", jdbcUrl); throw new ConnectException("Could not connect to remote datasource at " + jdbcUrl + ",cause:" + sqle.getMessage()); } } @@ -155,7 +155,7 @@ protected boolean isClosed() { if (handle instanceof Connection) return ((Connection) handle).isClosed(); } catch (SQLException e) { - LOG.warn("Could not determine whether jdbc connection is closed or not to "+ jdbcUrl, e); + LOG.warn("Could not determine whether jdbc connection, to {}, is closed or not: {} ", jdbcUrl, e); } return true; } @@ -165,7 +165,7 @@ protected boolean isClosed() { try { ((Connection)handle).close(); } catch (SQLException sqle) { - LOG.warn("Could not close jdbc connection to " + jdbcUrl, sqle); + LOG.warn("Could not close jdbc connection to {}: {}", jdbcUrl, sqle); throw new RuntimeException(sqle); } } @@ -191,7 +191,7 @@ protected boolean isClosed() { return tables; } } catch (SQLException sqle) { - LOG.warn("Could not retrieve tables from remote datasource, cause:" + sqle.getMessage()); + LOG.warn("Could not retrieve tables from remote datasource, cause: {}", sqle.getMessage()); throw new MetaException("Error retrieving remote table:" + sqle); } finally { try { @@ -221,7 +221,7 @@ protected boolean isClosed() { return tables; } } catch (SQLException sqle) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); + LOG.warn("Could not retrieve table names from remote datasource, cause: {}", sqle.getMessage()); throw new MetaException("Error retrieving remote table:" + sqle); } finally { try { @@ -234,9 +234,32 @@ protected boolean isClosed() { return null; } - protected abstract ResultSet fetchTableMetadata(String tableName) throws MetaException; + protected ResultSet fetchTableMetadata(String tableName) throws MetaException { + ResultSet rs = null; + try { + rs = getConnection().getMetaData().getTables(getCatalogName(), getDatabaseName(), null, new String[] { "TABLE" }); + } catch (SQLException sqle) { + LOG.warn("Could not retrieve table names from remote datasource, cause: {}", sqle.getMessage()); + throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); + } + return rs; + } - protected abstract ResultSet fetchTableNames() throws MetaException; + /** + * Returns a list of all table names from the remote database. + * @return List A collection of all the table names, null if there are no tables. + * @throws MetaException To indicate any failures with executing this API + */ + protected ResultSet fetchTableNames() throws MetaException { + ResultSet rs = null; + try { + rs = getConnection().getMetaData().getTables(getCatalogName(), getDatabaseName(), null, new String[] { "TABLE" }); + } catch (SQLException sqle) { + LOG.warn("Could not retrieve table names from remote datasource, cause: {}", sqle.getMessage()); + throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle); + } + return rs; + } protected abstract String getCatalogName(); @@ -270,7 +293,7 @@ protected boolean isClosed() { table = buildTableFromColsList(tableName, cols); //Setting the table properties. - table.getParameters().put(JDBC_DATABASE_TYPE, this.type); + table.getParameters().put(JDBC_DATABASE_TYPE, getDatasourceType()); table.getParameters().put(JDBC_DRIVER, this.driverClassName); table.getParameters().put(JDBC_TABLE, tableName); table.getParameters().put(JDBC_SCHEMA, scoped_db); @@ -285,8 +308,8 @@ protected boolean isClosed() { } return table; } catch (Exception e) { - LOG.warn("Exception retrieving remote table " + scoped_db + "." + tableName + " via data connector " - + connector.getName()); + LOG.warn("Exception retrieving remote table {}.{} via data connector {}", scoped_db, tableName + ,connector.getName()); throw new MetaException("Error retrieving remote table:" + e); } finally { try { @@ -302,7 +325,7 @@ private ResultSet fetchTablesViaDBMetaData(String regex) throws SQLException { try { rs = getConnection().getMetaData().getTables(getCatalogName(), getDatabaseName(), regex, new String[]{"TABLE"}); } catch (SQLException sqle) { - LOG.warn("Could not retrieve tables from JDBC table, cause:" + sqle.getMessage()); + LOG.warn("Could not retrieve tables from JDBC table, cause: {}", sqle.getMessage()); throw sqle; } return rs; @@ -313,7 +336,7 @@ private ResultSet fetchColumnsViaDBMetaData(String tableName) throws SQLExceptio try { rs = getConnection().getMetaData().getColumns(getCatalogName(), getDatabaseName(), tableName, null); } catch (SQLException sqle) { - LOG.warn("Could not retrieve columns from JDBC table, cause:" + sqle.getMessage()); + LOG.warn("Could not retrieve columns from JDBC table, cause: {}", sqle.getMessage()); throw sqle; } return rs; @@ -402,7 +425,7 @@ protected String getDataType(String mySqlType, int size) { try { return warehouse.getDefaultTablePath(scoped_db, tableName, true).toString(); } catch (MetaException e) { - LOG.info("Error determining default table path, cause:" + e.getMessage()); + LOG.info("Error determining default table path, cause: {}", e.getMessage()); } } return "some_dummy_path"; @@ -415,4 +438,7 @@ protected Properties getConnectionProperties() { connectorPropMap.forEach((k, v) -> connectionProperties.put(k, v)); return connectionProperties; } -} \ No newline at end of file + + @Override + protected String getDatasourceType() { return type; } +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/DerbySQLConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/DerbySQLConnectorProvider.java index 897f2b73b467..56819ea9b9fb 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/DerbySQLConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/DerbySQLConnectorProvider.java @@ -37,23 +37,6 @@ public DerbySQLConnectorProvider(String dbName, DataConnector connector) { super(dbName, connector, DRIVER_CLASS); } - /** - * Returns a list of all table names from the remote database. - * @return List A collection of all the table names, null if there are no tables. - * @throws MetaException To indicate any failures with executing this API - */ - @Override - protected ResultSet fetchTableNames() throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getTables(scoped_db, null, null, new String[] { "TABLE" }); - } catch (SQLException sqle) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle); - } - return rs; - } - /** * Fetch a single table with the given name, returns a Hive Table object from the remote database * @return Table A Table object for the matching table, null otherwise. diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/HiveJDBCConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/HiveJDBCConnectorProvider.java new file mode 100644 index 000000000000..5a6af2563e39 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/HiveJDBCConnectorProvider.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.metastore.dataconnector.jdbc; + +import org.apache.hadoop.hive.metastore.ColumnType; +import org.apache.hadoop.hive.metastore.api.DataConnector; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.ResultSet; +import java.sql.SQLException; + +public class HiveJDBCConnectorProvider extends AbstractJDBCConnectorProvider { + private static final Logger LOG = LoggerFactory.getLogger(HiveJDBCConnectorProvider.class); + private static final String DRIVER_CLASS = "org.apache.hive.jdbc.HiveDriver"; + // for Hive the type for connector is "HIVEJDBC" where as on the table we want it to be "HIVE" + protected static final String mappedType = "HIVE"; + + public HiveJDBCConnectorProvider(String dbName, DataConnector dataConn) { + super(dbName, dataConn, DRIVER_CLASS); + } + + @Override protected String getCatalogName() { + return null; + } + + @Override protected String getDatabaseName() { + return scoped_db; + } + + @Override protected String getDataType(String dbDataType, int size) { + String mappedType = super.getDataType(dbDataType, size); + if (!mappedType.equalsIgnoreCase(ColumnType.VOID_TYPE_NAME)) { + return mappedType; + } + + // map any db specific types here. + switch (dbDataType.trim().toLowerCase()) + { + case "string": + case "varchar": + mappedType = ColumnType.STRING_TYPE_NAME; + break; + default: + // TODO Hive has support for complex data types but JDBCSerDe only supports primitive types + // SerDe needs to enhanced first to be able to support complex types over federation + mappedType = ColumnType.VOID_TYPE_NAME; + break; + } + return mappedType; + } + + @Override protected String getDatasourceType() { return mappedType; } +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MSSQLConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MSSQLConnectorProvider.java index d678cc0bf456..c8bcf9780474 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MSSQLConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MSSQLConnectorProvider.java @@ -39,28 +39,6 @@ public MSSQLConnectorProvider(String dbName, DataConnector dataConn) { driverClassName = DRIVER_CLASS; } - @Override protected ResultSet fetchTableMetadata(String tableName) throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getColumns(null, scoped_db, tableName, null); - } catch (Exception ex) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + ex.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + ex); - } - return rs; - } - - @Override protected ResultSet fetchTableNames() throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getTables(null, scoped_db, null, new String[] { "TABLE" }); - } catch (SQLException sqle) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle); - } - return rs; - } - @Override protected String getCatalogName() { return null; } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MySQLConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MySQLConnectorProvider.java index faf54a574668..18b7376f547d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MySQLConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/MySQLConnectorProvider.java @@ -37,22 +37,6 @@ public MySQLConnectorProvider(String dbName, DataConnector dataConn) { super(dbName, dataConn, DRIVER_CLASS); } - /** - * Returns a list of all table names from the remote database. - * @return List A collection of all the table names, null if there are no tables. - * @throws MetaException To indicate any failures with executing this API - */ - @Override protected ResultSet fetchTableNames() throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getTables(scoped_db, null, null, new String[] { "TABLE" }); - } catch (SQLException sqle) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle); - } - return rs; - } - /** * Fetch a single table with the given name, returns a Hive Table object from the remote database * @return Table A Table object for the matching table, null otherwise. diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/OracleConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/OracleConnectorProvider.java index 78d537cb65bc..bcb3db11d730 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/OracleConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/OracleConnectorProvider.java @@ -39,28 +39,6 @@ public OracleConnectorProvider(String dbName, DataConnector dataConn) { driverClassName = DRIVER_CLASS; } - @Override protected ResultSet fetchTableMetadata(String tableName) throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getColumns(null, scoped_db, tableName, null); - } catch (Exception ex) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + ex.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + ex); - } - return rs; - } - - @Override protected ResultSet fetchTableNames() throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getTables(null, scoped_db, null, new String[] { "TABLE" }); - } catch (SQLException sqle) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle); - } - return rs; - } - @Override protected String getCatalogName() { return null; } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/PostgreSQLConnectorProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/PostgreSQLConnectorProvider.java index cafb0c0173e7..b79bee452dc7 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/PostgreSQLConnectorProvider.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/dataconnector/jdbc/PostgreSQLConnectorProvider.java @@ -35,21 +35,6 @@ public PostgreSQLConnectorProvider(String dbName, DataConnector dataConn) { super(dbName, dataConn, DRIVER_CLASS); } - @Override protected ResultSet fetchTableMetadata(String tableName) throws MetaException { - ResultSet rs = null; - try { - rs = getConnection().getMetaData().getTables(scoped_db, null, null, new String[] { "TABLE" }); - } catch (SQLException sqle) { - LOG.warn("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); - throw new MetaException("Could not retrieve table names from remote datasource, cause:" + sqle.getMessage()); - } - return rs; - } - - @Override protected ResultSet fetchTableNames() throws MetaException { - return null; - } - @Override protected String getCatalogName() { return scoped_db; } From ec51d3b2d07de665426427a79cacf548d988de09 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Tue, 10 Oct 2023 09:46:04 +0300 Subject: [PATCH 006/179] HIVE-27031: Addendum: Iceberg: Implement Copy-On-Write for Delete queries (Denys Kuzmenko, reviewed by Krisztian Kasa, Butao Zhang) Closes #4700 --- data/conf/iceberg/llap/tez-site.xml | 16 +- data/conf/iceberg/tez/tez-site.xml | 8 + .../iceberg/mr/hive/FilesForCommit.java | 19 +- .../mr/hive/HiveIcebergOutputCommitter.java | 30 +- .../mr/hive/HiveIcebergStorageHandler.java | 19 +- .../HiveIcebergCopyOnWriteRecordWriter.java | 86 + .../iceberg/mr/hive/writer/WriterBuilder.java | 11 +- ...delete_iceberg_copy_on_write_partitioned.q | 40 + ...lete_iceberg_copy_on_write_unpartitioned.q | 40 + .../queries/positive/iceberg_copy_on_write.q | 8 - ...te_iceberg_copy_on_write_partitioned.q.out | 1808 +++++++++++++++++ ..._iceberg_copy_on_write_unpartitioned.q.out | 1808 +++++++++++++++++ .../positive/dynamic_semijoin_reduction.q.out | 2 +- .../positive/iceberg_copy_on_write.q.out | 101 +- .../hive/ql/metadata/HiveStorageHandler.java | 4 + .../ql/optimizer/lineage/ExprProcFactory.java | 4 +- .../ql/parse/RewriteSemanticAnalyzer.java | 21 +- .../parse/UpdateDeleteSemanticAnalyzer.java | 100 +- 18 files changed, 4014 insertions(+), 111 deletions(-) create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out create mode 100644 iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out diff --git a/data/conf/iceberg/llap/tez-site.xml b/data/conf/iceberg/llap/tez-site.xml index 7ad5ad4c66b1..fff6c8756203 100644 --- a/data/conf/iceberg/llap/tez-site.xml +++ b/data/conf/iceberg/llap/tez-site.xml @@ -1,8 +1,4 @@ - - tez.am.resource.memory.mb - 128 - tez.am.dag.scheduler.class org.apache.tez.dag.app.dag.impl.DAGSchedulerNaturalOrderControlled @@ -11,4 +7,16 @@ tez.am.resource.memory.mb 256 + + tez.runtime.io.sort.mb + 24 + + + hive.tez.container.size + 512 + + + tez.counters.max + 1024 + diff --git a/data/conf/iceberg/tez/tez-site.xml b/data/conf/iceberg/tez/tez-site.xml index 467bfb4dad1a..fff6c8756203 100644 --- a/data/conf/iceberg/tez/tez-site.xml +++ b/data/conf/iceberg/tez/tez-site.xml @@ -7,8 +7,16 @@ tez.am.resource.memory.mb 256 + + tez.runtime.io.sort.mb + 24 + hive.tez.container.size 512 + + tez.counters.max + 1024 + diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/FilesForCommit.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/FilesForCommit.java index 953edfd0d2d8..1bc5ea3a6741 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/FilesForCommit.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/FilesForCommit.java @@ -33,10 +33,17 @@ public class FilesForCommit implements Serializable { private final Collection dataFiles; private final Collection deleteFiles; + private Collection referencedDataFiles; public FilesForCommit(Collection dataFiles, Collection deleteFiles) { + this(dataFiles, deleteFiles, Collections.emptyList()); + } + + public FilesForCommit(Collection dataFiles, Collection deleteFiles, + Collection referencedDataFiles) { this.dataFiles = dataFiles; this.deleteFiles = deleteFiles; + this.referencedDataFiles = referencedDataFiles; } public static FilesForCommit onlyDelete(Collection deleteFiles) { @@ -47,6 +54,10 @@ public static FilesForCommit onlyData(Collection dataFiles) { return new FilesForCommit(dataFiles, Collections.emptyList()); } + public static FilesForCommit onlyData(Collection dataFiles, Collection referencedDataFiles) { + return new FilesForCommit(dataFiles, Collections.emptyList(), referencedDataFiles); + } + public static FilesForCommit empty() { return new FilesForCommit(Collections.emptyList(), Collections.emptyList()); } @@ -59,12 +70,16 @@ public Collection deleteFiles() { return deleteFiles; } + public Collection referencedDataFiles() { + return referencedDataFiles; + } + public Collection allFiles() { return Stream.concat(dataFiles.stream(), deleteFiles.stream()).collect(Collectors.toList()); } public boolean isEmpty() { - return dataFiles.isEmpty() && deleteFiles.isEmpty(); + return dataFiles.isEmpty() && deleteFiles.isEmpty() && referencedDataFiles.isEmpty(); } @Override @@ -72,6 +87,8 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("dataFiles", dataFiles.toString()) .add("deleteFiles", deleteFiles.toString()) + .add("referencedDataFiles", referencedDataFiles.toString()) .toString(); } + } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index db62dcef1e9b..6816bdf71b00 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -56,6 +56,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Snapshot; @@ -140,12 +141,15 @@ public void commitTask(TaskAttemptContext originalContext) throws IOException { if (writers.get(output) != null) { Collection dataFiles = Lists.newArrayList(); Collection deleteFiles = Lists.newArrayList(); + Collection referencedDataFiles = Lists.newArrayList(); for (HiveIcebergWriter writer : writers.get(output)) { FilesForCommit files = writer.files(); dataFiles.addAll(files.dataFiles()); deleteFiles.addAll(files.deleteFiles()); + referencedDataFiles.addAll(files.referencedDataFiles()); } - createFileForCommit(new FilesForCommit(dataFiles, deleteFiles), fileForCommitLocation, table.io()); + createFileForCommit(new FilesForCommit(dataFiles, deleteFiles, referencedDataFiles), + fileForCommitLocation, table.io()); } else { LOG.info("CommitTask found no writer for specific table: {}, attemptID: {}", output, attemptID); createFileForCommit(FilesForCommit.empty(), fileForCommitLocation, table.io()); @@ -405,6 +409,7 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output } List dataFiles = Lists.newArrayList(); List deleteFiles = Lists.newArrayList(); + List referencedDataFiles = Lists.newArrayList(); Table table = null; String branchName = null; @@ -431,9 +436,10 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output numTasks, executor, outputTable.table.location(), jobContext, io, true); dataFiles.addAll(writeResults.dataFiles()); deleteFiles.addAll(writeResults.deleteFiles()); + referencedDataFiles.addAll(writeResults.referencedDataFiles()); } - FilesForCommit filesForCommit = new FilesForCommit(dataFiles, deleteFiles); + FilesForCommit filesForCommit = new FilesForCommit(dataFiles, deleteFiles, referencedDataFiles); long startTime = System.currentTimeMillis(); if (Operation.IOW != operation) { @@ -470,6 +476,21 @@ private Long getSnapshotId(Table table, String branchName) { private void commitWrite(Table table, String branchName, Long snapshotId, long startTime, FilesForCommit results, Operation operation) { + if (!results.referencedDataFiles().isEmpty()) { + OverwriteFiles write = table.newOverwrite(); + results.referencedDataFiles().forEach(write::deleteFile); + results.dataFiles().forEach(write::addFile); + + if (StringUtils.isNotEmpty(branchName)) { + write.toBranch(HiveUtils.getTableSnapshotRef(branchName)); + } + if (snapshotId != null) { + write.validateFromSnapshot(snapshotId); + } + write.commit(); + return; + } + if (results.deleteFiles().isEmpty() && Operation.MERGE != operation) { AppendFiles write = table.newAppend(); results.dataFiles().forEach(write::appendFile); @@ -620,6 +641,7 @@ private static FilesForCommit collectResults(int numTasks, ExecutorService execu // starting from 0. Collection dataFiles = new ConcurrentLinkedQueue<>(); Collection deleteFiles = new ConcurrentLinkedQueue<>(); + Collection referencedDataFiles = new ConcurrentLinkedQueue<>(); Tasks.range(numTasks) .throwFailureWhenFinished(throwOnFailure) .executeWith(executor) @@ -629,9 +651,11 @@ private static FilesForCommit collectResults(int numTasks, ExecutorService execu FilesForCommit files = readFileForCommit(taskFileName, io); dataFiles.addAll(files.dataFiles()); deleteFiles.addAll(files.deleteFiles()); + referencedDataFiles.addAll(files.referencedDataFiles()); + }); - return new FilesForCommit(dataFiles, deleteFiles); + return new FilesForCommit(dataFiles, deleteFiles, referencedDataFiles); } /** diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 4e451403b028..98fb25b1d374 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -370,7 +370,18 @@ public String toString() { public DecomposedPredicate decomposePredicate(JobConf jobConf, Deserializer deserializer, ExprNodeDesc exprNodeDesc) { DecomposedPredicate predicate = new DecomposedPredicate(); predicate.residualPredicate = (ExprNodeGenericFuncDesc) exprNodeDesc; - predicate.pushedPredicate = (ExprNodeGenericFuncDesc) exprNodeDesc; + ExprNodeDesc pushedPredicate = exprNodeDesc.clone(); + + List subExprNodes = pushedPredicate.getChildren(); + if (subExprNodes.removeIf(nodeDesc -> nodeDesc.getCols() != null && + nodeDesc.getCols().contains(VirtualColumn.FILE_PATH.getName()))) { + if (subExprNodes.size() == 1) { + pushedPredicate = subExprNodes.get(0); + } else if (subExprNodes.isEmpty()) { + pushedPredicate = null; + } + } + predicate.pushedPredicate = (ExprNodeGenericFuncDesc) pushedPredicate; return predicate; } @@ -1074,6 +1085,12 @@ public List acidSelectColumns(org.apache.hadoop.hive.ql.metadata.Ta } } + @Override + public FieldSchema getRowId() { + VirtualColumn rowId = VirtualColumn.ROW_POSITION; + return new FieldSchema(rowId.getName(), rowId.getTypeInfo().getTypeName(), ""); + } + @Override public List acidSortColumns(org.apache.hadoop.hive.ql.metadata.Table table, Operation operation) { switch (operation) { diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java new file mode 100644 index 000000000000..a6681a955232 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive.writer; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.io.Writable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.ClusteredDataWriter; +import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.mr.hive.FilesForCommit; +import org.apache.iceberg.mr.hive.IcebergAcidUtil; +import org.apache.iceberg.mr.mapred.Container; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class HiveIcebergCopyOnWriteRecordWriter extends HiveIcebergWriterBase { + + private final int currentSpecId; + + private final GenericRecord rowDataTemplate; + private final List referencedDataFiles; + + HiveIcebergCopyOnWriteRecordWriter(Schema schema, Map specs, int currentSpecId, + FileWriterFactory fileWriterFactory, OutputFileFactory fileFactory, FileIO io, + long targetFileSize) { + super(schema, specs, io, + new ClusteredDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize)); + this.currentSpecId = currentSpecId; + this.rowDataTemplate = GenericRecord.create(schema); + this.referencedDataFiles = Lists.newArrayList(); + } + + @Override + public void write(Writable row) throws IOException { + Record record = ((Container) row).get(); + PositionDelete positionDelete = IcebergAcidUtil.getPositionDelete(record, rowDataTemplate); + int specId = IcebergAcidUtil.parseSpecId(record); + Record rowData = positionDelete.row(); + + if (positionDelete.pos() < 0) { + DataFile dataFile = + DataFiles.builder(specs.get(specId)) + .withPath(positionDelete.path().toString()) + .withPartition(partition(rowData, specId)) + .withFileSizeInBytes(0) + .withRecordCount(0) + .build(); + referencedDataFiles.add(dataFile); + } else { + writer.write(rowData, specs.get(currentSpecId), partition(rowData, currentSpecId)); + } + } + + @Override + public FilesForCommit files() { + List dataFiles = ((DataWriteResult) writer.result()).dataFiles(); + return FilesForCommit.onlyData(dataFiles, referencedDataFiles); + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java index 31073f640186..530309172d34 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java @@ -26,6 +26,7 @@ import org.apache.hadoop.mapred.TaskAttemptID; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -120,11 +121,19 @@ public HiveIcebergWriter build() { new HiveFileWriterFactory(table, dataFileFormat, dataSchema, null, deleteFileFormat, null, null, null, skipRowData ? null : dataSchema); + boolean copyOnWriteMode = RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( + properties.get(TableProperties.DELETE_MODE)) && operation == Operation.DELETE; + HiveIcebergWriter writer; switch (operation) { case DELETE: - writer = new HiveIcebergDeleteWriter(dataSchema, specs, writerFactory, deleteOutputFileFactory, + if (copyOnWriteMode) { + writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, + outputFileFactory, io, targetFileSize); + } else { + writer = new HiveIcebergDeleteWriter(dataSchema, specs, writerFactory, deleteOutputFileFactory, io, targetFileSize, skipRowData); + } break; case OTHER: writer = new HiveIcebergRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, diff --git a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q new file mode 100644 index 000000000000..86c1b59ce23c --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q @@ -0,0 +1,40 @@ +set hive.explain.user=false; + +drop table if exists tbl_ice; +create external table tbl_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.delete.mode'='copy-on-write'); + +-- delete using simple predicates +insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56); +explain delete from tbl_ice where b in ('one', 'four') or a = 22; + +delete from tbl_ice where b in ('one', 'four') or a = 22; +select * from tbl_ice order by a; +-- (2, 'two', 51), (3, 'three', 52), (5, 'five', 54), (333, 'two', 56) + +-- delete using subqueries referencing the same table +insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801); +explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); + +delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); +select * from tbl_ice order by a; +-- (333, 'two', 56), (444, 'hola', 800) + +-- delete using a join subquery between the same table & another table +drop table if exists tbl_ice_other; +create external table tbl_ice_other(a int, b string) stored by iceberg; +insert into tbl_ice_other values (10, 'ten'), (333, 'hundred'); +explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); + +delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); +select * from tbl_ice order by a; +-- (444, 'hola', 800) + +-- delete using a join subquery between the same table & a non-iceberg table +drop table if exists tbl_standard_other; +create external table tbl_standard_other(a int, b string) stored as orc; +insert into tbl_standard_other values (10, 'ten'), (444, 'tutu'); +explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); + +delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); +select count(*) from tbl_ice; +-- 0 diff --git a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q new file mode 100644 index 000000000000..cf192789234a --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q @@ -0,0 +1,40 @@ +set hive.explain.user=false; + +drop table if exists tbl_ice; +create external table tbl_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.delete.mode'='copy-on-write'); + +-- delete using simple predicates +insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56); +explain delete from tbl_ice where b in ('one', 'four') or a = 22; + +delete from tbl_ice where b in ('one', 'four') or a = 22; +select * from tbl_ice order by a; +-- (2, 'two', 51), (3, 'three', 52), (5, 'five', 54), (333, 'two', 56) + +-- delete using subqueries referencing the same table +insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801); +explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); + +delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); +select * from tbl_ice order by a; +-- (333, 'two', 56), (444, 'hola', 800) + +-- delete using a join subquery between the same table & another table +drop table if exists tbl_ice_other; +create external table tbl_ice_other(a int, b string) stored by iceberg; +insert into tbl_ice_other values (10, 'ten'), (333, 'hundred'); +explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); + +delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); +select * from tbl_ice order by a; +-- (444, 'hola', 800) + +-- delete using a join subquery between the same table & a non-iceberg table +drop table if exists tbl_standard_other; +create external table tbl_standard_other(a int, b string) stored as orc; +insert into tbl_standard_other values (10, 'ten'), (444, 'tutu'); +explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); + +delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); +select count(*) from tbl_ice; +-- 0 diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q index b93f1d9e5bda..3bfe3dee23f4 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q @@ -34,14 +34,6 @@ select * from ice01; -- should be only one data file. select summary from default.ice01.snapshots; --- Null cases. - -delete from ice01 where null; - -select * from ice01; - -delete from ice01 where not null; - select * from ice01; delete from ice01 where name=null; diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out new file mode 100644 index 000000000000..0b84e42f45cc --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out @@ -0,0 +1,1808 @@ +PREHOOK: query: drop table if exists tbl_ice +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.delete.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: create external table tbl_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.delete.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: explain delete from tbl_ice where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: (((a <> 22) and (b <> 'one') and (b <> 'four')) or (b) IN ('one', 'four') or (a = 22)) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Filter Operator + predicate: (((b) IN ('one', 'four') or (a = 22)) and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Map 5 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 4 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col5 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col5: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col5 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 584 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Union 3 + Vertex: Union 3 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 two 51 +3 three 52 +5 five 54 +333 two 56 +PREHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +PREHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 21 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE) + Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) + Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) + Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 18 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 19 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 20 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 22 <- Map 1 (SIMPLE_EDGE) + Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 20 (XPROD_EDGE) + Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 22 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 23 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Union 7 (CONTAINS) + Reducer 8 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) + Reducer 9 <- Reducer 19 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 6 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 6 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Select Operator + expressions: a (type: int), c (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Filter Operator + predicate: (a <= 5) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + keys: a (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(a) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Filter Operator + predicate: (c > 800) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8333333 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + keys: c (type: int) + minReductionHashAggr: 0.6666666 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(c) + minReductionHashAggr: 0.8333333 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col5, _col6 + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col2, _col3, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col3 <> 0L) and _col5 is not null) or ((_col6 <> 0L) and _col8 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 9 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 9 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 13 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 14 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 15 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9 + Statistics: Num rows: 6 Data size: 1870 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 6 Data size: 1870 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) + Reducer 16 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col11 + Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col6 <> 0L) and _col8 is not null) or ((_col9 <> 0L) and _col11 is not null)) (type: boolean) + Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 17 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 3 Data size: 879 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 18 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 19 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 20 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 21 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 22 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 23 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 + Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 + Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 + Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1, _col2, _col3, _col5 + Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) + Union 7 + Vertex: Union 7 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +333 two 56 +444 hola 800 +PREHOOK: query: drop table if exists tbl_ice_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice_other +PREHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice_other +Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE) + Reducer 12 <- Reducer 8 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 4 <- Reducer 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Map 13 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 2 Data size: 610 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 2 Data size: 610 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Union 5 + Vertex: Union 5 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +444 hola 800 +PREHOOK: query: drop table if exists tbl_standard_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_standard_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_standard_other +PREHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_standard_other +POSTHOOK: Lineage: tbl_standard_other.a SCRIPT [] +POSTHOOK: Lineage: tbl_standard_other.b SCRIPT [] +Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 8 (SIMPLE_EDGE) + Reducer 11 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Map 13 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 10 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 1 Data size: 305 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 305 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Group By Operator + keys: _col1 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Union 5 + Vertex: Union 5 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[178][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select count(*) from tbl_ice +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from tbl_ice +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out new file mode 100644 index 000000000000..d9f95eb36f63 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out @@ -0,0 +1,1808 @@ +PREHOOK: query: drop table if exists tbl_ice +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.delete.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: create external table tbl_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.delete.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: explain delete from tbl_ice where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: (((a <> 22) and (b <> 'one') and (b <> 'four')) or (b) IN ('one', 'four') or (a = 22)) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Filter Operator + predicate: (((b) IN ('one', 'four') or (a = 22)) and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Map 5 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 4 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col5 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col5: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col5 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 584 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 4 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Union 3 + Vertex: Union 3 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 two 51 +3 three 52 +5 five 54 +333 two 56 +PREHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +PREHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 21 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE) + Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) + Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) + Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 18 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 19 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 20 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 22 <- Map 1 (SIMPLE_EDGE) + Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 20 (XPROD_EDGE) + Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 22 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 23 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Union 7 (CONTAINS) + Reducer 8 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) + Reducer 9 <- Reducer 19 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 6 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 6 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Select Operator + expressions: a (type: int), c (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Filter Operator + predicate: (a <= 5) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + keys: a (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(a) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Filter Operator + predicate: (c > 800) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8333333 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + keys: c (type: int) + minReductionHashAggr: 0.6666666 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(c) + minReductionHashAggr: 0.8333333 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col5, _col6 + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col2, _col3, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col3 <> 0L) and _col5 is not null) or ((_col6 <> 0L) and _col8 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 9 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 9 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 13 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 14 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 15 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9 + Statistics: Num rows: 6 Data size: 1870 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 6 Data size: 1870 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) + Reducer 16 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col11 + Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col6 <> 0L) and _col8 is not null) or ((_col9 <> 0L) and _col11 is not null)) (type: boolean) + Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 17 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 6 Data size: 1758 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 3 Data size: 879 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 18 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 19 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 20 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 21 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 22 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 23 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 + Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 + Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 + Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1, _col2, _col3, _col5 + Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) + Union 7 + Vertex: Union 7 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +333 two 56 +444 hola 800 +PREHOOK: query: drop table if exists tbl_ice_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice_other +PREHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice_other +Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE) + Reducer 12 <- Reducer 8 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 4 <- Reducer 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Map 13 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 2 Data size: 610 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 2 Data size: 610 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Union 5 + Vertex: Union 5 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +444 hola 800 +PREHOOK: query: drop table if exists tbl_standard_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_standard_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_standard_other +PREHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_standard_other +POSTHOOK: Lineage: tbl_standard_other.a SCRIPT [] +POSTHOOK: Lineage: tbl_standard_other.b SCRIPT [] +Warning: Shuffle Join MERGEJOIN[178][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE) + Reducer 12 <- Reducer 8 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 3 <- Reducer 12 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) + Reducer 4 <- Reducer 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Map 13 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 1 Data size: 305 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 305 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col1 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Select Operator + expressions: _col1 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Union 5 + Vertex: Union 5 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[178][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select count(*) from tbl_ice +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from tbl_ice +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out index 4e6e7d2c0791..2d2da52de0b2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out @@ -79,7 +79,7 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart_date_n7 - filterExpr: (key is not null and key BETWEEN DynamicValue(RS_7_srcpart_small_n3_key1_min) AND DynamicValue(RS_7_srcpart_small_n3_key1_max) and in_bloom_filter(key, DynamicValue(RS_7_srcpart_small_n3_key1_bloom_filter))) (type: boolean) + filterExpr: key is not null (type: boolean) Statistics: Num rows: 2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (key is not null and key BETWEEN DynamicValue(RS_7_srcpart_small_n3_key1_min) AND DynamicValue(RS_7_srcpart_small_n3_key1_max) and in_bloom_filter(key, DynamicValue(RS_7_srcpart_small_n3_key1_bloom_filter))) (type: boolean) diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out index 893fc51dee3a..83ee890d7b75 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out @@ -67,6 +67,11 @@ POSTHOOK: query: explain delete from ice01 where id>4 OR id=2 POSTHOOK: type: QUERY POSTHOOK: Input: default@ice01 POSTHOOK: Output: default@ice01 +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 3 (CONTAINS) +Reducer 4 <- Map 1 (SIMPLE_EDGE) +Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 3 (CONTAINS) + Stage-3 Stats Work{} Stage-0 @@ -75,15 +80,60 @@ Stage-3 Stage-2 Dependency Collection{} Stage-1 - Map 1 vectorized - File Output Operator [FS_7] - table:{"name:":"default.ice01"} - Select Operator [SEL_6] (rows=5 width=91) - Output:["_col0","_col1"] - Filter Operator [FIL_5] (rows=5 width=91) - predicate:(((id <= 4) and (id <> 2)) or ((id > 4) or (id = 2)) is null) - TableScan [TS_0] (rows=7 width=78) - default@ice01,ice01,Tbl:COMPLETE,Col:COMPLETE,Output:["id","name"] + Union 3 + <-Reducer 2 [CONTAINS] + File Output Operator [FS_46] + table:{"name:":"default.ice01"} + Select Operator [SEL_44] (rows=2 width=295) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + Merge Join Operator [MERGEJOIN_43] (rows=2 width=295) + Conds:RS_57._col4=RS_63._col0(Left Semi),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_57] + PartitionCols:_col4 + Select Operator [SEL_55] (rows=2 width=295) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + Filter Operator [FIL_53] (rows=2 width=91) + predicate:((id <= 4) and (id <> 2) and FILE__PATH is not null) + TableScan [TS_0] (rows=7 width=78) + default@ice01,ice01,Tbl:COMPLETE,Col:COMPLETE,Output:["id","name"] + <-Reducer 4 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_63] + PartitionCols:_col0 + Group By Operator [GBY_62] (rows=3 width=184) + Output:["_col0"],keys:_col0 + Select Operator [SEL_61] (rows=3 width=184) + Output:["_col0"] + Filter Operator [FIL_60] (rows=3 width=184) + predicate:(row_number_window_0 = 1) + PTF Operator [PTF_59] (rows=6 width=184) + Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4 ASC NULLS FIRST","partition by:":"_col4"}] + Select Operator [SEL_58] (rows=6 width=184) + Output:["_col4"] + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_56] + PartitionCols:FILE__PATH + Filter Operator [FIL_54] (rows=6 width=4) + predicate:(((id > 4) or (id = 2)) and FILE__PATH is not null) + Please refer to the previous TableScan [TS_0] + <-Reducer 6 [CONTAINS] vectorized + File Output Operator [FS_70] + table:{"name:":"default.ice01"} + Select Operator [SEL_69] (rows=3 width=266) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + Filter Operator [FIL_68] (rows=3 width=258) + predicate:(row_number_window_0 = 1) + PTF Operator [PTF_67] (rows=6 width=272) + Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4 ASC NULLS FIRST","partition by:":"_col4"}] + Select Operator [SEL_66] (rows=6 width=272) + Output:["_col0","_col1","_col2","_col3","_col4"] + <-Map 5 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_65] + PartitionCols:FILE__PATH + Filter Operator [FIL_64] (rows=6 width=76) + predicate:((id > 4) or (id = 2)) + TableScan [TS_18] (rows=7 width=78) + default@ice01,ice01,Tbl:COMPLETE,Col:COMPLETE,Output:["id","name"] PREHOOK: query: delete from ice01 where id>4 OR id=2 PREHOOK: type: QUERY @@ -114,34 +164,7 @@ POSTHOOK: Input: default@ice01 POSTHOOK: Output: hdfs://### HDFS PATH ### {"added-data-files":"1","added-records":"9","added-files-size":"#FileSize#","changed-partition-count":"1","total-records":"9","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"} {"added-position-delete-files":"1","added-delete-files":"1","added-files-size":"#FileSize#","added-position-deletes":"2","changed-partition-count":"1","total-records":"9","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"1","total-position-deletes":"2","total-equality-deletes":"0"} -{"replace-partitions":"true","added-data-files":"1","deleted-data-files":"1","removed-position-delete-files":"1","removed-delete-files":"1","added-records":"3","deleted-records":"9","added-files-size":"#FileSize#","removed-files-size":"#FileSize#","removed-position-deletes":"2","changed-partition-count":"1","total-records":"3","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"} -PREHOOK: query: delete from ice01 where null -PREHOOK: type: QUERY -PREHOOK: Input: default@ice01 -PREHOOK: Output: default@ice01 -POSTHOOK: query: delete from ice01 where null -POSTHOOK: type: QUERY -POSTHOOK: Input: default@ice01 -POSTHOOK: Output: default@ice01 -PREHOOK: query: select * from ice01 -PREHOOK: type: QUERY -PREHOOK: Input: default@ice01 -PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from ice01 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@ice01 -POSTHOOK: Output: hdfs://### HDFS PATH ### -1 ABC -3 NULL -4 POPI -PREHOOK: query: delete from ice01 where not null -PREHOOK: type: QUERY -PREHOOK: Input: default@ice01 -PREHOOK: Output: default@ice01 -POSTHOOK: query: delete from ice01 where not null -POSTHOOK: type: QUERY -POSTHOOK: Input: default@ice01 -POSTHOOK: Output: default@ice01 +{"added-data-files":"1","deleted-data-files":"1","added-records":"3","deleted-records":"9","added-files-size":"#FileSize#","removed-files-size":"#FileSize#","changed-partition-count":"1","total-records":"3","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"1","total-position-deletes":"2","total-equality-deletes":"0"} PREHOOK: query: select * from ice01 PREHOOK: type: QUERY PREHOOK: Input: default@ice01 @@ -155,10 +178,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 4 POPI PREHOOK: query: delete from ice01 where name=null PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Input: default@ice01 PREHOOK: Output: default@ice01 POSTHOOK: query: delete from ice01 where name=null POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Input: default@ice01 POSTHOOK: Output: default@ice01 PREHOOK: query: select * from ice01 @@ -174,10 +199,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 4 POPI PREHOOK: query: delete from ice01 where name!=null PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Input: default@ice01 PREHOOK: Output: default@ice01 POSTHOOK: query: delete from ice01 where name!=null POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Input: default@ice01 POSTHOOK: Output: default@ice01 PREHOOK: query: select * from ice01 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 4d89b0d81141..8037403f3acc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -463,6 +463,10 @@ default List acidSelectColumns(org.apache.hadoop.hive.ql.metadata.T return Collections.emptyList(); } + default FieldSchema getRowId() { + throw new UnsupportedOperationException(); + } + /** * {@link org.apache.hadoop.hive.ql.parse.UpdateDeleteSemanticAnalyzer} rewrites DELETE/UPDATE queries into INSERT * queries. E.g. DELETE FROM T WHERE A = 32 is rewritten into diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java index 1463739ec21e..8ba2c51e8506 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java @@ -234,7 +234,9 @@ public static String getExprString(RowSchema rs, ExprNodeDesc expr, if (baseCols != null && !baseCols.isEmpty()) { BaseColumnInfo baseCol = baseCols.iterator().next(); tabAlias = baseCol.getTabAlias().getAlias(); - alias = baseCol.getColumn().getName(); + if (baseCol.getColumn() != null) { + alias = baseCol.getColumn().getName(); + } } } if (tabAlias != null && tabAlias.length() > 0 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java index 469e85b39064..8bd4c9611d5e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java @@ -581,6 +581,11 @@ protected ColumnAppender(Table table, HiveConf conf, String subQueryAlias) { } public abstract void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation); + + public void appendAcidSelectColumnsForDeletedRecords(StringBuilder stringBuilder, Context.Operation operation) { + throw new UnsupportedOperationException(); + } + public abstract List getDeleteValues(Context.Operation operation); public abstract List getSortKeys(); @@ -634,12 +639,22 @@ public NonNativeAcidColumnAppender(Table table, HiveConf conf, String subQueryAl @Override public void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation) { + appendAcidSelectColumns(stringBuilder, operation, false); + } + + @Override + public void appendAcidSelectColumnsForDeletedRecords(StringBuilder stringBuilder, Context.Operation operation) { + appendAcidSelectColumns(stringBuilder, operation, true); + } + + private void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation, boolean markRowIdAsDeleted) { List acidSelectColumns = table.getStorageHandler().acidSelectColumns(table, operation); for (FieldSchema fieldSchema : acidSelectColumns) { - String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); + String identifier = markRowIdAsDeleted && fieldSchema.equals(table.getStorageHandler().getRowId()) ? + "-1" : HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); stringBuilder.append(identifier); - - if (StringUtils.isNotEmpty(deletePrefix)) { + + if (StringUtils.isNotEmpty(deletePrefix) && !markRowIdAsDeleted) { stringBuilder.append(" AS "); String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); stringBuilder.append(prefixedIdentifier); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java index b8a7ad16b150..86ded055e71c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java @@ -23,7 +23,6 @@ import java.util.Map; import java.util.Set; -import org.antlr.runtime.CommonToken; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.Context; @@ -119,31 +118,23 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN updateOutputs(mTable); return; } - - boolean shouldOverwrite = false; + + boolean copyOnWriteMode = false; HiveStorageHandler storageHandler = mTable.getStorageHandler(); if (storageHandler != null) { - shouldOverwrite = storageHandler.shouldOverwrite(mTable, operation.name()); + copyOnWriteMode = storageHandler.shouldOverwrite(mTable, operation.name()); } StringBuilder rewrittenQueryStr = new StringBuilder(); - if (shouldOverwrite) { - rewrittenQueryStr.append("insert overwrite table "); - } else { - rewrittenQueryStr.append("insert into table "); - } + rewrittenQueryStr.append("insert into table "); rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); addPartitionColsToInsert(mTable.getPartCols(), rewrittenQueryStr); ColumnAppender columnAppender = getColumnAppender(null, DELETE_PREFIX); int columnOffset = columnAppender.getDeleteValues(operation).size(); - if (!shouldOverwrite) { - rewrittenQueryStr.append(" select "); - columnAppender.appendAcidSelectColumns(rewrittenQueryStr, operation); - rewrittenQueryStr.setLength(rewrittenQueryStr.length() - 1); - } else { - rewrittenQueryStr.append(" select * "); - } + rewrittenQueryStr.append(" select "); + columnAppender.appendAcidSelectColumns(rewrittenQueryStr, operation); + rewrittenQueryStr.setLength(rewrittenQueryStr.length() - 1); Map setColExprs = null; Map setCols = null; @@ -178,36 +169,50 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN ASTNode where = null; int whereIndex = deleting() ? 1 : 2; + if (children.size() > whereIndex) { where = (ASTNode)children.get(whereIndex); assert where.getToken().getType() == HiveParser.TOK_WHERE : "Expected where clause, but found " + where.getName(); + + if (copyOnWriteMode) { + String whereClause = ctx.getTokenRewriteStream().toString( + where.getChild(0).getTokenStartIndex(), where.getChild(0).getTokenStopIndex()); + String filePathCol = HiveUtils.unparseIdentifier("FILE__PATH", conf); + + // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. + rewrittenQueryStr.append("\nwhere NOT (").append(whereClause).append(")"); + rewrittenQueryStr.append("\n").append(INDENT); + // Add the file path filter that matches the delete condition. + rewrittenQueryStr.append("AND ").append(filePathCol); + rewrittenQueryStr.append(" IN ( select ").append(filePathCol).append(" from t )"); + rewrittenQueryStr.append("\nunion all"); + rewrittenQueryStr.append("\nselect * from t"); - if (shouldOverwrite) { - if (where.getChildCount() == 1) { - - // Add isNull check for the where clause condition, since null is treated as false in where condition and - // not null also resolves to false, so we need to explicitly handle this case. - ASTNode isNullFuncNodeExpr = new ASTNode(new CommonToken(HiveParser.TOK_FUNCTION, "TOK_FUNCTION")); - isNullFuncNodeExpr.addChild(new ASTNode(new CommonToken(HiveParser.Identifier, "isNull"))); - isNullFuncNodeExpr.addChild(where.getChild(0)); - - ASTNode orNodeExpr = new ASTNode(new CommonToken(HiveParser.KW_OR, "OR")); - orNodeExpr.addChild(isNullFuncNodeExpr); - - // Add the inverted where clause condition, since we want to hold the records which doesn't satisfy this - // condition. - ASTNode notNodeExpr = new ASTNode(new CommonToken(HiveParser.KW_NOT, "!")); - notNodeExpr.addChild(where.getChild(0)); - orNodeExpr.addChild(notNodeExpr); - where.setChild(0, orNodeExpr); - } else if (where.getChildCount() > 1) { - throw new SemanticException("Overwrite mode not supported with more than 1 children in where clause."); - } + StringBuilder withQueryStr = new StringBuilder(); + withQueryStr.append("WITH t AS ("); + withQueryStr.append("\n").append(INDENT); + withQueryStr.append("select "); + columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, operation); + withQueryStr.setLength(withQueryStr.length() - 1); + withQueryStr.append(" from ("); + withQueryStr.append("\n").append(INDENT).append(INDENT); + withQueryStr.append("select "); + columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, operation); + withQueryStr.append(" row_number() OVER (partition by ").append(filePathCol).append(") rn"); + withQueryStr.append(" from ").append(getFullTableNameForSQL(tabNameNode)); + withQueryStr.append("\n").append(INDENT).append(INDENT); + withQueryStr.append("where ").append(whereClause); + withQueryStr.append("\n").append(INDENT); + withQueryStr.append(") q"); + withQueryStr.append("\n").append(INDENT); + withQueryStr.append("where rn=1\n)\n"); + + rewrittenQueryStr.insert(0, withQueryStr.toString()); } } - - if (!shouldOverwrite) { + + if (!copyOnWriteMode) { // Add a sort by clause so that the row ids come out in the correct order appendSortBy(rewrittenQueryStr, columnAppender.getSortKeys()); } @@ -216,23 +221,17 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN ASTNode rewrittenTree = rr.rewrittenTree; ASTNode rewrittenInsert = (ASTNode)rewrittenTree.getChildren().get(1); - assert rewrittenInsert.getToken().getType() == HiveParser.TOK_INSERT : - "Expected TOK_INSERT as second child of TOK_QUERY but found " + rewrittenInsert.getName(); - if (updating()) { rewrittenCtx.setOperation(Context.Operation.UPDATE); rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); } else if (deleting()) { - if (shouldOverwrite) { - // We are now actually executing an Insert query, so set the modes accordingly. - rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.INSERT); - } else { - rewrittenCtx.setOperation(Context.Operation.DELETE); - rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE); - } + rewrittenCtx.setOperation(Context.Operation.DELETE); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE); } - if (where != null) { + if (where != null && !copyOnWriteMode) { + assert rewrittenInsert.getToken().getType() == HiveParser.TOK_INSERT : + "Expected TOK_INSERT as second child of TOK_QUERY but found " + rewrittenInsert.getName(); // The structure of the AST for the rewritten insert statement is: // TOK_QUERY -> TOK_FROM // \-> TOK_INSERT -> TOK_INSERT_INTO @@ -273,7 +272,6 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN updateOutputs(mTable); - if (updating()) { setUpAccessControlInfoForUpdate(mTable, setCols); From 8060629695a1573ebf606cfb3195176b2ba965ba Mon Sep 17 00:00:00 2001 From: Henrib Date: Tue, 10 Oct 2023 13:02:01 +0200 Subject: [PATCH 007/179] HIVE-27733: Ensure PerfLogger is thread-safe (#4749). (Henri Biestro, reviewed by Ayush Saxena) --- .../apache/hadoop/hive/ql/log/PerfLogger.java | 69 ++++++------ .../hadoop/hive/ql/log/PerfLoggerTest.java | 103 ++++++++++++++++++ 2 files changed, 137 insertions(+), 35 deletions(-) create mode 100644 common/src/test/org/apache/hadoop/hive/ql/log/PerfLoggerTest.java diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java index 6254e3932aba..d6fcc0bda750 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java +++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * PerfLogger. @@ -89,12 +90,11 @@ public class PerfLogger { public static final String HIVE_GET_NOT_NULL_CONSTRAINT = "getNotNullConstraints"; public static final String HIVE_GET_TABLE_CONSTRAINTS = "getTableConstraints"; - protected final Map startTimes = new HashMap(); - protected final Map endTimes = new HashMap(); - - static final private Logger LOG = LoggerFactory.getLogger(PerfLogger.class.getName()); - protected static final ThreadLocal perfLogger = new ThreadLocal(); + protected final Map startTimes = new ConcurrentHashMap<>(); + protected final Map endTimes = new ConcurrentHashMap<>(); + private static final Logger LOG = LoggerFactory.getLogger(PerfLogger.class.getName()); + protected static final ThreadLocal perfLogger = new ThreadLocal<>(); private PerfLogger() { // Use getPerfLogger to get an instance of PerfLogger @@ -134,6 +134,7 @@ public void perfLogBegin(String callerName, String method) { LOG.debug("", method, callerName); beginMetrics(method); } + /** * Call this function in correspondence of PerfLogBegin to mark the end of the measurement. * @param callerName @@ -151,18 +152,18 @@ public long perfLogEnd(String callerName, String method) { * @return long duration the difference between now and startTime, or -1 if startTime is null */ public long perfLogEnd(String callerName, String method, String additionalInfo) { - Long startTime = startTimes.get(method); + long startTime = startTimes.getOrDefault(method, -1L); long endTime = System.currentTimeMillis(); + long duration = startTime < 0 ? -1 : endTime - startTime; endTimes.put(method, Long.valueOf(endTime)); - long duration = startTime == null ? -1 : endTime - startTime.longValue(); if (LOG.isDebugEnabled()) { StringBuilder sb = new StringBuilder("= 100); + } + + @Test + public void testMT() throws InterruptedException { + final PerfLogger pl = PerfLogger.getPerfLogger(null, true); + // we run concurrently the getEndTimes and perfLogBegin/perfLogEnd: + // on a Mac M1, this test fails easily if the perflogger maps are hashmaps + ExecutorService executorService = Executors.newFixedThreadPool(64); + // An executing threads counter + AtomicInteger count = new AtomicInteger(0); + // getEndTimes in a loop + executorService.execute(() -> { + PerfLogger.setPerfLogger(pl); + try { + count.incrementAndGet(); + snooze(100); + for (int i = 0; i < 64; ++i) { + snooze(50); + Map et = pl.getEndTimes(); + Assert.assertNotNull(et); + } + } finally { + count.decrementAndGet(); + synchronized (count) { + count.notifyAll(); + } + } + }); + // 32 threads calling perLogBeing/perfLogEnd + for(int t = 0; t < 31; ++t) { + executorService.execute(() -> { + try { + int cnt = count.incrementAndGet(); + PerfLogger.setPerfLogger(pl); + for (int i = 0; i < 64; ++i) { + pl.perfLogBegin("test", PerfLogger.COMPILE + "_ "+ cnt + "_" + i); + snooze(50); + pl.perfLogEnd("test", PerfLogger.COMPILE + "_ " + cnt + "_" + i); + } + } catch(Exception xany) { + String msg = xany.getMessage(); + } finally { + count.decrementAndGet(); + synchronized (count) { + count.notifyAll(); + } + } + }); + } + // wait for all threads to end + while(count.get() != 0) { + synchronized (count) { + count.wait(); + } + } + executorService.shutdown(); + } +} From 53e172fa85440ab96901d52e79e4a42a03e401d8 Mon Sep 17 00:00:00 2001 From: Soumyakanti Das Date: Tue, 10 Oct 2023 10:23:54 -0700 Subject: [PATCH 008/179] HIVE-27761: Compilation of nested CTEs throws SemanticException (Soumyakanti Das, reviewed by Krisztian Kasa) ADDENDUM: fix cycle detection Cycle was getting detected incorrectly because the cte names were not scoped. In this patch, we are adding the query block id to the cte name to differentiate ctes with same names but different scopes. --- .../hive/ql/parse/SemanticAnalyzer.java | 27 +++++++++++-------- .../test/queries/clientpositive/nested_ctes.q | 12 +++++++++ .../clientpositive/llap/nested_ctes.q.out | 27 +++++++++++++++++++ 3 files changed, 55 insertions(+), 11 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index ed8983690312..6315d74885c3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -1345,9 +1345,7 @@ private void processCTE(QB qb, ASTNode ctes, Map aliasToCTEs) ASTNode cteQry = (ASTNode) cte.getChild(0); String alias = unescapeIdentifier(cte.getChild(1).getText()); ASTNode withColList = cte.getChildCount() == 3 ? (ASTNode) cte.getChild(2) : null; - - String qName = qb.getId() == null ? "" : qb.getId() + ":"; - qName += alias.toLowerCase(); + String qName = getAliasId(alias, qb); if ( aliasToCTEs.containsKey(qName)) { throw new SemanticException(ASTErrorUtils.getMsg( @@ -2241,11 +2239,6 @@ private void gatherCTEReferences(QB qb, CTEClause current, CTEClause cte = findCTEFromName(qb, cteName, materializationAliasToCTEs); if (cte != null) { - if (ctesExpanded.contains(cteName)) { - throw new SemanticException("Recursive cte " + cteName + - " detected (cycle: " + StringUtils.join(ctesExpanded, " -> ") + - " -> " + cteName + ")."); - } cte.reference++; current.parents.add(cte); if (cte.qbExpr != null) { @@ -2253,10 +2246,7 @@ private void gatherCTEReferences(QB qb, CTEClause current, } cte.qbExpr = new QBExpr(cteName); doPhase1QBExpr(cte.cteNode, cte.qbExpr, qb.getId(), cteName, cte.withColList, materializationAliasToCTEs); - - ctesExpanded.add(cteName); gatherCTEReferences(cte.qbExpr, cte, materializationAliasToCTEs); - ctesExpanded.remove(ctesExpanded.size() - 1); } } for (String alias : qb.getSubqAliases()) { @@ -2267,6 +2257,20 @@ private void gatherCTEReferences(QB qb, CTEClause current, } } + private void checkRecursiveCTE(CTEClause current, Set path) throws SemanticException { + + for (CTEClause child : current.parents) { + if (path.contains(child.alias)) { + throw new SemanticException("Recursive cte " + child.alias + + " detected (cycle: " + StringUtils.join(path, " -> ") + + " -> " + child.alias + ")."); + } + path.add(child.alias); + checkRecursiveCTE(child, path); + path.remove(child.alias); + } + } + void getMetaData(QB qb) throws SemanticException { getMetaData(qb, false); } @@ -2277,6 +2281,7 @@ private void getMetaData(QB qb, boolean enableMaterialization) throws SemanticEx if (enableMaterialization) { materializationAliasToCTEs = getMaterializationMetadata(qb); } + checkRecursiveCTE(rootClause, new HashSet<>()); getMetaData(qb, null); if (materializationAliasToCTEs != null && !materializationAliasToCTEs.isEmpty()) { this.aliasToCTEs.putAll(materializationAliasToCTEs); diff --git a/ql/src/test/queries/clientpositive/nested_ctes.q b/ql/src/test/queries/clientpositive/nested_ctes.q index e68adb8420b2..5337f1504b85 100644 --- a/ql/src/test/queries/clientpositive/nested_ctes.q +++ b/ql/src/test/queries/clientpositive/nested_ctes.q @@ -5,6 +5,7 @@ with select 1 ) select * from test1; + with test2 as ( with @@ -36,3 +37,14 @@ with select * from t1 ) select * from test4; + +with + q1 as ( + with t3 as (select 1) + select * from t3 + ), + q2 as ( + with t3 as (select * from q1) + select * from t3 + ) +select * from q2; diff --git a/ql/src/test/results/clientpositive/llap/nested_ctes.q.out b/ql/src/test/results/clientpositive/llap/nested_ctes.q.out index 0b771699b5ba..fda998825bd0 100644 --- a/ql/src/test/results/clientpositive/llap/nested_ctes.q.out +++ b/ql/src/test/results/clientpositive/llap/nested_ctes.q.out @@ -100,3 +100,30 @@ select * from test4 POSTHOOK: type: QUERY POSTHOOK: Input: default@table_1 #### A masked pattern was here #### +PREHOOK: query: with + q1 as ( + with t3 as (select 1) + select * from t3 + ), + q2 as ( + with t3 as (select * from q1) + select * from t3 + ) +select * from q2 +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: with + q1 as ( + with t3 as (select 1) + select * from t3 + ), + q2 as ( + with t3 as (select * from q1) + select * from t3 + ) +select * from q2 +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +1 From 00a121e3c74fc1f46568d9fb4ffefc6931a53784 Mon Sep 17 00:00:00 2001 From: Gergely Farkas Date: Wed, 11 Oct 2023 15:06:21 +0200 Subject: [PATCH 009/179] HIVE-27764: Add "WWW-Authenticate: Negotiate" header to the response when the client is unauthorized and Kerberos is enabled (Gergely Farkas, reviewed by Zhihua Deng) Closes #4774 --- .../service/cli/thrift/ThriftHttpServlet.java | 3 +- .../cli/thrift/ThriftHttpServletTest.java | 34 ++++++++++++++++++- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java index a7130bcc190d..a6566cd07330 100644 --- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -283,7 +283,8 @@ protected void doPost(HttpServletRequest request, HttpServletResponse response) } // Send a 401 to the client response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); - if(isAuthTypeEnabled(request, HiveAuthConstants.AuthTypes.KERBEROS)) { + if (e instanceof HttpEmptyAuthenticationException && + authType.isEnabled(HiveAuthConstants.AuthTypes.KERBEROS)) { response.addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); } else { try { diff --git a/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java b/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java index 02b24dd846b3..8278331958c8 100644 --- a/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java +++ b/service/src/test/org/apache/hive/service/cli/thrift/ThriftHttpServletTest.java @@ -21,7 +21,6 @@ import org.apache.hive.service.auth.HiveAuthConstants; import org.apache.hive.service.auth.HttpAuthUtils; import org.apache.hive.service.auth.ldap.HttpEmptyAuthenticationException; -import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -76,4 +75,37 @@ public void testEmptyAuthorizationHeader() throws Exception { thriftHttpServlet.doKerberosAuth(httpServletRequest); } + @Test + public void testWwwAuthenticateNegotiateHeaderAddedToTheResponse() throws Exception { + HttpServletRequest mockRequest = Mockito.mock(HttpServletRequest.class); + HttpServletResponse mockResponse = Mockito.mock(HttpServletResponse.class); + PrintWriter mockPrintWriter = Mockito.mock(PrintWriter.class); + Mockito.when(mockResponse.getWriter()).thenReturn(mockPrintWriter); + + thriftHttpServlet.doPost(mockRequest, mockResponse); + + Mockito.verify(mockResponse) + .setStatus(HttpServletResponse.SC_UNAUTHORIZED); + Mockito.verify(mockPrintWriter) + .println("Authentication Error: Authorization header received from the client is empty."); + Mockito.verify(mockResponse) + .addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); + } + + @Test + public void testWwwAuthenticateNegotiateHeaderNotAddedToTheResponseWhenNotEmptyAuthorizationHeaderExists() throws Exception { + HttpServletRequest mockRequest = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockRequest.getHeader(HttpAuthUtils.AUTHORIZATION)).thenReturn("Authorization: Negotiate"); + HttpServletResponse mockResponse = Mockito.mock(HttpServletResponse.class); + PrintWriter mockPrintWriter = Mockito.mock(PrintWriter.class); + Mockito.when(mockResponse.getWriter()).thenReturn(mockPrintWriter); + + thriftHttpServlet.doPost(mockRequest, mockResponse); + + Mockito.verify(mockResponse) + .setStatus(HttpServletResponse.SC_UNAUTHORIZED); + Mockito.verify(mockResponse, Mockito.times(0)) + .addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); + } + } From 6cd53ed2bbbe8ff1f5e52fcfd15e2d55b7ee393f Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Wed, 27 Sep 2023 22:04:23 +0200 Subject: [PATCH 010/179] HIVE-27747: Generalize TestSchemaToolForMetastore to run on every supported DBMS (Stamatis Zampetakis reviewed by Zhihua Deng) The goal is to make the tests run on every supported metastore DBMS (not only Derby), to increase code coverage for the SchemaTool. An overview of the changes can be seen below. 1. Parameterize the test using all available dockerized DBMS (except Mariadb which currently fails due to HIVE-27749) 2. Refactor hardcoded references to derby with the appropriate database type 3. Add missing semicolons in some statements in the test scripts 4. Use appropriate DDL statements for renaming a table based on the underlying database 5. Post-process DML statements in test scripts to: * quote table/column identifiers when necessary (only Postgres at the moment) * pick appropriate literal for indicating false in boolean/bit datatype based on the database Close apache/hive#4754 --- .../TestSchemaToolForMetastore.java | 149 +++++++++++++----- 1 file changed, 110 insertions(+), 39 deletions(-) diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java index f6a029a44706..e82b943915bf 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java @@ -29,10 +29,14 @@ import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.SQLException; -import java.util.Random; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; import org.apache.commons.dbcp2.DelegatingConnection; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.text.StrTokenizer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.HiveMetaException; @@ -40,39 +44,64 @@ import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory; import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; +import org.apache.hadoop.hive.metastore.dbinstall.rules.DatabaseRule; +import org.apache.hadoop.hive.metastore.dbinstall.rules.Derby; +import org.apache.hadoop.hive.metastore.dbinstall.rules.Mssql; +import org.apache.hadoop.hive.metastore.dbinstall.rules.Mysql; +import org.apache.hadoop.hive.metastore.dbinstall.rules.Oracle; +import org.apache.hadoop.hive.metastore.dbinstall.rules.Postgres; + import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static java.lang.String.format; @Category(MetastoreCheckinTest.class) +@RunWith(Parameterized.class) public class TestSchemaToolForMetastore { + private static final Pattern IDENTIFIER = Pattern.compile("[A-Z_]+"); + private MetastoreSchemaTool schemaTool; private Connection conn; private Configuration conf; - private String testMetastoreDB; + private final DatabaseRule dbms; private PrintStream errStream; private PrintStream outStream; - private String argsBase; private SchemaToolTaskValidate validator; + public TestSchemaToolForMetastore(DatabaseRule dbms){ + this.dbms = dbms; + } + + @Parameterized.Parameters(name = "{0}") + public static Collection databases() { + List dbs = new ArrayList<>(); + dbs.add(new Object[] { new Derby(true) }); + dbs.add(new Object[] { new Mysql() }); + dbs.add(new Object[] { new Oracle() }); + dbs.add(new Object[] { new Postgres() }); +// dbs.add(new Object[] { new Mariadb() }); Disabled due to HIVE-27749 + dbs.add(new Object[] { new Mssql() }); + return dbs; + } + @Before - public void setUp() throws HiveMetaException, IOException { - testMetastoreDB = System.getProperty("java.io.tmpdir") + - File.separator + "test_metastore-" + new Random().nextInt(); - System.setProperty(ConfVars.CONNECT_URL_KEY.toString(), - "jdbc:derby:" + testMetastoreDB + ";create=true"); + public void setUp() throws Exception { + dbms.before(); + dbms.createUser(); conf = MetastoreConf.newMetastoreConf(); schemaTool = new MetastoreSchemaTool(); schemaTool.init(System.getProperty("test.tmp.dir", "target/tmp"), - new String[]{"-dbType", "derby", "--info"}, null, conf); - String userName = MetastoreConf.getVar(schemaTool.getConf(), ConfVars.CONNECTION_USER_NAME); - String passWord = MetastoreConf.getPassword(schemaTool.getConf(), ConfVars.PWD); - schemaTool.setUserName(userName); - schemaTool.setPassWord(passWord); - argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " "; + new String[]{"-dbType", dbms.getDbType(), "--info"}, null, conf); + schemaTool.setUserName(dbms.getHiveUser()); + schemaTool.setPassWord(dbms.getHivePassword()); + schemaTool.setUrl(dbms.getJdbcUrl()); + schemaTool.setDriver(dbms.getJdbcDriver()); System.setProperty("beeLine.system.exit", "true"); errStream = System.err; outStream = System.out; @@ -84,15 +113,12 @@ public void setUp() throws HiveMetaException, IOException { @After public void tearDown() throws IOException, SQLException { - File metaStoreDir = new File(testMetastoreDB); - if (metaStoreDir.exists()) { - FileUtils.forceDeleteOnExit(metaStoreDir); - } System.setOut(outStream); System.setErr(errStream); if (conn != null) { conn.close(); } + dbms.after(); } /* @@ -147,23 +173,27 @@ public void testValidateSchemaTables() throws Exception { Assert.assertTrue(isValid); // Simulate a missing table scenario by renaming a couple of tables - String[] scripts = new String[] { - "RENAME TABLE SEQUENCE_TABLE to SEQUENCE_TABLE_RENAMED;", - "RENAME TABLE NUCLEUS_TABLES to NUCLEUS_TABLES_RENAMED;" - }; + Map tblRenames = new HashMap<>(); + tblRenames.put("SEQUENCE_TABLE", "SEQUENCE_TABLE_RENAMED"); + if (!dbms.getDbType().equals("mssql")) { + // HIVE-27748: NUCLEUS_TABLES DDLs are missing from MSSQL metastore installation scripts + tblRenames.put("NUCLEUS_TABLES", "NUCLEUS_TABLES_RENAMED"); + } + String[] deleteScripts = new String[tblRenames.size()]; + String[] restoreScripts = new String[tblRenames.size()]; + int i = 0; + for (Map.Entry namePair : tblRenames.entrySet()) { + deleteScripts[i] = renameTableStmt(namePair.getKey(), namePair.getValue()); + restoreScripts[i] = renameTableStmt(namePair.getValue(), namePair.getKey()); + i++; + } - File scriptFile = generateTestScript(scripts); + File scriptFile = generateTestScript(deleteScripts); schemaTool.execSql(scriptFile.getPath()); isValid = validator.validateSchemaTables(conn); Assert.assertFalse(isValid); - // Restored the renamed tables - scripts = new String[] { - "RENAME TABLE SEQUENCE_TABLE_RENAMED to SEQUENCE_TABLE;", - "RENAME TABLE NUCLEUS_TABLES_RENAMED to NUCLEUS_TABLES;" - }; - - scriptFile = generateTestScript(scripts); + scriptFile = generateTestScript(restoreScripts); schemaTool.execSql(scriptFile.getPath()); isValid = validator.validateSchemaTables(conn); Assert.assertTrue(isValid); @@ -248,7 +278,7 @@ public void testSchemaUpgradeDryRun() throws Exception { @Test public void testSchemaInit() throws Exception { IMetaStoreSchemaInfo metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(conf, - System.getProperty("test.tmp.dir", "target/tmp"), "derby"); + System.getProperty("test.tmp.dir", "target/tmp"), dbms.getDbType()); execute(new SchemaToolTaskInit(), "-initSchemaTo " + metastoreSchemaInfo.getHiveSchemaVersion()); schemaTool.verifySchemaVersion(); } @@ -335,15 +365,16 @@ public void testSchemaUpgrade() throws Exception { "Hive operations shouldn't pass with older version schema"); } + String db = dbms.getDbType(); // Generate dummy pre-upgrade script with errors String invalidPreUpgradeScript = writeDummyPreUpgradeScript( - 0, "upgrade-2.3.0-to-3.0.0.derby.sql", "foo bar;"); + 0, "upgrade-2.3.0-to-3.0.0."+db+".sql", "foo bar;"); // Generate dummy pre-upgrade scripts with valid SQL String validPreUpgradeScript0 = writeDummyPreUpgradeScript( - 1, "upgrade-2.3.0-to-3.0.0.derby.sql", + 1, "upgrade-2.3.0-to-3.0.0."+db+".sql", "CREATE TABLE schema_test0 (id integer);"); String validPreUpgradeScript1 = writeDummyPreUpgradeScript( - 2, "upgrade-2.3.0-to-3.0.0.derby.sql", + 2, "upgrade-2.3.0-to-3.0.0."+db+".sql", "CREATE TABLE schema_test1 (id integer);"); // Capture system out and err @@ -473,6 +504,8 @@ private File generateTestScript(String [] stmts) throws IOException { FileWriter fstream = new FileWriter(testScriptFile.getPath()); BufferedWriter out = new BufferedWriter(fstream); for (String line: stmts) { + line = quoteIdentifiers(line); + line = line.replaceAll("'[Nn]'", booleanFalse()); out.write(line); out.newLine(); } @@ -480,13 +513,49 @@ private File generateTestScript(String [] stmts) throws IOException { return testScriptFile; } + private String quoteIdentifiers(String line) { + if (!dbms.getDbType().equalsIgnoreCase("postgres")) { + return line; + } + String idWithQuote = "\"$0\""; + String[] part = line.split("values"); + if (part.length == 2) { + return IDENTIFIER.matcher(part[0]).replaceAll(idWithQuote) + " values " + part[1]; + } else { + return IDENTIFIER.matcher(line).replaceAll(idWithQuote); + } + } + + private String booleanFalse() { + switch (dbms.getDbType()) { + case "derby": + return "'N'"; + case "postgres": + return "'0'"; + default: + return "0"; + } + } + + private String renameTableStmt(String oldName, String newName) { + switch (dbms.getDbType()) { + case "mssql": + return format("exec sp_rename '%s', '%s';", oldName, newName); + case "postgres": + case "oracle": + return format("alter table %s rename to %s;", oldName, newName); + default: + return format("rename table %s to %s;", oldName, newName); + } + } + private void validateMetastoreDbPropertiesTable() throws HiveMetaException, IOException { boolean isValid = (boolean) validator.validateSchemaTables(conn); Assert.assertTrue(isValid); // adding same property key twice should throw unique key constraint violation exception String[] scripts = new String[] { - "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-1', 'dummy uuid 1')", - "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-2', 'dummy uuid 2')", }; + "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-1', 'dummy uuid 1');", + "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-2', 'dummy uuid 2');", }; File scriptFile = generateTestScript(scripts); Exception ex = null; try { @@ -505,7 +574,7 @@ private String writeDummyPreUpgradeScript(int index, String upgradeScriptName, String preUpgradeScript = "pre-" + index + "-" + upgradeScriptName; String dummyPreScriptPath = System.getProperty("test.tmp.dir", "target/tmp") + File.separatorChar + "scripts" + File.separatorChar + "metastore" + - File.separatorChar + "upgrade" + File.separatorChar + "derby" + + File.separatorChar + "upgrade" + File.separatorChar + dbms.getDbType() + File.separatorChar + preUpgradeScript; FileWriter fstream = new FileWriter(dummyPreScriptPath); BufferedWriter out = new BufferedWriter(fstream); @@ -547,6 +616,8 @@ public DatabaseMetaData getMetaData() throws SQLException { } private void execute(SchemaToolTask task, String taskArgs) throws HiveMetaException { + String argsBase = + format("-dbType %s -userName %s -passWord %s ", dbms.getDbType(), dbms.getHiveUser(), dbms.getHivePassword()); try { StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"'); SchemaToolCommandLine cl = new SchemaToolCommandLine(tokenizer.getTokenArray(), null); From 1843105f27f0b3550f70f71e2bf44830c79a6c69 Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Thu, 28 Sep 2023 14:27:15 +0200 Subject: [PATCH 011/179] HIVE-27755: Quote identifiers in SQL emitted by SchemaTool for MySQL (Stamatis Zampetakis reviewed by Zhihua Deng) The main motivation behind this change is to avoid unexpected query failures when/if the MySQL database decides to turn some identifier names we are using internally to reserved keywords. This is a rather likely scenario considering that recently a MySQL fork (https://docs.percona.com/percona-server/8.0/flexibility/sequence_table.html) turned SEQUENCE_TABLE into a reserved keyword. Close apache/hive#4747 --- .../hive/metastore/MetaStoreSchemaInfo.java | 16 ++++++---------- .../tools/schematool/HiveSchemaHelper.java | 5 +++++ .../tools/schematool/MetastoreSchemaTool.java | 4 ++++ 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java index 223455d40a01..d5e35d6b067d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hive.metastore.tools.schematool.HiveSchemaHelper.MetaStoreConnectionInfo; import org.apache.hadoop.hive.metastore.utils.MetastoreVersionInfo; +import static org.apache.hadoop.hive.metastore.tools.schematool.MetastoreSchemaTool.quote; public class MetaStoreSchemaInfo implements IMetaStoreSchemaInfo { protected static final String UPGRADE_FILE_PREFIX = "upgrade-"; @@ -45,6 +46,7 @@ public class MetaStoreSchemaInfo implements IMetaStoreSchemaInfo { protected static final String VERSION_UPGRADE_LIST = "upgrade.order"; protected static final String PRE_UPGRADE_PREFIX = "pre-"; protected static final String CREATE_USER_PREFIX = "create-user"; + private static final String VERSION_QUERY = "SELECT t.SCHEMA_VERSION from VERSION t"; private String[] hiveSchemaVersions; private final String metastoreHome; @@ -234,18 +236,12 @@ private int compareVersion(String dbVerPart, String hiveVerPart) { @Override public String getMetaStoreSchemaVersion(MetaStoreConnectionInfo connectionInfo) throws HiveMetaException { - String versionQuery; - boolean needsQuotedIdentifier = HiveSchemaHelper.getDbCommandParser(connectionInfo.getDbType(), - connectionInfo.getMetaDbType(), false).needsQuotedIdentifier(); - if (needsQuotedIdentifier) { - versionQuery = "select t.\"SCHEMA_VERSION\" from \"VERSION\" t"; - } else { - versionQuery = "select t.SCHEMA_VERSION from VERSION t"; - } - String schema = ( HiveSchemaHelper.DB_HIVE.equals(connectionInfo.getDbType()) ? "SYS" : null ); + HiveSchemaHelper.NestedScriptParser db = + HiveSchemaHelper.getDbCommandParser(connectionInfo.getDbType(), connectionInfo.getMetaDbType(), false); + String schema = (HiveSchemaHelper.DB_HIVE.equals(connectionInfo.getDbType()) ? "SYS" : null); try (Connection metastoreDbConnection = HiveSchemaHelper.getConnectionToMetastore(connectionInfo, schema); Statement stmt = metastoreDbConnection.createStatement()) { - ResultSet res = stmt.executeQuery(versionQuery); + ResultSet res = stmt.executeQuery(quote(VERSION_QUERY, db.needsQuotedIdentifier(), db.getQuoteCharacter())); if (!res.next()) { throw new HiveMetaException("Could not find version info in metastore VERSION table."); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/HiveSchemaHelper.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/HiveSchemaHelper.java index cfec036559f8..fe9dbb8ccb3d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/HiveSchemaHelper.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/HiveSchemaHelper.java @@ -454,6 +454,11 @@ public String getQuoteCharacter() { return "`"; } + @Override + public boolean needsQuotedIdentifier() { + return true; + } + @Override public boolean isNonExecCommand(String dbCommand) { return super.isNonExecCommand(dbCommand) || diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/MetastoreSchemaTool.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/MetastoreSchemaTool.java index c373b32138d9..849cec01adb9 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/MetastoreSchemaTool.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/MetastoreSchemaTool.java @@ -345,6 +345,10 @@ protected void testConnectionToMetastore() throws HiveMetaException { // Quote if the database requires it protected String quote(String stmt) { + return quote(stmt, needsQuotedIdentifier, quoteCharacter); + } + + public static String quote(String stmt, boolean needsQuotedIdentifier, String quoteCharacter) { stmt = stmt.replace("", needsQuotedIdentifier ? quoteCharacter : ""); stmt = stmt.replace("", quoteCharacter); return stmt; From 246392dcbbe466e36d0ea98c1d7eeaa48e579ad1 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sat, 14 Oct 2023 16:34:30 +0800 Subject: [PATCH 012/179] HIVE-27793: Iceberg: Support setting current snapshot with SnapshotRef (#4797). (zhangbutao, reviewed by Ayush Saxena) --- .../mr/hive/HiveIcebergStorageHandler.java | 4 +- .../iceberg/mr/hive/IcebergTableUtil.java | 23 +++++++++--- .../TestHiveIcebergSetCurrentSnapshot.java | 37 +++++++++++++++++++ .../hadoop/hive/ql/parse/AlterClauseParser.g | 2 +- .../execute/AlterTableExecuteAnalyzer.java | 2 +- .../hive/ql/parse/AlterTableExecuteSpec.java | 14 +++---- 6 files changed, 66 insertions(+), 16 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 98fb25b1d374..8214c9e68034 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -817,8 +817,8 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable, AlterTableExecuteSpec.SetCurrentSnapshotSpec setSnapshotVersionSpec = (AlterTableExecuteSpec.SetCurrentSnapshotSpec) executeSpec.getOperationParams(); LOG.debug("Executing set current snapshot operation on iceberg table {}.{} to version {}", hmsTable.getDbName(), - hmsTable.getTableName(), setSnapshotVersionSpec.getSnapshotId()); - IcebergTableUtil.setCurrentSnapshot(icebergTable, setSnapshotVersionSpec.getSnapshotId()); + hmsTable.getTableName(), setSnapshotVersionSpec.getSnapshotIdOrRefName()); + IcebergTableUtil.setCurrentSnapshot(icebergTable, setSnapshotVersionSpec.getSnapshotIdOrRefName()); break; case FAST_FORWARD: AlterTableExecuteSpec.FastForwardSpec fastForwardSpec = diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java index 9bcd321bf703..8527e25cbfe6 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.function.Function; import org.apache.hadoop.conf.Configuration; @@ -28,10 +29,12 @@ import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; import org.apache.hadoop.hive.ql.parse.TransformSpec; +import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.session.SessionStateUtil; import org.apache.iceberg.ManageSnapshots; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdatePartitionSpec; @@ -235,13 +238,23 @@ public static void rollback(Table table, AlterTableExecuteSpec.RollbackSpec.Roll /** * Set the current snapshot for the iceberg table * @param table the iceberg table - * @param value parameter of the rollback, that can be a timestamp in millis or a snapshot id + * @param value parameter of the rollback, that can be a snapshot id or a SnapshotRef name */ - public static void setCurrentSnapshot(Table table, Long value) { + public static void setCurrentSnapshot(Table table, String value) { ManageSnapshots manageSnapshots = table.manageSnapshots(); - LOG.debug("Rolling the iceberg table {} from snapshot id {} to snapshot ID {}", table.name(), - table.currentSnapshot().snapshotId(), value); - manageSnapshots.setCurrentSnapshot(value); + long snapshotId; + try { + snapshotId = Long.parseLong(value); + LOG.debug("Rolling the iceberg table {} from snapshot id {} to snapshot ID {}", table.name(), + table.currentSnapshot().snapshotId(), snapshotId); + } catch (NumberFormatException e) { + String refName = PlanUtils.stripQuotes(value); + snapshotId = Optional.ofNullable(table.refs().get(refName)).map(SnapshotRef::snapshotId).orElseThrow(() -> + new IllegalArgumentException(String.format("SnapshotRef %s does not exist", refName))); + LOG.debug("Rolling the iceberg table {} from snapshot id {} to the snapshot ID {} of SnapshotRef {}", + table.name(), table.currentSnapshot().snapshotId(), snapshotId, refName); + } + manageSnapshots.setCurrentSnapshot(snapshotId); manageSnapshots.commit(); } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java index e45e4b26ee0f..0fddae1b83be 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java @@ -21,9 +21,12 @@ import java.io.IOException; import java.util.List; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; /** @@ -60,4 +63,38 @@ public void testSetCurrentSnapshot() throws IOException, InterruptedException { HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, currentResult), HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, result4), 0); } + + @Test + public void testSetCurrentSnapshotBySnapshotRef() throws IOException, InterruptedException { + // enough to test once + Assume.assumeTrue(fileFormat == FileFormat.ORC && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG); + TableIdentifier identifier = TableIdentifier.of("default", "source"); + Table table = + testTables.createTableWithVersions(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 5); + shell.executeStatement("ALTER TABLE " + identifier.name() + " CREATE TAG test_tag"); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE SET_CURRENT_SNAPSHOT('test_tag')"); + table.refresh(); + Assert.assertEquals(table.currentSnapshot().snapshotId(), table.refs().get("test_tag").snapshotId()); + + shell.executeStatement("ALTER TABLE " + identifier.name() + " CREATE BRANCH test_branch"); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE SET_CURRENT_SNAPSHOT('test_branch')"); + table.refresh(); + Assert.assertEquals(table.currentSnapshot().snapshotId(), table.refs().get("test_branch").snapshotId()); + + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "SnapshotRef unknown_ref does not exist", () -> { + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE SET_CURRENT_SNAPSHOT('unknown_ref')"); + }); + + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE SET_CURRENT_SNAPSHOT" + + "(" + table.currentSnapshot().snapshotId() + ")"); + + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "SnapshotRef " + table.currentSnapshot().snapshotId() + " does not exist", () -> { + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE SET_CURRENT_SNAPSHOT" + + "('" + table.currentSnapshot().snapshotId() + "')"); + }); + } } diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g index 46a00fe5c873..28d60d6262f2 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g @@ -477,7 +477,7 @@ alterStatementSuffixExecute -> ^(TOK_ALTERTABLE_EXECUTE KW_ROLLBACK $rollbackParam) | KW_EXECUTE KW_EXPIRE_SNAPSHOTS LPAREN (expireParam=StringLiteral) RPAREN -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS $expireParam) - | KW_EXECUTE KW_SET_CURRENT_SNAPSHOT LPAREN (snapshotParam=Number) RPAREN + | KW_EXECUTE KW_SET_CURRENT_SNAPSHOT LPAREN (snapshotParam=expression) RPAREN -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam) | KW_EXECUTE KW_FAST_FORWARD sourceBranch=StringLiteral (targetBranch=StringLiteral)? -> ^(TOK_ALTERTABLE_EXECUTE KW_FAST_FORWARD $sourceBranch $targetBranch?) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java index ddc12935700d..8d4a902b56b4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java @@ -128,7 +128,7 @@ private static AlterTableExecuteDesc getSetCurrentSnapshotDesc(TableName tableNa ASTNode childNode) throws SemanticException { AlterTableExecuteSpec spec = new AlterTableExecuteSpec(SET_CURRENT_SNAPSHOT, - new AlterTableExecuteSpec.SetCurrentSnapshotSpec(Long.valueOf(childNode.getText()))); + new AlterTableExecuteSpec.SetCurrentSnapshotSpec(childNode.getText())); return new AlterTableExecuteDesc(tableName, partitionSpec, spec); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java index c469b24415f6..5102959f087b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java @@ -161,23 +161,23 @@ public String toString() { /** * Value object class, that stores the set snapshot version operation specific parameters *
    - *
  • snapshot Id: it should be a valid snapshot version
  • + *
  • snapshot Id: it should be a valid snapshot version or a SnapshotRef name
  • *
*/ public static class SetCurrentSnapshotSpec { - private final long snapshotId; + private final String snapshotIdOrRefName; - public SetCurrentSnapshotSpec(Long snapshotId) { - this.snapshotId = snapshotId; + public SetCurrentSnapshotSpec(String snapshotIdOrRefName) { + this.snapshotIdOrRefName = snapshotIdOrRefName; } - public Long getSnapshotId() { - return snapshotId; + public String getSnapshotIdOrRefName() { + return snapshotIdOrRefName; } @Override public String toString() { - return MoreObjects.toStringHelper(this).add("snapshotId", snapshotId).toString(); + return MoreObjects.toStringHelper(this).add("snapshotIdOrRefName", snapshotIdOrRefName).toString(); } } From da13ee3d39bd825c1c7b86a76b68a0177008456e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 14 Oct 2023 14:06:28 +0530 Subject: [PATCH 013/179] HIVE-27799: Bump org.apache.zookeeper:zookeeper from 3.7.1 to 3.7.2 (#4794). (dependabot, Reviewed by Ayush Saxena) --- pom.xml | 2 +- standalone-metastore/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index d7e5f90f2c24..20ab0f0bf3be 100644 --- a/pom.xml +++ b/pom.xml @@ -207,7 +207,7 @@ 2.3 2.12.2 2.3.0 - 3.7.1 + 3.7.2 1.1 2.4.0 5.2.0 diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml index 0c623d1db851..9b0286e24798 100644 --- a/standalone-metastore/pom.xml +++ b/standalone-metastore/pom.xml @@ -99,7 +99,7 @@ 1.9.4 1.3 5.2.0 - 3.7.1 + 3.7.2 9.1.6 4.0.3 2.8.4 From c126422a91be695c75ec4a750638a0aa4d1ba6cd Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Wed, 11 Oct 2023 12:17:23 +0200 Subject: [PATCH 014/179] HIVE-27695: HIVE-26828: Intermittent OOM when running TestMiniTezCliDriver (Stamatis Zampetakis reviewed by Ayush Saxena) java.lang.OutOfMemoryError: GC overhead limit exceeded is thrown by the Tez Application Master (AM) cause the current heap size (128MB) is not enough to accommodate the needs of multiple Tez containers running. Each running container requires roughly 10MB of memory in the AM. The AM accumulates/manipulates multiple configuration objects (some of them retaining as much as 1MB of heap) per container. The heap gradually becomes full and GC is spending a lot of CPU time to clean things up without really making much progress since containers are reused and heap cannot shrink. There are multiple solutions to the problem but the easiest and most effective is to increase the heap size for the AM. At this point in time, 512MB is a good value. The hybridgrace_hashjoin_2.q test which was failing to due OOM can now be re-enabled. Close apache/hive#4792 --- data/conf/tez/tez-site.xml | 2 +- ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/data/conf/tez/tez-site.xml b/data/conf/tez/tez-site.xml index ba4df319a559..88adb6a57e80 100644 --- a/data/conf/tez/tez-site.xml +++ b/data/conf/tez/tez-site.xml @@ -1,7 +1,7 @@ tez.am.resource.memory.mb - 128 + 512 tez.task.resource.memory.mb diff --git a/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q b/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q index 93b8c13a49f3..6ed771ba685b 100644 --- a/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q +++ b/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q @@ -1,7 +1,6 @@ --! qt:dataset:srcpart --! qt:dataset:src1 --! qt:dataset:src ---! qt:disabled:HIVE-26820 Disable hybridgrace_hashjoin_2.q flaky test set hive.mapred.mode=nonstrict; set hive.explain.user=false; -- Hybrid Grace Hash Join From 357714abbf196edf1ebbfbdfedae831a8d58db46 Mon Sep 17 00:00:00 2001 From: yi linwei Date: Mon, 16 Oct 2023 16:18:50 +0800 Subject: [PATCH 015/179] HIVE-27798: Correct configuration item in hive-site.xml in docker. (#4803). (xiaolin84250 , Reviewed by Ayush Saxena) --- packaging/src/docker/conf/hive-site.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packaging/src/docker/conf/hive-site.xml b/packaging/src/docker/conf/hive-site.xml index d382d489efc5..89823059eb7d 100644 --- a/packaging/src/docker/conf/hive-site.xml +++ b/packaging/src/docker/conf/hive-site.xml @@ -53,7 +53,7 @@ tez - metastore.warehouse.dir + hive.metastore.warehouse.dir /opt/hive/data/warehouse From 5d58a210375ec80a3b6a956f8cb11416b7972b54 Mon Sep 17 00:00:00 2001 From: Zoltan Ratkai <117656751+zratkai@users.noreply.github.com> Date: Mon, 16 Oct 2023 10:36:58 +0200 Subject: [PATCH 016/179] HIVE-27686 ORC upgraded to 1.8.5. (#4690) (Zoltan Ratkai reviewed by Laszlo Bodor) --- .../hive/ql/txn/compactor/TestCompactor.java | 6 - .../txn/compactor/TestCrudCompactorOnTez.java | 105 ++++++++++-------- pom.xml | 2 +- .../hadoop/hive/ql/TestTxnNoBuckets.java | 2 +- .../acid_bloom_filter_orc_file_dump.q | 1 + .../queries/clientpositive/acid_no_buckets.q | 1 + .../clientpositive/default_constraint.q | 1 + .../queries/clientpositive/deleteAnalyze.q | 1 + .../materialized_view_create_rewrite.q | 1 + .../materialized_view_create_rewrite_4.q | 2 +- ...zed_view_create_rewrite_by_text_multi_db.q | 1 + .../materialized_view_create_rewrite_dummy.q | 1 + ...aterialized_view_create_rewrite_multi_db.q | 1 + ...rialized_view_create_rewrite_time_window.q | 1 + ...alized_view_create_rewrite_time_window_2.q | 1 + .../test/queries/clientpositive/orc_analyze.q | 1 + .../queries/clientpositive/orc_file_dump.q | 1 + .../clientpositive/orc_llap_counters.q | 1 + .../clientpositive/orc_llap_counters1.q | 1 + .../test/queries/clientpositive/orc_merge10.q | 2 +- .../test/queries/clientpositive/orc_merge11.q | 1 + .../test/queries/clientpositive/orc_merge12.q | 1 + ql/src/test/queries/clientpositive/row__id.q | 1 + .../queries/clientpositive/smb_mapjoin_1.q | 1 + .../queries/clientpositive/sqlmerge_stats.q | 1 + .../queries/clientpositive/stats_histogram.q | 1 + .../clientpositive/stats_histogram_null.q | 1 + .../test/queries/clientpositive/stats_part2.q | 1 + .../materialized_view_create_rewrite.q.out | 4 +- .../beeline/smb_mapjoin_1.q.out | 2 +- .../acid_bloom_filter_orc_file_dump.q.out | 4 +- .../clientpositive/llap/acid_no_buckets.q.out | 4 +- .../llap/default_constraint.q.out | 14 +-- .../clientpositive/llap/deleteAnalyze.q.out | 4 +- .../materialized_view_create_rewrite.q.out | 4 +- .../materialized_view_create_rewrite_4.q.out | 12 +- ...view_create_rewrite_by_text_multi_db.q.out | 4 +- ...terialized_view_create_rewrite_dummy.q.out | 4 +- ...ialized_view_create_rewrite_multi_db.q.out | 4 +- ...ized_view_create_rewrite_time_window.q.out | 6 +- ...ed_view_create_rewrite_time_window_2.q.out | 6 +- .../clientpositive/llap/orc_analyze.q.out | 34 +++--- .../clientpositive/llap/orc_file_dump.q.out | 6 +- .../llap/orc_llap_counters.q.out | 2 +- .../llap/orc_llap_counters1.q.out | 2 +- .../clientpositive/llap/orc_merge10.q.out | 4 +- .../clientpositive/llap/orc_merge11.q.out | 6 +- .../clientpositive/llap/sqlmerge_stats.q.out | 10 +- .../clientpositive/llap/stats_histogram.q.out | 2 +- .../llap/stats_histogram_null.q.out | 2 +- .../clientpositive/llap/stats_part2.q.out | 30 ++--- .../test/results/clientpositive/row__id.q.out | 18 +-- .../clientpositive/tez/orc_merge12.q.out | 2 +- standalone-metastore/pom.xml | 2 +- 54 files changed, 179 insertions(+), 154 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java index 7083df59828c..fd6e111df0ac 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java @@ -398,7 +398,6 @@ public void testStatsAfterCompactionPartTbl() throws Exception { .getParameters(); Assert.assertEquals("The number of files is differing from the expected", "2", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "1396", parameters.get("totalSize")); parameters = partitions .stream() @@ -408,7 +407,6 @@ public void testStatsAfterCompactionPartTbl() throws Exception { .getParameters(); Assert.assertEquals("The number of files is differing from the expected", "2", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "1453", parameters.get("totalSize")); //Do a major compaction CompactionRequest rqst = new CompactionRequest(dbName, tblName, CompactionType.MAJOR); @@ -433,7 +431,6 @@ public void testStatsAfterCompactionPartTbl() throws Exception { .getParameters(); Assert.assertEquals("The number of files is differing from the expected", "1", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "808", parameters.get("totalSize")); parameters = partitions .stream() @@ -443,7 +440,6 @@ public void testStatsAfterCompactionPartTbl() throws Exception { .getParameters(); Assert.assertEquals("The number of files is differing from the expected", "2", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "1453", parameters.get("totalSize")); } /** @@ -486,7 +482,6 @@ public void testStatsAfterCompactionTbl() throws Exception { Map parameters = Hive.get().getTable(tblName).getParameters(); Assert.assertEquals("The number of files is differing from the expected", "2", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "1446", parameters.get("totalSize")); //Do a major compaction CompactionRequest rqst = new CompactionRequest(dbName, tblName, CompactionType.MAJOR); @@ -504,7 +499,6 @@ public void testStatsAfterCompactionTbl() throws Exception { parameters = Hive.get().getTable(tblName).getParameters(); Assert.assertEquals("The number of files is differing from the expected", "1", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "783", parameters.get("totalSize")); } @Test diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java index 838c4bd34952..e72811e103bf 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java @@ -108,8 +108,8 @@ private void testRebalanceCompactionWithParallelDeleteAsSecond(boolean optimisti conf.setBoolVar(HiveConf.ConfVars.TXN_WRITE_X_LOCK, optimisticLock); //set grouping size to have 3 buckets, and re-create driver with the new config - conf.set("tez.grouping.min-size", "1000"); - conf.set("tez.grouping.max-size", "80000"); + conf.set("tez.grouping.min-size", "400"); + conf.set("tez.grouping.max-size", "5000"); driver = new Driver(conf); final String tableName = "rebalance_test"; @@ -145,8 +145,8 @@ private void testRebalanceCompactionWithParallelDeleteAsSecond(boolean optimisti Assert.fail("In case of TXN_WRITE_X_LOCK = true, the transaction must be retried instead of being aborted."); } aborted = true; - Assert.assertEquals(e.getCause().getClass(), LockException.class); - Assert.assertEquals(e.getCauseMessage(), "Transaction manager has aborted the transaction txnid:21. Reason: Aborting [txnid:21,24] due to a write conflict on default/rebalance_test committed by [txnid:20,24] d/u"); + Assert.assertEquals(LockException.class, e.getCause().getClass()); + Assert.assertEquals( "Transaction manager has aborted the transaction txnid:21. Reason: Aborting [txnid:21,24] due to a write conflict on default/rebalance_test committed by [txnid:20,24] d/u", e.getCauseMessage()); // Delete the record, so the rest of the test can be the same in both cases executeStatementOnDriver("DELETE FROM " + tableName + " WHERE b = 12", driver); } finally { @@ -177,24 +177,26 @@ private void testRebalanceCompactionWithParallelDeleteAsSecond(boolean optimisti "{\"writeid\":7,\"bucketid\":536870912,\"rowid\":4}\t13\t13", }, { - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":6}\t2\t4", + "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":6}\t4\t4", "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":7}\t3\t4", - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":8}\t4\t4", + "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":8}\t2\t4", "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":9}\t5\t4", - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":10}\t6\t4", - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":11}\t5\t3", }, { - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":12}\t6\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":13}\t4\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":14}\t2\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":15}\t3\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":16}\t6\t2", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":17}\t5\t2", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":10}\t6\t4", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":11}\t5\t3", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":12}\t3\t3", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":13}\t2\t3", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":14}\t4\t3", + }, + { + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":15}\t6\t3", + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":16}\t5\t2", + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":17}\t6\t2", }, }; verifyRebalance(testDataProvider, tableName, null, expectedBuckets, - new String[] {"bucket_00000", "bucket_00001", "bucket_00002"}, "base_0000007_v0000020"); + new String[] {"bucket_00000", "bucket_00001", "bucket_00002", "bucket_00003"}, "base_0000007_v0000020"); } @Test @@ -204,8 +206,8 @@ public void testRebalanceCompactionOfNotPartitionedImplicitlyBucketedTableWithOr conf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false); //set grouping size to have 3 buckets, and re-create driver with the new config - conf.set("tez.grouping.min-size", "1000"); - conf.set("tez.grouping.max-size", "80000"); + conf.set("tez.grouping.min-size", "400"); + conf.set("tez.grouping.max-size", "5000"); driver = new Driver(conf); final String tableName = "rebalance_test"; @@ -228,27 +230,29 @@ public void testRebalanceCompactionOfNotPartitionedImplicitlyBucketedTableWithOr "{\"writeid\":7,\"bucketid\":536870912,\"rowid\":2}\t15\t15", "{\"writeid\":7,\"bucketid\":536870912,\"rowid\":3}\t14\t14", "{\"writeid\":7,\"bucketid\":536870912,\"rowid\":4}\t13\t13", - "{\"writeid\":7,\"bucketid\":536870912,\"rowid\":5}\t12\t12", }, { - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":6}\t2\t4", + "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":5}\t12\t12", + "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":6}\t4\t4", "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":7}\t3\t4", - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":8}\t4\t4", + "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":8}\t2\t4", "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":9}\t5\t4", - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":10}\t6\t4", - "{\"writeid\":7,\"bucketid\":536936448,\"rowid\":11}\t5\t3", }, { - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":12}\t6\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":13}\t4\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":14}\t2\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":15}\t3\t3", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":16}\t6\t2", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":17}\t5\t2", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":10}\t6\t4", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":11}\t5\t3", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":12}\t3\t3", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":13}\t2\t3", + "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":14}\t4\t3", + }, + { + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":15}\t6\t3", + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":16}\t5\t2", + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":17}\t6\t2", }, }; verifyRebalance(testDataProvider, tableName, null, expectedBuckets, - new String[] {"bucket_00000", "bucket_00001", "bucket_00002"}, "base_0000007_v0000020"); + new String[] {"bucket_00000", "bucket_00001", "bucket_00002","bucket_00003"}, "base_0000007_v0000020"); } @Test @@ -258,8 +262,8 @@ public void testRebalanceCompactionOfNotPartitionedImplicitlyBucketedTable() thr conf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false); //set grouping size to have 3 buckets, and re-create driver with the new config - conf.set("tez.grouping.min-size", "1000"); - conf.set("tez.grouping.max-size", "80000"); + conf.set("tez.grouping.min-size", "400"); + conf.set("tez.grouping.max-size", "5000"); driver = new Driver(conf); final String tableName = "rebalance_test"; @@ -279,27 +283,30 @@ public void testRebalanceCompactionOfNotPartitionedImplicitlyBucketedTable() thr "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":2}\t6\t3", "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":3}\t6\t4", "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":4}\t5\t2", - "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":5}\t5\t3", }, { + + "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":5}\t5\t3", "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":6}\t2\t4", "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":7}\t3\t3", "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":8}\t4\t4", "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":9}\t4\t3", - "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":10}\t2\t3", - "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":11}\t3\t4", }, { + "{\"writeid\":1,\"bucketid\":537001984,\"rowid\":10}\t2\t3", + "{\"writeid\":1,\"bucketid\":537001984,\"rowid\":11}\t3\t4", "{\"writeid\":2,\"bucketid\":537001984,\"rowid\":12}\t12\t12", "{\"writeid\":3,\"bucketid\":537001984,\"rowid\":13}\t13\t13", "{\"writeid\":4,\"bucketid\":537001984,\"rowid\":14}\t14\t14", - "{\"writeid\":5,\"bucketid\":537001984,\"rowid\":15}\t15\t15", - "{\"writeid\":6,\"bucketid\":537001984,\"rowid\":16}\t16\t16", - "{\"writeid\":7,\"bucketid\":537001984,\"rowid\":17}\t17\t17", + }, + { + "{\"writeid\":5,\"bucketid\":537067520,\"rowid\":15}\t15\t15", + "{\"writeid\":6,\"bucketid\":537067520,\"rowid\":16}\t16\t16", + "{\"writeid\":7,\"bucketid\":537067520,\"rowid\":17}\t17\t17", }, }; verifyRebalance(testDataProvider, tableName, null, expectedBuckets, - new String[] {"bucket_00000", "bucket_00001", "bucket_00002"}, "base_0000007_v0000020"); + new String[] {"bucket_00000", "bucket_00001", "bucket_00002","bucket_00003"}, "base_0000007_v0000020"); } @Test @@ -440,8 +447,8 @@ public void testRebalanceCompactionNotPartitionedExplicitBucketNumbers() throws conf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false); //set grouping size to have 3 buckets, and re-create driver with the new config - conf.set("tez.grouping.min-size", "1000"); - conf.set("tez.grouping.max-size", "80000"); + conf.set("tez.grouping.min-size", "400"); + conf.set("tez.grouping.max-size", "5000"); driver = new Driver(conf); final String tableName = "rebalance_test"; @@ -509,7 +516,7 @@ private TestDataProvider prepareRebalanceTestData(String tableName) throws Excep Table table = msClient.getTable("default", tableName); FileSystem fs = FileSystem.get(conf); Assert.assertEquals("Test setup does not match the expected: different buckets", - Arrays.asList("bucket_00000_0", "bucket_00001_0", "bucket_00002_0"), + Arrays.asList("bucket_00000_0", "bucket_00001_0", "bucket_00002_0","bucket_00003_0"), CompactorTestUtil.getBucketFileNames(fs, table, null, "base_0000001")); String[][] expectedBuckets = new String[][] { { @@ -519,7 +526,6 @@ private TestDataProvider prepareRebalanceTestData(String tableName) throws Excep "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":3}\t6\t4", "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":4}\t5\t2", "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":5}\t5\t3", - "{\"writeid\":1,\"bucketid\":536870912,\"rowid\":6}\t2\t4", "{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\t12\t12", "{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\t13\t13", "{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t14\t14", @@ -528,13 +534,16 @@ private TestDataProvider prepareRebalanceTestData(String tableName) throws Excep "{\"writeid\":7,\"bucketid\":536870912,\"rowid\":0}\t17\t17", }, { - "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t3\t3", - "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":1}\t4\t4", - "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":2}\t4\t3", - "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":3}\t2\t3", + "{\"writeid\":1,\"bucketid\":536936448,\"rowid\":0}\t2\t4", + }, + { + "{\"writeid\":1,\"bucketid\":537001984,\"rowid\":0}\t3\t3", + "{\"writeid\":1,\"bucketid\":537001984,\"rowid\":1}\t4\t4", + "{\"writeid\":1,\"bucketid\":537001984,\"rowid\":2}\t4\t3", }, { - "{\"writeid\":1,\"bucketid\":537001984,\"rowid\":0}\t3\t4", + "{\"writeid\":1,\"bucketid\":537067520,\"rowid\":0}\t2\t3", + "{\"writeid\":1,\"bucketid\":537067520,\"rowid\":1}\t3\t4", }, }; AcidOutputFormat.Options options = new AcidOutputFormat.Options(conf); @@ -826,7 +835,7 @@ public void testStatsAfterQueryCompactionOnTez() throws Exception { parameters = Hive.get().getTable(tblName).getParameters(); Assert.assertEquals("The number of files is differing from the expected", "1", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "2", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "735", parameters.get("totalSize")); + Assert.assertEquals("The total table size is differing from the expected", "736", parameters.get("totalSize")); } @Test diff --git a/pom.xml b/pom.xml index 20ab0f0bf3be..f76569deb4c3 100644 --- a/pom.xml +++ b/pom.xml @@ -182,7 +182,7 @@ 42.5.1 21.3.0.0 2.3 - 1.8.3 + 1.8.5 3.4.4 4.11.0 2.0.0-M5 diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java index f8ee483053f7..a00886bb9cd9 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java @@ -848,7 +848,7 @@ public void testCompactStatsGather() throws Exception { .getParameters(); Assert.assertEquals("The number of files is differing from the expected", "1", parameters.get("numFiles")); Assert.assertEquals("The number of rows is differing from the expected", "4", parameters.get("numRows")); - Assert.assertEquals("The total table size is differing from the expected", "704", parameters.get("totalSize")); + Assert.assertEquals("The total table size is differing from the expected", "705", parameters.get("totalSize")); } @Test diff --git a/ql/src/test/queries/clientpositive/acid_bloom_filter_orc_file_dump.q b/ql/src/test/queries/clientpositive/acid_bloom_filter_orc_file_dump.q index 30daaf86aff7..b00222789f59 100644 --- a/ql/src/test/queries/clientpositive/acid_bloom_filter_orc_file_dump.q +++ b/ql/src/test/queries/clientpositive/acid_bloom_filter_orc_file_dump.q @@ -1,3 +1,4 @@ +--! qt:replace:/(File Version:)(.+)/$1#Masked#/ SET hive.vectorized.execution.enabled=FALSE; SET hive.mapred.mode=nonstrict; diff --git a/ql/src/test/queries/clientpositive/acid_no_buckets.q b/ql/src/test/queries/clientpositive/acid_no_buckets.q index b3bcb85843bb..368c50cd7b5a 100644 --- a/ql/src/test/queries/clientpositive/acid_no_buckets.q +++ b/ql/src/test/queries/clientpositive/acid_no_buckets.q @@ -1,4 +1,5 @@ --! qt:dataset:srcpart +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ --this has 4 groups of tests --Acid tables w/o bucketing --the tests with bucketing (make sure we get the same results) diff --git a/ql/src/test/queries/clientpositive/default_constraint.q b/ql/src/test/queries/clientpositive/default_constraint.q index 36aa894c5f24..135f41e885ad 100644 --- a/ql/src/test/queries/clientpositive/default_constraint.q +++ b/ql/src/test/queries/clientpositive/default_constraint.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ -- create table -- numeric type diff --git a/ql/src/test/queries/clientpositive/deleteAnalyze.q b/ql/src/test/queries/clientpositive/deleteAnalyze.q index c60da972a9c2..c22b9e5463a4 100644 --- a/ql/src/test/queries/clientpositive/deleteAnalyze.q +++ b/ql/src/test/queries/clientpositive/deleteAnalyze.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ set hive.stats.autogather=true; set hive.explain.user=true; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q index 6b42cf307995..5d718625163b 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ -- SORT_QUERY_RESULTS set hive.vectorized.execution.enabled=false; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q index 24195eea10c2..e8a22c0a3f9f 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q @@ -1,6 +1,6 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ -- Test Incremental rebuild of materialized view with aggregate but without count(*) -- when source tables have delete operations since last rebuild. - SET hive.vectorized.execution.enabled=false; set hive.support.concurrency=true; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_by_text_multi_db.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_by_text_multi_db.q index 501e67e92b17..6e9034e2a556 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_by_text_multi_db.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_by_text_multi_db.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.support.concurrency=true; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q index 1aadbffad52a..bdeb146aecf0 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ -- SORT_QUERY_RESULTS SET hive.vectorized.execution.enabled=false; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q index 5d390629ea1d..dc07ae695222 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.support.concurrency=true; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q index fa07401611dd..d85e33dcb87c 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ set hive.support.concurrency=true; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; set hive.strict.checks.cartesian.product=false; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q index 418ec45c91de..df4af8f3d8fe 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ set hive.support.concurrency=true; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; set hive.strict.checks.cartesian.product=false; diff --git a/ql/src/test/queries/clientpositive/orc_analyze.q b/ql/src/test/queries/clientpositive/orc_analyze.q index 056663ccb5d7..97f27ac0703f 100644 --- a/ql/src/test/queries/clientpositive/orc_analyze.q +++ b/ql/src/test/queries/clientpositive/orc_analyze.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.mapred.mode=nonstrict; set hive.exec.submitviachild=false; diff --git a/ql/src/test/queries/clientpositive/orc_file_dump.q b/ql/src/test/queries/clientpositive/orc_file_dump.q index 0aec810fc6a6..16eb8c8f1df1 100644 --- a/ql/src/test/queries/clientpositive/orc_file_dump.q +++ b/ql/src/test/queries/clientpositive/orc_file_dump.q @@ -1,3 +1,4 @@ +--! qt:replace:/(File Version:)(.+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.mapred.mode=nonstrict; diff --git a/ql/src/test/queries/clientpositive/orc_llap_counters.q b/ql/src/test/queries/clientpositive/orc_llap_counters.q index 912d4a93c845..130cd0167e72 100644 --- a/ql/src/test/queries/clientpositive/orc_llap_counters.q +++ b/ql/src/test/queries/clientpositive/orc_llap_counters.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.compute.query.using.stats=false; set hive.mapred.mode=nonstrict; diff --git a/ql/src/test/queries/clientpositive/orc_llap_counters1.q b/ql/src/test/queries/clientpositive/orc_llap_counters1.q index d7861fe95d1b..a3571abf2325 100644 --- a/ql/src/test/queries/clientpositive/orc_llap_counters1.q +++ b/ql/src/test/queries/clientpositive/orc_llap_counters1.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.mapred.mode=nonstrict; SET hive.optimize.index.filter=true; diff --git a/ql/src/test/queries/clientpositive/orc_merge10.q b/ql/src/test/queries/clientpositive/orc_merge10.q index 5ef66e7f0180..961457944b3f 100644 --- a/ql/src/test/queries/clientpositive/orc_merge10.q +++ b/ql/src/test/queries/clientpositive/orc_merge10.q @@ -1,6 +1,6 @@ --! qt:dataset:src --! qt:dataset:part - +--! qt:replace:/(File Version:)(.+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; set hive.compute.query.using.stats=false; set hive.mapred.mode=nonstrict; diff --git a/ql/src/test/queries/clientpositive/orc_merge11.q b/ql/src/test/queries/clientpositive/orc_merge11.q index 208c5b726c1f..b471475308cd 100644 --- a/ql/src/test/queries/clientpositive/orc_merge11.q +++ b/ql/src/test/queries/clientpositive/orc_merge11.q @@ -1,3 +1,4 @@ +--! qt:replace:/(File Version:)(.+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; DROP TABLE orcfile_merge1_n2; diff --git a/ql/src/test/queries/clientpositive/orc_merge12.q b/ql/src/test/queries/clientpositive/orc_merge12.q index a5534bdff02d..348c3a8f5e08 100644 --- a/ql/src/test/queries/clientpositive/orc_merge12.q +++ b/ql/src/test/queries/clientpositive/orc_merge12.q @@ -1,3 +1,4 @@ +--! qt:replace:/(File Version:)(.+)/$1#Masked#/ set hive.vectorized.execution.enabled=false; CREATE TABLE `alltypesorc3xcols`( diff --git a/ql/src/test/queries/clientpositive/row__id.q b/ql/src/test/queries/clientpositive/row__id.q index df2cb15f67cf..0c7a835792e9 100644 --- a/ql/src/test/queries/clientpositive/row__id.q +++ b/ql/src/test/queries/clientpositive/row__id.q @@ -1,3 +1,4 @@ +--! qt:replace:/(.+)(Data size: \d+)(.+)/$1#Masked#$3/ -- tid is flaky when compute column stats set hive.stats.column.autogather=false; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/ql/src/test/queries/clientpositive/smb_mapjoin_1.q index 7a98d2349e0a..1c7ee761d0cc 100644 --- a/ql/src/test/queries/clientpositive/smb_mapjoin_1.q +++ b/ql/src/test/queries/clientpositive/smb_mapjoin_1.q @@ -1,3 +1,4 @@ +--! qt:replace:/(totalSize\s+)(\S+|\s+|.+)/$1#Masked#/ set hive.strict.checks.bucketing=false; diff --git a/ql/src/test/queries/clientpositive/sqlmerge_stats.q b/ql/src/test/queries/clientpositive/sqlmerge_stats.q index 3913eeac57a8..fce1be55703b 100644 --- a/ql/src/test/queries/clientpositive/sqlmerge_stats.q +++ b/ql/src/test/queries/clientpositive/sqlmerge_stats.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ -- SORT_QUERY_RESULTS set hive.mapred.mode=nonstrict; diff --git a/ql/src/test/queries/clientpositive/stats_histogram.q b/ql/src/test/queries/clientpositive/stats_histogram.q index b98eaa349a4f..799a5f33840d 100644 --- a/ql/src/test/queries/clientpositive/stats_histogram.q +++ b/ql/src/test/queries/clientpositive/stats_histogram.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ set hive.stats.kll.enable=true; set metastore.stats.fetch.bitvector=true; set metastore.stats.fetch.kll=true; diff --git a/ql/src/test/queries/clientpositive/stats_histogram_null.q b/ql/src/test/queries/clientpositive/stats_histogram_null.q index 0d97395f9226..990d2a0e2cef 100644 --- a/ql/src/test/queries/clientpositive/stats_histogram_null.q +++ b/ql/src/test/queries/clientpositive/stats_histogram_null.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ set hive.stats.kll.enable=true; set metastore.stats.fetch.bitvector=true; set metastore.stats.fetch.kll=true; diff --git a/ql/src/test/queries/clientpositive/stats_part2.q b/ql/src/test/queries/clientpositive/stats_part2.q index 068e928ebeae..224f776c912a 100644 --- a/ql/src/test/queries/clientpositive/stats_part2.q +++ b/ql/src/test/queries/clientpositive/stats_part2.q @@ -1,3 +1,4 @@ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ set hive.stats.dbclass=fs; set hive.stats.fetch.column.stats=true; set datanucleus.cache.collections=false; diff --git a/ql/src/test/results/clientpositive/beeline/materialized_view_create_rewrite.q.out b/ql/src/test/results/clientpositive/beeline/materialized_view_create_rewrite.q.out index 74a840041b4b..07babeb81fd5 100644 --- a/ql/src/test/results/clientpositive/beeline/materialized_view_create_rewrite.q.out +++ b/ql/src/test/results/clientpositive/beeline/materialized_view_create_rewrite.q.out @@ -63,7 +63,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 408 -totalSize 474 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create materialized view if not exists cmv_mat_view2_n4 as select a, c from cmv_basetable_n10 where a = 3 @@ -99,7 +99,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 232 -totalSize 340 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: explain select a, c from cmv_basetable_n10 where a = 3 diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out index 87e8006076b7..ea1f2603fd71 100644 --- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out +++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out @@ -60,7 +60,7 @@ POSTHOOK: Input: default@smb_bucket_1_n3 numRows 0 rawDataSize 0 serialization.format 1 - totalSize 208 + totalSize #Masked# #### A masked pattern was here #### # Detailed Table Information NULL NULL # Storage Information NULL NULL diff --git a/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out b/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out index f075093a6176..96327ed0400d 100644 --- a/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out +++ b/ql/src/test/results/clientpositive/llap/acid_bloom_filter_orc_file_dump.q.out @@ -78,7 +78,7 @@ PREHOOK: Input: default@bloomtest #### A masked pattern was here #### -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 1 Compression: ZLIB Compression size: 32768 @@ -195,7 +195,7 @@ ________________________________________________________________________________ -- END ORC FILE DUMP -- -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 1 Compression: ZLIB Compression size: 32768 diff --git a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out index 3279ff9658fd..9b3df1134d8a 100644 --- a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out +++ b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out @@ -336,7 +336,7 @@ Table Parameters: numPartitions 4 numRows 2003 rawDataSize 0 - totalSize 18100 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -446,7 +446,7 @@ Table Parameters: numPartitions 4 numRows 2003 rawDataSize 0 - totalSize 18100 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/llap/default_constraint.q.out b/ql/src/test/results/clientpositive/llap/default_constraint.q.out index 1783ca9660a5..d9ef1a994a64 100644 --- a/ql/src/test/results/clientpositive/llap/default_constraint.q.out +++ b/ql/src/test/results/clientpositive/llap/default_constraint.q.out @@ -1566,7 +1566,7 @@ Table Type: MANAGED_TABLE Table Parameters: bucketing_version 2 numFiles 1 - totalSize 1107 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -1740,7 +1740,7 @@ Table Parameters: bucketing_version 2 #### A masked pattern was here #### numFiles 2 - totalSize 2213 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -1818,7 +1818,7 @@ Table Parameters: bucketing_version 2 #### A masked pattern was here #### numFiles 2 - totalSize 2213 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -2003,7 +2003,7 @@ Table Parameters: bucketing_version 2 #### A masked pattern was here #### numFiles 3 - totalSize 3306 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -2084,7 +2084,7 @@ Table Parameters: bucketing_version 2 #### A masked pattern was here #### numFiles 3 - totalSize 3306 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -2162,7 +2162,7 @@ Table Parameters: bucketing_version 2 #### A masked pattern was here #### numFiles 3 - totalSize 3306 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -2772,7 +2772,7 @@ Table Type: MANAGED_TABLE Table Parameters: bucketing_version 2 numFiles 1 - totalSize 1107 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/llap/deleteAnalyze.q.out b/ql/src/test/results/clientpositive/llap/deleteAnalyze.q.out index 41f27e5b0181..3b8bc17eb8b7 100644 --- a/ql/src/test/results/clientpositive/llap/deleteAnalyze.q.out +++ b/ql/src/test/results/clientpositive/llap/deleteAnalyze.q.out @@ -54,7 +54,7 @@ Table Parameters: numFiles 1 numRows 2 rawDataSize 634 - totalSize 604 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -121,7 +121,7 @@ Table Parameters: numFiles 0 numRows 0 rawDataSize 0 - totalSize 0 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite.q.out index 2af2b9bc0078..46850147f80a 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite.q.out @@ -63,7 +63,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 408 -totalSize 474 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create materialized view if not exists cmv_mat_view2_n4 as select a, c from cmv_basetable_n10 where a = 3 @@ -99,7 +99,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 232 -totalSize 340 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: explain select a, c from cmv_basetable_n10 where a = 3 diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out index c7225b9c6dfe..ac0eb35cb900 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_4.q.out @@ -296,7 +296,7 @@ Table Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1554 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -540,7 +540,7 @@ Table Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1554 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -1099,7 +1099,7 @@ Table Parameters: numFiles 3 numRows 3 rawDataSize 0 - totalSize 2325 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -1415,7 +1415,7 @@ Table Parameters: numFiles 2 numRows 3 rawDataSize 0 - totalSize 1061 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -1725,7 +1725,7 @@ Table Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1059 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -2211,7 +2211,7 @@ Table Parameters: numFiles 3 numRows 3 rawDataSize 0 - totalSize 1828 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_by_text_multi_db.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_by_text_multi_db.q.out index b4fd488bc8cf..355644d424dd 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_by_text_multi_db.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_by_text_multi_db.q.out @@ -87,7 +87,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 408 -totalSize 474 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create materialized view if not exists cmv_mat_view2_n2 as select a, c from db1.cmv_basetable_n7 where a = 3 @@ -123,7 +123,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 232 -totalSize 340 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create database db3 PREHOOK: type: CREATEDATABASE diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_dummy.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_dummy.q.out index bf53cf8cbf47..900589df81c6 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_dummy.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_dummy.q.out @@ -63,7 +63,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 408 -totalSize 474 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create materialized view if not exists cmv_mat_view2 as select a, c from cmv_basetable_n0 where a = 3 @@ -99,7 +99,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 232 -totalSize 340 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: explain select a, c from cmv_basetable_n0 where a = 3 diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_multi_db.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_multi_db.q.out index b4fd488bc8cf..355644d424dd 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_multi_db.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_multi_db.q.out @@ -87,7 +87,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 408 -totalSize 474 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create materialized view if not exists cmv_mat_view2_n2 as select a, c from db1.cmv_basetable_n7 where a = 3 @@ -123,7 +123,7 @@ numFiles 1 numFilesErasureCoded 0 numRows 2 rawDataSize 232 -totalSize 340 +totalSize #Masked# #### A masked pattern was here #### PREHOOK: query: create database db3 PREHOOK: type: CREATEDATABASE diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out index e716aee2295b..889a80675ff5 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window.q.out @@ -290,7 +290,7 @@ Table Parameters: numRows 2 rawDataSize 232 rewriting.time.window 5min - totalSize 624 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -529,7 +529,7 @@ Table Parameters: numRows 2 rawDataSize 232 rewriting.time.window 5min - totalSize 624 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -873,7 +873,7 @@ Table Parameters: numRows 3 rawDataSize 348 rewriting.time.window 5min - totalSize 654 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window_2.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window_2.q.out index a7b836949e05..52a9775ec753 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window_2.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_time_window_2.q.out @@ -96,7 +96,7 @@ Table Parameters: numFiles 2 numRows 2 rawDataSize 232 - totalSize 624 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -160,7 +160,7 @@ Table Parameters: numFiles 2 numRows 2 rawDataSize 232 - totalSize 624 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -224,7 +224,7 @@ Table Parameters: numFiles 2 numRows 3 rawDataSize 348 - totalSize 654 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/orc_analyze.q.out b/ql/src/test/results/clientpositive/llap/orc_analyze.q.out index f76adadfe265..27c455273d29 100644 --- a/ql/src/test/results/clientpositive/llap/orc_analyze.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_analyze.q.out @@ -102,7 +102,7 @@ Table Parameters: numFiles 1 numRows 100 rawDataSize 53000 - totalSize 3238 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -150,7 +150,7 @@ Table Parameters: numFiles 1 numRows 100 rawDataSize 53000 - totalSize 3238 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -239,7 +239,7 @@ Table Parameters: numFiles 1 numRows 100 rawDataSize 52600 - totalSize 3238 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -350,7 +350,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22150 - totalSize 2138 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -391,7 +391,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22250 - totalSize 2150 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -444,7 +444,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22150 - totalSize 2138 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -485,7 +485,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22250 - totalSize 2150 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -584,7 +584,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 21950 - totalSize 2138 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -625,7 +625,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22050 - totalSize 2150 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -742,7 +742,7 @@ Partition Parameters: numFiles 4 numRows 50 rawDataSize 22155 - totalSize 5424 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -783,7 +783,7 @@ Partition Parameters: numFiles 4 numRows 50 rawDataSize 22243 - totalSize 5403 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -836,7 +836,7 @@ Partition Parameters: numFiles 4 numRows 50 rawDataSize 22155 - totalSize 5424 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -877,7 +877,7 @@ Partition Parameters: numFiles 4 numRows 50 rawDataSize 22243 - totalSize 5403 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -982,7 +982,7 @@ Partition Parameters: numFiles 4 numRows 50 rawDataSize 21955 - totalSize 5424 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -1023,7 +1023,7 @@ Partition Parameters: numFiles 4 numRows 50 rawDataSize 22043 - totalSize 5403 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -1134,7 +1134,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22150 - totalSize 2138 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -1187,7 +1187,7 @@ Partition Parameters: numFiles 1 numRows 50 rawDataSize 22150 - totalSize 2138 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/orc_file_dump.q.out b/ql/src/test/results/clientpositive/llap/orc_file_dump.q.out index f1122b11e713..103256301ed4 100644 --- a/ql/src/test/results/clientpositive/llap/orc_file_dump.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_file_dump.q.out @@ -93,7 +93,7 @@ PREHOOK: Input: default@orc_ppd_n0 #### A masked pattern was here #### -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 1049 Compression: ZLIB Compression size: 262144 @@ -291,7 +291,7 @@ PREHOOK: Input: default@orc_ppd_n0 #### A masked pattern was here #### -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 1049 Compression: ZLIB Compression size: 262144 @@ -501,7 +501,7 @@ PREHOOK: Input: default@orc_ppd_part@ds=2015/hr=10 #### A masked pattern was here #### -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 1049 Compression: ZLIB Compression size: 262144 diff --git a/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out b/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out index 79029107e957..7d28a27438fd 100644 --- a/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out @@ -239,7 +239,7 @@ Table Parameters: orc.bloom.filter.columns * orc.row.index.stride 1000 rawDataSize 1139514 - totalSize 64520 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out b/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out index 609fd3ebba23..890a801cc53b 100644 --- a/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out @@ -239,7 +239,7 @@ Table Parameters: orc.bloom.filter.columns * orc.row.index.stride 1000 rawDataSize 1139514 - totalSize 64520 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/orc_merge10.q.out b/ql/src/test/results/clientpositive/llap/orc_merge10.q.out index 33c9dc3cdefe..97550a144242 100644 --- a/ql/src/test/results/clientpositive/llap/orc_merge10.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_merge10.q.out @@ -750,7 +750,7 @@ PREHOOK: Input: default@orcfile_merge1@ds=1/part=0 PREHOOK: Output: hdfs://### HDFS PATH ### -- BEGIN ORC FILE DUMP -- Structure for hdfs://### HDFS PATH ### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 242 Compression: SNAPPY Compression size: 4096 @@ -801,7 +801,7 @@ PREHOOK: Input: default@orcfile_merge1c@ds=1/part=0 PREHOOK: Output: hdfs://### HDFS PATH ### -- BEGIN ORC FILE DUMP -- Structure for hdfs://### HDFS PATH ### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 242 Compression: SNAPPY Compression size: 4096 diff --git a/ql/src/test/results/clientpositive/llap/orc_merge11.q.out b/ql/src/test/results/clientpositive/llap/orc_merge11.q.out index dc8a78951e49..37b2b43504ea 100644 --- a/ql/src/test/results/clientpositive/llap/orc_merge11.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_merge11.q.out @@ -76,7 +76,7 @@ PREHOOK: Input: default@orcfile_merge1_n2 #### A masked pattern was here #### -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 50000 Compression: ZLIB Compression size: 4096 @@ -168,7 +168,7 @@ ________________________________________________________________________________ -- END ORC FILE DUMP -- -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 50000 Compression: ZLIB Compression size: 4096 @@ -281,7 +281,7 @@ PREHOOK: Input: default@orcfile_merge1_n2 #### A masked pattern was here #### -- BEGIN ORC FILE DUMP -- #### A masked pattern was here #### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 100000 Compression: ZLIB Compression size: 4096 diff --git a/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out b/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out index 96b9952adc63..0532bbf5de9e 100644 --- a/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out +++ b/ql/src/test/results/clientpositive/llap/sqlmerge_stats.q.out @@ -36,7 +36,7 @@ Table Parameters: numFiles 0 numRows 0 rawDataSize 0 - totalSize 0 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -91,7 +91,7 @@ Table Parameters: numFiles 1 numRows 1 rawDataSize 0 - totalSize 666 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -447,7 +447,7 @@ Table Parameters: numFiles 4 numRows 2 rawDataSize 0 - totalSize 2730 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -511,7 +511,7 @@ Table Parameters: numFiles 6 numRows 0 rawDataSize 0 - totalSize 4116 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -587,7 +587,7 @@ Table Parameters: numFiles 0 numRows 0 rawDataSize 0 - totalSize 0 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/llap/stats_histogram.q.out b/ql/src/test/results/clientpositive/llap/stats_histogram.q.out index 80a435c55002..47b626415685 100644 --- a/ql/src/test/results/clientpositive/llap/stats_histogram.q.out +++ b/ql/src/test/results/clientpositive/llap/stats_histogram.q.out @@ -260,7 +260,7 @@ Table Parameters: numFiles 15 numRows 15 rawDataSize 4287 - totalSize 11142 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/stats_histogram_null.q.out b/ql/src/test/results/clientpositive/llap/stats_histogram_null.q.out index 18ffdc88f6a6..e997a607aac8 100644 --- a/ql/src/test/results/clientpositive/llap/stats_histogram_null.q.out +++ b/ql/src/test/results/clientpositive/llap/stats_histogram_null.q.out @@ -335,7 +335,7 @@ Table Parameters: numFiles 20 numRows 20 rawDataSize 5332 - totalSize 14529 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/llap/stats_part2.q.out b/ql/src/test/results/clientpositive/llap/stats_part2.q.out index 3e6a0eddf1b3..d4e7e34293e6 100644 --- a/ql/src/test/results/clientpositive/llap/stats_part2.q.out +++ b/ql/src/test/results/clientpositive/llap/stats_part2.q.out @@ -209,7 +209,7 @@ Table Parameters: numPartitions 0 numRows 0 rawDataSize 0 - totalSize 0 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -279,7 +279,7 @@ Table Parameters: numPartitions 3 numRows 6 rawDataSize 0 - totalSize 4425 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -376,7 +376,7 @@ Table Parameters: numPartitions 3 numRows 8 rawDataSize 0 - totalSize 5899 + totalSize #Masked# transactional true transactional_properties default #### A masked pattern was here #### @@ -530,7 +530,7 @@ Partition Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1471 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -565,7 +565,7 @@ Partition Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1477 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -600,7 +600,7 @@ Partition Parameters: numFiles 4 numRows 4 rawDataSize 0 - totalSize 2951 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -736,7 +736,7 @@ Partition Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1471 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -771,7 +771,7 @@ Partition Parameters: numFiles 4 numRows 2 rawDataSize 0 - totalSize 2945 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -806,7 +806,7 @@ Partition Parameters: numFiles 4 numRows 4 rawDataSize 0 - totalSize 2951 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -872,7 +872,7 @@ Partition Parameters: numFiles 2 numRows 2 rawDataSize 0 - totalSize 1471 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -907,7 +907,7 @@ Partition Parameters: numFiles 4 numRows 2 rawDataSize 0 - totalSize 2945 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -942,7 +942,7 @@ Partition Parameters: numFiles 4 numRows 4 rawDataSize 0 - totalSize 2951 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -1010,7 +1010,7 @@ Partition Parameters: numFiles 3 numRows 1 rawDataSize 0 - totalSize 2182 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -1045,7 +1045,7 @@ Partition Parameters: numFiles 5 numRows 1 rawDataSize 0 - totalSize 3649 + totalSize #Masked# #### A masked pattern was here #### # Storage Information @@ -1080,7 +1080,7 @@ Partition Parameters: numFiles 4 numRows 4 rawDataSize 0 - totalSize 2951 + totalSize #Masked# #### A masked pattern was here #### # Storage Information diff --git a/ql/src/test/results/clientpositive/row__id.q.out b/ql/src/test/results/clientpositive/row__id.q.out index 23df3ce74641..40deeb53a0a7 100644 --- a/ql/src/test/results/clientpositive/row__id.q.out +++ b/ql/src/test/results/clientpositive/row__id.q.out @@ -74,25 +74,25 @@ STAGE PLANS: Map Operator Tree: TableScan alias: hello_acid - Statistics: Num rows: 3 Data size: 20144 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ROW__ID.writeid (type: bigint) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 20144 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 #Masked# Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: bigint) null sort order: z sort order: + - Statistics: Num rows: 3 Data size: 20144 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 #Masked# Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: bigint) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 20144 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 #Masked# Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 20144 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 #Masked# Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -148,17 +148,17 @@ STAGE PLANS: TableScan alias: hello_acid filterExpr: (ROW__ID.writeid = 3L) (type: boolean) - Statistics: Num rows: 3 Data size: 20144 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 #Masked# Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (ROW__ID.writeid = 3L) (type: boolean) - Statistics: Num rows: 1 Data size: 6714 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ROW__ID.writeid (type: bigint) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 6714 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 #Masked# Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 6714 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 #Masked# Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git a/ql/src/test/results/clientpositive/tez/orc_merge12.q.out b/ql/src/test/results/clientpositive/tez/orc_merge12.q.out index d7f278b466ba..73526e4e93e2 100644 --- a/ql/src/test/results/clientpositive/tez/orc_merge12.q.out +++ b/ql/src/test/results/clientpositive/tez/orc_merge12.q.out @@ -144,7 +144,7 @@ PREHOOK: Input: default@alltypesorc3xcols PREHOOK: Output: hdfs://### HDFS PATH ### -- BEGIN ORC FILE DUMP -- Structure for hdfs://### HDFS PATH ### -File Version: 0.12 with ORC_14 by ORC Java 1.8.3 +File Version:#Masked# Rows: 24576 Compression: ZLIB Compression size: 131072 diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml index 9b0286e24798..caba6ea7f230 100644 --- a/standalone-metastore/pom.xml +++ b/standalone-metastore/pom.xml @@ -89,7 +89,7 @@ 0.16.0 2.18.0 3.3.3 - 1.8.3 + 1.8.5 3.21.7 1.51.0 1.9.0 From 1c126d947448ffc9784a1465306e018ba183a014 Mon Sep 17 00:00:00 2001 From: SimhadriG Date: Thu, 5 Oct 2023 14:31:12 +0530 Subject: [PATCH 017/179] HIVE-27772: UNIX_TIMESTAMP should return NULL when date fields are out of bounds (Simhadri Govindappa reviewed by Stamatis Zampetakis) In the case of invalid dates, such as '2001-02-31' (day field exceeds valid range for the given month), the UNIX_TIMESTAMP function behaves unexpectedly. Instead of returning NULL (as it happens in other systems like Spark, MySQL, etc.), it provides a value corresponding to another valid date based on some resolution rules (e.g., Feb 28th or March 1st). The resolution rules and results depend on the underlying formatter implementation used by UNIX_TIMESTAMP. By default, the DATETIME formatter uses the SMART resolution style and the SIMPLE formatter the LENIENT. Both of these styles are able to resolve "invalid" bounds to valid dates. In order to prevent seemingly "invalid" dates to be parsed correctly we have to use the STRICT resolution style. However, we cannot simply switch the formatters to always use the STRICT resolution cause that would break existing applications relying on the existing resolution rules. To address the problem reported here and retain the previous behaviour we opted to make the resolution style configurable by adding a new property. The new property only affects the DATETIME formatter; the SIMPLE formatter is almost deprecated so we don't add new features to it. Close apache/hive#4777 --- .../org/apache/hadoop/hive/conf/HiveConf.java | 4 + .../hive/conf/TestHiveConfVarsValidate.java | 9 ++ .../udf/generic/InstantDateTimeFormatter.java | 6 +- .../hive/ql/udf/generic/InstantFormatter.java | 16 +- ...DFToUnixTimestampEvaluateStringString.java | 30 +++- ...UDFToUnixTimestampEvaluateStringString.csv | 140 +++++++++++------- 6 files changed, 135 insertions(+), 70 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 5e9c8425ddfe..290cd8a4efa1 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3860,6 +3860,10 @@ public static enum ConfVars { "is discouraged. It suffers from known bugs that are unlikely to be fixed in subsequent versions of the product." + "Furthermore, using SIMPLE formatter may lead to strange behavior, and unexpected results when combined " + "with SQL functions/operators that are using the new DATETIME formatter."), + HIVE_DATETIME_RESOLVER_STYLE("hive.datetime.formatter.resolver.style", "SMART", + new StringSet("SMART", "STRICT", "LENIENT"), + "The style used by the hive.datetime.formatter (only applicable to DATETIME) to resolve dates amd times." + + "The possible values are STRICT, SMART, and LENIENT and their behavior follows the java.time.format.ResolverStyle API."), // HiveServer2 specific configs HIVE_SERVER2_CLEAR_DANGLING_SCRATCH_DIR("hive.server2.clear.dangling.scratchdir", false, "Clear dangling scratch dir periodically in HS2"), diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfVarsValidate.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfVarsValidate.java index 7ac44588c08a..42736ddb3d6f 100644 --- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfVarsValidate.java +++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfVarsValidate.java @@ -26,6 +26,7 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_DATETIME_FORMATTER; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_DATETIME_RESOLVER_STYLE; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_EXPLAIN_NODE_VISIT_LIMIT; import static org.junit.Assert.assertEquals; @@ -62,6 +63,14 @@ public static Collection generateParameters() { list.add(new Object[] { HIVE_DATETIME_FORMATTER, "simple", null }); list.add(new Object[] { HIVE_DATETIME_FORMATTER, "dateTime", null }); list.add(new Object[] { HIVE_DATETIME_FORMATTER, "OTHER", "Invalid value.. expects one of [datetime, simple]" }); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "SMART", null}); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "STRICT", null}); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "LENIENT", null}); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "smart", null}); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "strict", null}); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "lenient", null}); + list.add(new Object[] { HIVE_DATETIME_RESOLVER_STYLE, "OTHER", "Invalid value.. expects one of [smart, strict, " + + "lenient]" }); return list; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantDateTimeFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantDateTimeFormatter.java index 67ca27e57730..d2f5e160d3cb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantDateTimeFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantDateTimeFormatter.java @@ -27,13 +27,15 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; +import java.time.format.ResolverStyle; import java.util.Objects; final class InstantDateTimeFormatter extends InstantFormatterCache { - InstantDateTimeFormatter(final ZoneId zoneId) { + InstantDateTimeFormatter(final ZoneId zoneId, ResolverStyle resolverStyle) { super(zoneId, - s -> new DateTimeFormatterBuilder().parseCaseInsensitive().appendPattern(s).toFormatter().withZone(zoneId)); + s -> new DateTimeFormatterBuilder().parseCaseInsensitive() + .appendPattern(s).toFormatter().withResolverStyle(resolverStyle).withZone(zoneId)); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantFormatter.java index 382a10089ddd..9828bef07b7f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/InstantFormatter.java @@ -22,6 +22,7 @@ import java.time.Instant; import java.time.ZoneId; +import java.time.format.ResolverStyle; /** * Formatter for parsing and printing {@link Instant} objects. @@ -47,7 +48,7 @@ enum Type { */ SIMPLE { @Override - InstantFormatter newFormatter(ZoneId zone) { + InstantFormatter newFormatter(ZoneId zone, ResolverStyle resolverStyle) { return new InstantSimpleDateFormatter(zone); } }, @@ -56,16 +57,17 @@ InstantFormatter newFormatter(ZoneId zone) { */ DATETIME { @Override - InstantFormatter newFormatter(ZoneId zone) { - return new InstantDateTimeFormatter(zone); + InstantFormatter newFormatter(ZoneId zone, ResolverStyle resolverStyle) { + return new InstantDateTimeFormatter(zone, resolverStyle); } }; /** * Creates a new formatter with the specified zone id. - * @param zone - the zone id + * @param zone - the zone id + * @param resolverStyle - The style is used to control how the input is resolved. * @return a new formatter with the specified zone id. */ - abstract InstantFormatter newFormatter(ZoneId zone); + abstract InstantFormatter newFormatter(ZoneId zone, ResolverStyle resolverStyle); } /** @@ -77,7 +79,9 @@ InstantFormatter newFormatter(ZoneId zone) { static InstantFormatter ofConfiguration(Configuration conf) { ZoneId zoneId = TimestampTZUtil.parseTimeZone(HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_LOCAL_TIME_ZONE)); Type type = Type.valueOf(HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DATETIME_FORMATTER).toUpperCase()); - return type.newFormatter(zoneId); + ResolverStyle resolverStyle = ResolverStyle.valueOf(HiveConf.getVar(conf, + HiveConf.ConfVars.HIVE_DATETIME_RESOLVER_STYLE).toUpperCase()); + return type.newFormatter(zoneId, resolverStyle); } /** diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.java index e51b3910458f..bda6a171b161 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.java @@ -51,24 +51,27 @@ @RunWith(Parameterized.class) public class TestGenericUDFToUnixTimestampEvaluateStringString { private final GenericUDFToUnixTimeStamp udf = new GenericUDFToUnixTimeStamp(); + private final GenericUDFUnixTimeStamp udfUnixTimeStamp = new GenericUDFUnixTimeStamp(); private final ObjectInspector[] argInspectors = new ObjectInspector[2]; private final String value; private final String pattern; private final String zone; private final String formatter; + private final String resolverStyle; private final LongWritable expectedResult; public TestGenericUDFToUnixTimestampEvaluateStringString(String value, String pattern, String zone, String formatter, - String expectedResult) { + String resolverStyle, String expectedResult) { this.value = value; this.pattern = pattern; this.zone = zone; this.formatter = formatter; + this.resolverStyle = resolverStyle; this.expectedResult = expectedResult.equals("null") ? null : new LongWritable(Long.parseLong(expectedResult)); Arrays.fill(argInspectors, PrimitiveObjectInspectorFactory.writableStringObjectInspector); } - @Parameterized.Parameters(name = "('{0}','{1}'), zone={2}, parserLegacy={3}") + @Parameterized.Parameters(name = "('{0}','{1}'), zone={2}, parserLegacy={3}, resolverStyle={4}") public static Collection readInputs() throws IOException, CsvException { CSVParser parser = new CSVParserBuilder().withSeparator(';').withIgnoreQuotations(true).build(); try (CSVReader reader = new CSVReaderBuilder(new InputStreamReader( @@ -79,19 +82,30 @@ public static Collection readInputs() throws IOException, CsvException } @Test - public void testEvaluate() throws HiveException, InterruptedException { + public void testEvaluateToUnixTimeStamp() throws HiveException, InterruptedException { + testEvaluateWithUDF(udf); + } + + @Test + public void testEvaluateUnixTimeStamp() throws HiveException, InterruptedException { + testEvaluateWithUDF(udfUnixTimeStamp); + } + + private void testEvaluateWithUDF(GenericUDF udfToTest) throws HiveException, InterruptedException { HiveConf conf = new HiveConf(); conf.setVar(HiveConf.ConfVars.HIVE_DATETIME_FORMATTER, formatter); conf.setVar(HiveConf.ConfVars.HIVE_LOCAL_TIME_ZONE, zone); + conf.setVar(HiveConf.ConfVars.HIVE_DATETIME_RESOLVER_STYLE, resolverStyle); SessionState state = SessionState.start(conf); - udf.initialize(argInspectors); - LongWritable result = (LongWritable) udf.evaluate( + udfToTest.initialize(argInspectors); + LongWritable result = (LongWritable) udfToTest.evaluate( new DeferredObject[] { new DeferredJavaObject(new Text(value)), new DeferredJavaObject(new Text(pattern)) }); - assertEquals(udfDisplayWithInputs(), expectedResult, result); + assertEquals(udfDisplayWithInputs(udfToTest), expectedResult, result); SessionState.endStart(state); } - private String udfDisplayWithInputs() { - return udf.getDisplayString(new String[] { value, pattern }) + " sessionZone=" + zone + ", formatter=" + formatter; + private String udfDisplayWithInputs(GenericUDF udf) { + return udf.getDisplayString(new String[] { value, pattern }) + " sessionZone=" + zone + ", formatter=" + formatter + + ", resolver Style=" + resolverStyle; } } diff --git a/ql/src/test/resources/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.csv b/ql/src/test/resources/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.csv index ff4ee0725dfc..b3e34017245a 100644 --- a/ql/src/test/resources/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.csv +++ b/ql/src/test/resources/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestampEvaluateStringString.csv @@ -1,54 +1,86 @@ -1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;0 -1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;0 -1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Atlantic/Azores;DATETIME;3600 -1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Atlantic/Azores;SIMPLE;3600 -1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Europe/Paris;DATETIME;-3600 -1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Europe/Paris;SIMPLE;-3600 -1970-01-01 00:00:00 GMT;yyyy-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;0 -1970-01-01 00:00:00 GMT;yyyy-MM-dd HH:mm:ss z;Etc/GMT;SIMPLE;0 -1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;3600 -1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;SIMPLE;3600 -1970-01-01 00:00:00 GMT+01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;-3600 -1970-01-01 00:00:00 GMT+01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;SIMPLE;-3600 -1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Europe/Paris;DATETIME;3600 -1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Europe/Paris;SIMPLE;3600 -1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;-5364662400 -1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;-5364662400 -1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Asia/Kolkata;DATETIME;-5364683608 -1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Asia/Kolkata;SIMPLE;-5364682200 -Jul 9 2023;MMM dd yyyy;Etc/GMT;DATETIME;null -Jul 9 2023;MMM dd yyyy;Etc/GMT;SIMPLE;1688860800 -Jul 09 2023;MMM dd yyyy;Etc/GMT;DATETIME;1688860800 -Jul 09 2023;MMM dd yyyy;Etc/GMT;SIMPLE;1688860800 -Jul 21 2023;MMM dd yyyy;Etc/GMT;DATETIME;1689897600 -Jul 21 2023;MMM dd yyyy;Etc/GMT;SIMPLE;1689897600 -2023-07-21;YYYY-MM-DD;Etc/GMT;DATETIME;null -2023-07-21;YYYY-MM-DD;Etc/GMT;SIMPLE;1672531200 -Jul 21 2023 09:13;MMM dd yyyy HH:mm;Etc/GMT;DATETIME;1689930780 -Jul 21 2023 09:13;MMM dd yyyy HH:mm;Etc/GMT;SIMPLE;1689930780 -Jul 21 2023 9:13;MMM dd yyyy HH:mm;Etc/GMT;DATETIME;null -Jul 21 2023 9:13;MMM dd yyyy HH:mm;Etc/GMT;SIMPLE;1689930780 -2023-07-21 09:13;yyyy-MM-dd HH:mm;Etc/GMT;DATETIME;1689930780 -2023-07-21 09:13;yyyy-MM-dd HH:mm;Etc/GMT;SIMPLE;1689930780 -2023-07-21 9:13;yyyy-MM-dd HH:mm;Etc/GMT;DATETIME;null -2023-07-21 9:13;yyyy-MM-dd HH:mm;Etc/GMT;SIMPLE;1689930780 -2023-07-21 9:13PM;yyyy-MM-dd h:mma;Etc/GMT;DATETIME;1689973980 -2023-07-21 9:13PM;yyyy-MM-dd h:mma;Etc/GMT;SIMPLE;1689973980 -2023-07-21 09:13AM;yyyy-MM-dd HH:mmAA;Etc/GMT;DATETIME;null -2023-07-21 09:13AM;yyyy-MM-dd HH:mmAA;Etc/GMT;SIMPLE;null -2023-07-21 09:13AM;yyyy-MM-dd HH:mmaa;Etc/GMT;DATETIME;null -2023-07-21 09:13AM;yyyy-MM-dd HH:mmaa;Etc/GMT;SIMPLE;1689930780 -2023-07-21 09:13AM;yyyy-MM-dd HH:mma;Etc/GMT;DATETIME;1689930780 -2023-07-21 09:13AM;yyyy-MM-dd HH:mma;Etc/GMT;SIMPLE;1689930780 -2023-07-21 09:13PM;yyyy-MM-dd HH:mma;Etc/GMT;DATETIME;null -2023-07-21 09:13PM;yyyy-MM-dd HH:mma;Etc/GMT;SIMPLE;1689930780 -2023-07-21 09:13PM;yyyy-MM-dd hh:mmaa;Etc/GMT;DATETIME;null -2023-07-21 09:13PM;yyyy-MM-dd hh:mmaa;Etc/GMT;SIMPLE;1689973980 -2023-07-21 09:13PM;yyyy-MM-dd hh:mma;Etc/GMT;DATETIME;1689973980 -2023-07-21 09:13PM;yyyy-MM-dd hh:mma;Etc/GMT;SIMPLE;1689973980 -2023-07-21 09:13:10;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;1689930790 -2023-07-21 09:13:10;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;1689930790 -2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.sss;Etc/GMT;DATETIME;null -2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.sss;Etc/GMT;SIMPLE;1689930903 -2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.SSS;Etc/GMT;DATETIME;1689930790 -2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.SSS;Etc/GMT;DATETIME;1689930790 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;SMART;0 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;LENIENT;0 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Atlantic/Azores;DATETIME;SMART;3600 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Atlantic/Azores;SIMPLE;LENIENT;3600 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Europe/Paris;DATETIME;SMART;-3600 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Europe/Paris;SIMPLE;LENIENT;-3600 +1970-01-01 00:00:00 GMT;yyyy-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;SMART;0 +1970-01-01 00:00:00 GMT;yyyy-MM-dd HH:mm:ss z;Etc/GMT;SIMPLE;LENIENT;0 +1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;SMART;3600 +1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;SIMPLE;LENIENT;3600 +1970-01-01 00:00:00 GMT+01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;SMART;-3600 +1970-01-01 00:00:00 GMT+01:00;yyyy-MM-dd HH:mm:ss z;Etc/GMT;SIMPLE;LENIENT;-3600 +1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Europe/Paris;DATETIME;SMART;3600 +1970-01-01 00:00:00 GMT-01:00;yyyy-MM-dd HH:mm:ss z;Europe/Paris;SIMPLE;LENIENT;3600 +1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;SMART;-5364662400 +1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;LENIENT;-5364662400 +1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Asia/Kolkata;DATETIME;SMART;-5364683608 +1800-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Asia/Kolkata;SIMPLE;LENIENT;-5364682200 +Jul 9 2023;MMM dd yyyy;Etc/GMT;DATETIME;SMART;null +Jul 9 2023;MMM dd yyyy;Etc/GMT;SIMPLE;LENIENT;1688860800 +Jul 09 2023;MMM dd yyyy;Etc/GMT;DATETIME;SMART;1688860800 +Jul 09 2023;MMM dd yyyy;Etc/GMT;SIMPLE;LENIENT;1688860800 +Jul 21 2023;MMM dd yyyy;Etc/GMT;DATETIME;SMART;1689897600 +Jul 21 2023;MMM dd yyyy;Etc/GMT;SIMPLE;LENIENT;1689897600 +2023-07-21;YYYY-MM-DD;Etc/GMT;DATETIME;SMART;null +2023-07-21;YYYY-MM-DD;Etc/GMT;SIMPLE;LENIENT;1672531200 +Jul 21 2023 09:13;MMM dd yyyy HH:mm;Etc/GMT;DATETIME;SMART;1689930780 +Jul 21 2023 09:13;MMM dd yyyy HH:mm;Etc/GMT;SIMPLE;LENIENT;1689930780 +Jul 21 2023 9:13;MMM dd yyyy HH:mm;Etc/GMT;DATETIME;SMART;null +Jul 21 2023 9:13;MMM dd yyyy HH:mm;Etc/GMT;SIMPLE;LENIENT;1689930780 +2023-07-21 09:13;yyyy-MM-dd HH:mm;Etc/GMT;DATETIME;SMART;1689930780 +2023-07-21 09:13;yyyy-MM-dd HH:mm;Etc/GMT;SIMPLE;LENIENT;1689930780 +2023-07-21 9:13;yyyy-MM-dd HH:mm;Etc/GMT;DATETIME;SMART;null +2023-07-21 9:13;yyyy-MM-dd HH:mm;Etc/GMT;SIMPLE;LENIENT;1689930780 +2023-07-21 9:13PM;yyyy-MM-dd h:mma;Etc/GMT;DATETIME;SMART;1689973980 +2023-07-21 9:13PM;yyyy-MM-dd h:mma;Etc/GMT;SIMPLE;LENIENT;1689973980 +2023-07-21 09:13AM;yyyy-MM-dd HH:mmAA;Etc/GMT;DATETIME;SMART;null +2023-07-21 09:13AM;yyyy-MM-dd HH:mmAA;Etc/GMT;SIMPLE;LENIENT;null +2023-07-21 09:13AM;yyyy-MM-dd HH:mmaa;Etc/GMT;DATETIME;SMART;null +2023-07-21 09:13AM;yyyy-MM-dd HH:mmaa;Etc/GMT;SIMPLE;LENIENT;1689930780 +2023-07-21 09:13AM;yyyy-MM-dd HH:mma;Etc/GMT;DATETIME;SMART;1689930780 +2023-07-21 09:13AM;yyyy-MM-dd HH:mma;Etc/GMT;SIMPLE;LENIENT;1689930780 +2023-07-21 09:13PM;yyyy-MM-dd HH:mma;Etc/GMT;DATETIME;SMART;null +2023-07-21 09:13PM;yyyy-MM-dd HH:mma;Etc/GMT;SIMPLE;LENIENT;1689930780 +2023-07-21 09:13PM;yyyy-MM-dd hh:mmaa;Etc/GMT;DATETIME;SMART;null +2023-07-21 09:13PM;yyyy-MM-dd hh:mmaa;Etc/GMT;SIMPLE;LENIENT;1689973980 +2023-07-21 09:13PM;yyyy-MM-dd hh:mma;Etc/GMT;DATETIME;SMART;1689973980 +2023-07-21 09:13PM;yyyy-MM-dd hh:mma;Etc/GMT;SIMPLE;LENIENT;1689973980 +2023-07-21 09:13:10;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;SMART;1689930790 +2023-07-21 09:13:10;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;LENIENT;1689930790 +2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.sss;Etc/GMT;DATETIME;SMART;null +2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.sss;Etc/GMT;SIMPLE;LENIENT;1689930903 +2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.SSS;Etc/GMT;DATETIME;SMART;1689930790 +2023-07-21 09:13:10.123;yyyy-MM-dd HH:mm:ss.SSS;Etc/GMT;DATETIME;SMART;1689930790 +1970-01-01 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;STRICT;null +1970-01-01 00:00:00;uuuu-MM-dd HH:mm:ss;Etc/GMT;DATETIME;STRICT;0 +1970-01-01 00:00:00;uuuu-MM-dd HH:mm:ss;Atlantic/Azores;DATETIME;STRICT;3600 +1970-01-01 00:00:00;uuuu-MM-dd HH:mm:ss;Europe/Paris;DATETIME;STRICT;-3600 +1970-01-01 00:00:00 GMT;uuuu-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;STRICT;0 +1970-01-01 00:00:00 GMT+01:00;uuuu-MM-dd HH:mm:ss z;Etc/GMT;DATETIME;STRICT;-3600 +1970-01-01 00:00:00 GMT-01:00;uuuu-MM-dd HH:mm:ss z;Europe/Paris;DATETIME;STRICT;3600 +1800-01-01 00:00:00;uuuu-MM-dd HH:mm:ss;Etc/GMT;DATETIME;STRICT;-5364662400 +Jul 9 2023;MMM dd yyyy;Etc/GMT;SIMPLE;LENIENT;1688860800 +Jul 9 2023;MMM dd yyyy;Etc/GMT;DATETIME;SMART;null +Jul 9 2023;MMM dd uuuu;Etc/GMT;DATETIME;STRICT;null +Jul 09 2023;MMM dd uuuu;Etc/GMT;DATETIME;STRICT;1688860800 +Jul 21 2023;MMM dd uuuu;Etc/GMT;DATETIME;STRICT;1689897600 +2001-02-28 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;LENIENT;983318400 +2001-02-28 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;DATETIME;SMART;983318400 +2001-02-28 00:00:00;uuuu-MM-dd HH:mm:ss;Etc/GMT;DATETIME;STRICT;983318400 +2001-02-29;yyyy-MM-dd;Etc/GMT;SIMPLE;LENIENT;983404800 +2001-02-29;yyyy-MM-dd;Etc/GMT;DATETIME;LENIENT;983404800 +2001-02-29;yyyy-MM-dd;Etc/GMT;DATETIME;SMART;983318400 +2001-02-29;uuuu-MM-dd;Etc/GMT;DATETIME;STRICT;null +2001-02-31;yyyy-MM-dd;Etc/GMT;SIMPLE;LENIENT;983577600 +2001-02-31;yyyy-MM-dd;Etc/GMT;DATETIME;LENIENT;983577600 +2001-02-31;yyyy-MM-dd;Etc/GMT;DATETIME;SMART;983318400 +2001-02-31;uuuu-MM-dd;Etc/GMT;DATETIME;STRICT;null +Apr 31 2001;MMM dd yyyy;Etc/GMT;SIMPLE;LENIENT;988675200 +Apr 31 2001;MMM dd uuuu;Etc/GMT;DATETIME;LENIENT;988675200 +Apr 31 2001;MMM dd uuuu;Etc/GMT;DATETIME;SMART;988588800 +Apr 31 2001;MMM dd uuuu;Etc/GMT;DATETIME;STRICT;null +2001-06-31 00:00:00;yyyy-MM-dd HH:mm:ss;Etc/GMT;SIMPLE;LENIENT;993945600 +2001-06-31 00:00:00;uuuu-MM-dd HH:mm:ss;Etc/GMT;DATETIME;LENIENT;993945600 +2001-06-31 00:00:00;uuuu-MM-dd HH:mm:ss;Etc/GMT;DATETIME;SMART;993859200 +2001-06-31 00:00:00;uuuu-MM-dd HH:mm:ss;Etc/GMT;DATETIME;STRICT;null From 9eeab40173479c74b6fbf6657c3472b81ce4efcd Mon Sep 17 00:00:00 2001 From: Raghav Aggarwal Date: Mon, 16 Oct 2023 21:03:51 +0530 Subject: [PATCH 018/179] HIVE-27169: New Locked List to prevent configuration change at runtime without throwing error (#4731) (Raghav Aggarwal, reviewed by Okumin, Pravin Kumar Sinha) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 33 ++++++++++++++++ .../apache/hadoop/hive/conf/HiveConfUtil.java | 16 ++++++++ .../apache/hadoop/hive/conf/TestHiveConf.java | 15 +++++++ .../hive/ql/processors/SetProcessor.java | 5 +++ .../hadoop/hive/ql/session/SessionState.java | 39 +++++++++++++++++++ 5 files changed, 108 insertions(+) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 290cd8a4efa1..adc6503debeb 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -100,6 +100,7 @@ public class HiveConf extends Configuration { private static final Map metaConfs = new HashMap(); private final List restrictList = new ArrayList(); private final Set hiddenSet = new HashSet(); + private final Set lockedSet = new HashSet<>(); private final List rscList = new ArrayList<>(); private Pattern modWhiteListPattern = null; @@ -850,6 +851,10 @@ public static enum ConfVars { HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"), + HIVE_CONF_LOCKED_LIST("hive.conf.locked.list", "", "Comma separated " + + "list of configuration options which are locked and can not be changed at runtime. Warning is logged and the " + + "change is ignored when user try to set these configs during runtime"), + HIVE_FILE_MAX_FOOTER("hive.file.max.footer", 100, "maximum number of lines for footer user can define for a table file"), @@ -6010,6 +6015,9 @@ public void verifyAndSet(String name, String value) throws IllegalArgumentExcept throw new IllegalArgumentException("Cannot modify " + name + " at runtime. It is in the list" + " of parameters that can't be modified at runtime or is prefixed by a restricted variable"); } + if (isLockedConfig(name)) { + return; + } String oldValue = name != null ? get(name) : null; if (name == null || value == null || !value.equals(oldValue)) { // When either name or value is null, the set method below will fail, @@ -6022,6 +6030,10 @@ public boolean isHiddenConfig(String name) { return Iterables.any(hiddenSet, hiddenVar -> name.startsWith(hiddenVar)); } + public boolean isLockedConfig(String name) { + return Iterables.any(lockedSet, lockedVar -> name != null && name.equalsIgnoreCase(lockedVar)); + } + public static boolean isEncodedPar(String name) { for (ConfVars confVar : HiveConf.ENCODED_CONF) { ConfVars confVar1 = confVar; @@ -6427,6 +6439,7 @@ public HiveConf(HiveConf other) { origProp = (Properties)other.origProp.clone(); restrictList.addAll(other.restrictList); hiddenSet.addAll(other.hiddenSet); + lockedSet.addAll(other.lockedSet); modWhiteListPattern = other.modWhiteListPattern; } @@ -6560,6 +6573,9 @@ private void initialize(Class cls) { setupRestrictList(); hiddenSet.clear(); hiddenSet.addAll(HiveConfUtil.getHiddenSet(this)); + + lockedSet.clear(); + lockedSet.addAll(HiveConfUtil.getLockedSet(this)); } /** @@ -6938,6 +6954,22 @@ public void addToRestrictList(String restrictListStr) { setupRestrictList(); } + public void addToLockedSet(String lockedListStr) { + String oldList = this.getVar(ConfVars.HIVE_CONF_LOCKED_LIST); + if (oldList == null || oldList.isEmpty()) { + this.setVar(ConfVars.HIVE_CONF_LOCKED_LIST, lockedListStr); + } else { + this.setVar(ConfVars.HIVE_CONF_LOCKED_LIST, oldList + "," + lockedListStr); + } + String modifiedLockedSet = this.getVar(ConfVars.HIVE_CONF_LOCKED_LIST); + lockedSet.clear(); + if (modifiedLockedSet != null) { + for (String entry : modifiedLockedSet.split(",")) { + lockedSet.add(entry.trim()); + } + } + } + /** * Set white list of parameters that are allowed to be modified * @@ -6975,6 +7007,7 @@ private void setupRestrictList() { restrictList.add(ConfVars.HIVE_CONF_RESTRICTED_LIST.varname); restrictList.add(ConfVars.HIVE_CONF_HIDDEN_LIST.varname); restrictList.add(ConfVars.HIVE_CONF_INTERNAL_VARIABLE_LIST.varname); + restrictList.add(ConfVars.HIVE_CONF_LOCKED_LIST.varname); } /** diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java index 179ee83b1093..d48c884ae7c7 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConfUtil.java @@ -96,6 +96,22 @@ public static Set getHiddenSet(Configuration configuration) { return hiddenSet; } + /** + * Getting the set of locked configurations + * @param configuration The original configuration + * @return The list of the configuration values to be locked + */ + public static Set getLockedSet(Configuration configuration) { + Set lockedSet = new HashSet<>(); + String lockedListStr = HiveConf.getVar(configuration, ConfVars.HIVE_CONF_LOCKED_LIST); + if (lockedListStr != null) { + for (String entry : lockedListStr.split(",")) { + lockedSet.add(entry.trim()); + } + } + return lockedSet; + } + /** * Strips hidden config entries from configuration * @param conf The configuration to strip from diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java index decba6dbea0a..bff79a98faa5 100644 --- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java +++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java @@ -176,6 +176,21 @@ public void testHiddenConfig() throws Exception { } } + @Test + public void testLockedConfig() throws Exception { + HiveConf conf = new HiveConf(); + // Set the default value of the config + conf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "mr"); + String defaultVal = conf.get(ConfVars.HIVE_EXECUTION_ENGINE.varname); + // Update the lockedSet variable + conf.addToLockedSet(ConfVars.HIVE_EXECUTION_ENGINE.varname); + // Update the value of sample/test config + conf.verifyAndSet(ConfVars.HIVE_EXECUTION_ENGINE.varname, "tez"); + String modifiedVal = conf.get(ConfVars.HIVE_EXECUTION_ENGINE.varname); + // Check if the value is changed. + Assert.assertEquals(defaultVal, modifiedVal); + } + @Test public void testEncodingDecoding() throws UnsupportedEncodingException { HiveConf conf = new HiveConf(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java index 9c89c3bd3ae2..fbaabe06f258 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java @@ -52,6 +52,7 @@ */ public class SetProcessor implements CommandProcessor { private static final Logger LOG = LoggerFactory.getLogger(SetProcessor.class); + private static final SessionState.LogHelper console = SessionState.getConsole(); private static final String prefix = "set: "; private static final Set removedConfigs = @@ -255,6 +256,10 @@ public Map getHiveVariable() { } } conf.verifyAndSet(key, value); + if (conf.isLockedConfig(key)) { + console.printWarn("Cannot modify " + key + " at runtime. " + + "It is in the list of locked configurations that can't be modified at runtime"); + } if (HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname.equals(key)) { if ("mr".equals(value)) { result = HiveConf.generateMrDeprecationWarning(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 9614d95e395b..cb7ed48a58a5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -1321,6 +1321,45 @@ public void printInfo(String info, String detail, boolean isSilent) { LOG.info(info + StringUtils.defaultString(detail)); } + /** + * Logs warn into the log file, and if the LogHelper is not silent then into the HiveServer2 or + * HiveCli info stream too. + * BeeLine uses the operation log file to show the logs to the user, so depending on the + * BeeLine settings it could be shown to the user. + * @param warn The log message + */ + public void printWarn(String warn) { + printWarn(warn, null); + } + + /** + * Logs warn into the log file, and if the LogHelper is not silent then into the HiveServer2 or + * HiveCli info stream too. Handles an extra detail which will not be printed if null. + * BeeLine uses the operation log file to show the logs to the user, so depending on the + * BeeLine settings it could be shown to the user. + * @param warn The log message + * @param detail Extra detail to log which will be not printed if null + */ + public void printWarn(String warn, String detail) { + printWarn(warn, detail, getIsSilent()); + } + + /** + * Logs warn into the log file, and if not silent then into the HiveServer2 or HiveCli info + * stream too. Handles an extra detail which will not be printed if null. + * BeeLine uses the operation log file to show the logs to the user, so depending on the + * BeeLine settings it could be shown to the user. + * @param warn The log message + * @param detail Extra detail to log which will be not printed if null + * @param isSilent If true then the message will not be printed to the info stream + */ + public void printWarn(String warn, String detail, boolean isSilent) { + if (!isSilent) { + getInfoStream().println(warn); + } + LOG.warn(warn + StringUtils.defaultString(detail)); + } + /** * Logs an error into the log file, and into the HiveServer2 or HiveCli error stream too. * BeeLine uses the operation log file to show the logs to the user, so depending on the From a0364474ab2bf9926b32d7df31948fd49871cc35 Mon Sep 17 00:00:00 2001 From: dengzh Date: Tue, 17 Oct 2023 09:29:09 +0800 Subject: [PATCH 019/179] HIVE-27682: AlterTableAlterPartitionOperation cannot change the type if the column has default partition (Zhihua Deng, reviewed by Sai Hemanth Gantasala) Closes #4684 --- .../AlterTableAlterPartitionOperation.java | 20 ++++++------ .../clientpositive/alter_partition_coltype.q | 4 +++ .../llap/alter_partition_coltype.q.out | 31 +++++++++++++++++++ 3 files changed, 45 insertions(+), 10 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java index 2046cbdb4320..0fd8785d1bc7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java @@ -20,9 +20,9 @@ import java.util.ArrayList; import java.util.List; -import java.util.Set; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.ddl.DDLOperation; import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; @@ -92,14 +91,15 @@ private void checkPartitionValues(Table tbl, int colIndex) throws HiveException Converter converter = ObjectInspectorConverters.getConverter( PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI); - Set partitions = context.getDb().getAllPartitionsOf(tbl); - for (Partition part : partitions) { - if (part.getName().equals(context.getConf().getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME))) { - continue; - } - + List partNames = context.getDb().getPartitionNames(tbl.getDbName(), + tbl.getTableName(), (short) -1); + for (String partName : partNames) { try { - String value = part.getValues().get(colIndex); + List values = Warehouse.getPartValuesFromPartName(partName); + String value = values.get(colIndex); + if (value.equals(context.getConf().getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME))) { + continue; + } Object convertedValue = converter.convert(value); if (convertedValue == null) { throw new HiveException(" Converting from " + TypeInfoFactory.stringTypeInfo + " to " + expectedType + @@ -107,7 +107,7 @@ private void checkPartitionValues(Table tbl, int colIndex) throws HiveException } } catch (Exception e) { throw new HiveException("Exception while converting " + TypeInfoFactory.stringTypeInfo + " to " + - expectedType + " for value : " + part.getValues().get(colIndex)); + expectedType + " for partition : " + partName + ", index: " + colIndex); } } } diff --git a/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/ql/src/test/queries/clientpositive/alter_partition_coltype.q index e4a7c0015ae9..058d73c92883 100644 --- a/ql/src/test/queries/clientpositive/alter_partition_coltype.q +++ b/ql/src/test/queries/clientpositive/alter_partition_coltype.q @@ -63,6 +63,7 @@ insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) selec insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select '1', '2', '1' from src where key=150 limit 5; insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select NULL, '1', '1' from src where key=150 limit 5; +insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select '2', '2', NULL; alter table pt.alterdynamic_part_table partition column (partcol1 int); @@ -71,5 +72,8 @@ explain extended select intcol from pt.alterdynamic_part_table where partcol1='1 explain extended select intcol from pt.alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__'); select intcol from pt.alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__'); +alter table pt.alterdynamic_part_table partition column (partcol2 int); +select intcol from pt.alterdynamic_part_table where (partcol1=2 and partcol2=1) or (partcol1=2 and isnull(partcol2)); + drop table pt.alterdynamic_part_table; drop database pt; diff --git a/ql/src/test/results/clientpositive/llap/alter_partition_coltype.q.out b/ql/src/test/results/clientpositive/llap/alter_partition_coltype.q.out index 01a660166946..e6c1340008af 100644 --- a/ql/src/test/results/clientpositive/llap/alter_partition_coltype.q.out +++ b/ql/src/test/results/clientpositive/llap/alter_partition_coltype.q.out @@ -453,6 +453,16 @@ POSTHOOK: Input: default@src POSTHOOK: Output: pt@alterdynamic_part_table POSTHOOK: Output: pt@alterdynamic_part_table@partcol1=1/partcol2=1 POSTHOOK: Lineage: alterdynamic_part_table PARTITION(partcol1=1,partcol2=1).intcol EXPRESSION [] +PREHOOK: query: insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select '2', '2', NULL +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: pt@alterdynamic_part_table +POSTHOOK: query: insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select '2', '2', NULL +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: pt@alterdynamic_part_table +POSTHOOK: Output: pt@alterdynamic_part_table@partcol1=2/partcol2=__HIVE_DEFAULT_PARTITION__ +POSTHOOK: Lineage: alterdynamic_part_table PARTITION(partcol1=2,partcol2=__HIVE_DEFAULT_PARTITION__).intcol SIMPLE [] PREHOOK: query: alter table pt.alterdynamic_part_table partition column (partcol1 int) PREHOOK: type: ALTERTABLE_PARTCOLTYPE PREHOOK: Input: pt@alterdynamic_part_table @@ -603,6 +613,27 @@ POSTHOOK: Input: pt@alterdynamic_part_table POSTHOOK: Input: pt@alterdynamic_part_table@partcol1=2/partcol2=1 #### A masked pattern was here #### 1 +PREHOOK: query: alter table pt.alterdynamic_part_table partition column (partcol2 int) +PREHOOK: type: ALTERTABLE_PARTCOLTYPE +PREHOOK: Input: pt@alterdynamic_part_table +POSTHOOK: query: alter table pt.alterdynamic_part_table partition column (partcol2 int) +POSTHOOK: type: ALTERTABLE_PARTCOLTYPE +POSTHOOK: Input: pt@alterdynamic_part_table +POSTHOOK: Output: pt@alterdynamic_part_table +PREHOOK: query: select intcol from pt.alterdynamic_part_table where (partcol1=2 and partcol2=1) or (partcol1=2 and isnull(partcol2)) +PREHOOK: type: QUERY +PREHOOK: Input: pt@alterdynamic_part_table +PREHOOK: Input: pt@alterdynamic_part_table@partcol1=2/partcol2=1 +PREHOOK: Input: pt@alterdynamic_part_table@partcol1=2/partcol2=__HIVE_DEFAULT_PARTITION__ +#### A masked pattern was here #### +POSTHOOK: query: select intcol from pt.alterdynamic_part_table where (partcol1=2 and partcol2=1) or (partcol1=2 and isnull(partcol2)) +POSTHOOK: type: QUERY +POSTHOOK: Input: pt@alterdynamic_part_table +POSTHOOK: Input: pt@alterdynamic_part_table@partcol1=2/partcol2=1 +POSTHOOK: Input: pt@alterdynamic_part_table@partcol1=2/partcol2=__HIVE_DEFAULT_PARTITION__ +#### A masked pattern was here #### +1 +2 PREHOOK: query: drop table pt.alterdynamic_part_table PREHOOK: type: DROPTABLE PREHOOK: Input: pt@alterdynamic_part_table From b6847ed38b7d32586ab22e224904867f159b510e Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 17 Oct 2023 07:50:22 +0200 Subject: [PATCH 020/179] HIVE-27802: Simplify TestTezSessionState.testSymlinkedLocalFilesAreLocalizedOnce (#4804) (Laszlo Bodor reviewed by Ayush Saxena) --- .../hive/ql/exec/tez/TestTezSessionState.java | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezSessionState.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezSessionState.java index 521134bdfa5a..8e48c0f99987 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezSessionState.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezSessionState.java @@ -28,22 +28,6 @@ public class TestTezSessionState { - private class TestTezSessionPoolManager extends TezSessionPoolManager { - public TestTezSessionPoolManager() { - super(); - } - - @Override - public void setupPool(HiveConf conf) throws Exception { - super.setupPool(conf); - } - - @Override - public TezSessionPoolSession createSession(String sessionId, HiveConf conf) { - return new SampleTezSessionState(sessionId, this, conf); - } - } - @Test public void testSymlinkedLocalFilesAreLocalizedOnce() throws Exception { Path jarPath = Files.createTempFile("jar", ""); @@ -57,9 +41,8 @@ public void testSymlinkedLocalFilesAreLocalizedOnce() throws Exception { HiveConf hiveConf = new HiveConf(); hiveConf.set(HiveConf.ConfVars.HIVE_JAR_DIRECTORY.varname, "/tmp"); - TezSessionPoolManager poolManager = new TestTezSessionPoolManager(); - TezSessionState sessionState = poolManager.getSession(null, hiveConf, true, false); + TezSessionState sessionState = new TezSessionState(DagUtils.getInstance(), hiveConf); LocalResource l1 = sessionState.createJarLocalResource(jarPath.toUri().toString()); LocalResource l2 = sessionState.createJarLocalResource(symlinkPath.toUri().toString()); From f88d77ba0a050c25303c0c7100758b069c07a783 Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Wed, 18 Oct 2023 01:59:10 +0800 Subject: [PATCH 021/179] HIVE-27676: Reuse the add_partitions logic for add_partition in ObjetStore (#4678) (Wechar Yu, Reviewed by Sai Hemanth Gantasala) --- .../hadoop/hive/metastore/ObjectStore.java | 202 +++++++----------- .../hive/metastore/tools/BenchmarkTool.java | 4 +- .../hive/metastore/tools/HMSBenchmarks.java | 16 +- 3 files changed, 94 insertions(+), 128 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index ddacabe0fa5c..e7df10673b5a 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -21,6 +21,7 @@ import static org.apache.commons.lang3.StringUtils.join; import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.COMPACTOR_USE_CUSTOM_POOL; import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.newMetaException; import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier; import java.io.IOException; @@ -2660,88 +2661,93 @@ public boolean addPartitions(String catName, String dbName, String tblName, List boolean success = false; openTransaction(); try { - List tabGrants = null; - List tabColumnGrants = null; - MTable table = this.getMTable(catName, dbName, tblName); - if (table == null) { - throw new InvalidObjectException("Unable to add partitions because " - + TableName.getQualified(catName, dbName, tblName) + - " does not exist"); + addPartitionsInternal(catName, dbName, tblName, parts); + success = commitTransaction(); + } finally { + if (!success) { + rollbackTransaction(); } - if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) { - tabGrants = this.listAllTableGrants(catName, dbName, tblName); - tabColumnGrants = this.listTableAllColumnGrants(catName, dbName, tblName); + } + return success; + } + + private void addPartitionsInternal(String catName, String dbName, + String tblName, List parts) + throws MetaException, InvalidObjectException { + List tabGrants = null; + List tabColumnGrants = null; + MTable table = this.getMTable(catName, dbName, tblName); + if (table == null) { + throw new InvalidObjectException("Unable to add partitions because " + + TableName.getQualified(catName, dbName, tblName) + + " does not exist"); + } + if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) { + tabGrants = this.listAllTableGrants(catName, dbName, tblName); + tabColumnGrants = this.listTableAllColumnGrants(catName, dbName, tblName); + } + List mParts = new ArrayList<>(); + List> mPartPrivilegesList = new ArrayList<>(); + List> mPartColPrivilegesList = new ArrayList<>(); + for (Partition part : parts) { + if (!part.getTableName().equals(tblName) || !part.getDbName().equals(dbName)) { + throw new MetaException("Partition does not belong to target table " + + dbName + "." + tblName + ": " + part); } - List mParts = new ArrayList<>(); - List> mPartPrivilegesList = new ArrayList<>(); - List> mPartColPrivilegesList = new ArrayList<>(); - for (Partition part : parts) { - if (!part.getTableName().equals(tblName) || !part.getDbName().equals(dbName)) { - throw new MetaException("Partition does not belong to target table " - + dbName + "." + tblName + ": " + part); - } - MPartition mpart = convertToMPart(part, table, true); - mParts.add(mpart); - int now = (int) (System.currentTimeMillis() / 1000); - List mPartPrivileges = new ArrayList<>(); - if (tabGrants != null) { - for (MTablePrivilege tab: tabGrants) { - MPartitionPrivilege mPartPrivilege = new MPartitionPrivilege(tab.getPrincipalName(), tab.getPrincipalType(), - mpart, tab.getPrivilege(), now, tab.getGrantor(), tab.getGrantorType(), tab.getGrantOption(), - tab.getAuthorizer()); - mPartPrivileges.add(mPartPrivilege); - } + MPartition mpart = convertToMPart(part, table, true); + mParts.add(mpart); + int now = (int) (System.currentTimeMillis() / 1000); + List mPartPrivileges = new ArrayList<>(); + if (tabGrants != null) { + for (MTablePrivilege tab: tabGrants) { + MPartitionPrivilege mPartPrivilege = new MPartitionPrivilege(tab.getPrincipalName(), tab.getPrincipalType(), + mpart, tab.getPrivilege(), now, tab.getGrantor(), tab.getGrantorType(), tab.getGrantOption(), + tab.getAuthorizer()); + mPartPrivileges.add(mPartPrivilege); } + } - List mPartColumnPrivileges = new ArrayList<>(); - if (tabColumnGrants != null) { - for (MTableColumnPrivilege col : tabColumnGrants) { - MPartitionColumnPrivilege mPartColumnPrivilege = new MPartitionColumnPrivilege(col.getPrincipalName(), - col.getPrincipalType(), mpart, col.getColumnName(), col.getPrivilege(), now, col.getGrantor(), - col.getGrantorType(), col.getGrantOption(), col.getAuthorizer()); - mPartColumnPrivileges.add(mPartColumnPrivilege); - } + List mPartColumnPrivileges = new ArrayList<>(); + if (tabColumnGrants != null) { + for (MTableColumnPrivilege col : tabColumnGrants) { + MPartitionColumnPrivilege mPartColumnPrivilege = new MPartitionColumnPrivilege(col.getPrincipalName(), + col.getPrincipalType(), mpart, col.getColumnName(), col.getPrivilege(), now, col.getGrantor(), + col.getGrantorType(), col.getGrantOption(), col.getAuthorizer()); + mPartColumnPrivileges.add(mPartColumnPrivilege); } - mPartPrivilegesList.add(mPartPrivileges); - mPartColPrivilegesList.add(mPartColumnPrivileges); } - if (CollectionUtils.isNotEmpty(mParts)) { - GetHelper helper = new GetHelper(null, null, null, true, - true) { - @Override - protected Void getSqlResult(GetHelper ctx) throws MetaException { - directSql.addPartitions(mParts, mPartPrivilegesList, mPartColPrivilegesList); - return null; - } + mPartPrivilegesList.add(mPartPrivileges); + mPartColPrivilegesList.add(mPartColumnPrivileges); + } + if (CollectionUtils.isNotEmpty(mParts)) { + GetHelper helper = new GetHelper(null, null, null, true, true) { + @Override + protected Void getSqlResult(GetHelper ctx) throws MetaException { + directSql.addPartitions(mParts, mPartPrivilegesList, mPartColPrivilegesList); + return null; + } - @Override - protected Void getJdoResult(GetHelper ctx) { - List toPersist = new ArrayList<>(mParts); - mPartPrivilegesList.forEach(toPersist::addAll); - mPartColPrivilegesList.forEach(toPersist::addAll); - pm.makePersistentAll(toPersist); - pm.flush(); - return null; - } + @Override + protected Void getJdoResult(GetHelper ctx) { + List toPersist = new ArrayList<>(mParts); + mPartPrivilegesList.forEach(toPersist::addAll); + mPartColPrivilegesList.forEach(toPersist::addAll); + pm.makePersistentAll(toPersist); + pm.flush(); + return null; + } - @Override - protected String describeResult() { - return "add partitions"; - } - }; - try { - helper.run(false); - } catch (NoSuchObjectException e) { - throw new MetaException(e.getMessage()); + @Override + protected String describeResult() { + return "add partitions"; } - } - success = commitTransaction(); - } finally { - if (!success) { - rollbackTransaction(); + }; + try { + helper.run(false); + } catch (NoSuchObjectException e) { + throw newMetaException(e); } } - return success; } private boolean isValidPartition( @@ -2824,62 +2830,18 @@ public boolean addPartitions(String catName, String dbName, String tblName, @Override public boolean addPartition(Partition part) throws InvalidObjectException, MetaException { - boolean success = false; - boolean commited = false; - + boolean committed = false; try { openTransaction(); String catName = part.isSetCatName() ? part.getCatName() : getDefaultCatalog(conf); - MTable table = this.getMTable(catName, part.getDbName(), part.getTableName()); - if (table == null) { - throw new InvalidObjectException("Unable to add partition because " - + TableName.getQualified(catName, part.getDbName(), part.getTableName()) + - " does not exist"); - } - List tabGrants = null; - List tabColumnGrants = null; - if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) { - tabGrants = this.listAllTableGrants(catName, part.getDbName(), part.getTableName()); - tabColumnGrants = this.listTableAllColumnGrants( - catName, part.getDbName(), part.getTableName()); - } - MPartition mpart = convertToMPart(part, table, true); - pm.makePersistent(mpart); - - int now = (int) (System.currentTimeMillis() / 1000); - List toPersist = new ArrayList<>(); - if (tabGrants != null) { - for (MTablePrivilege tab: tabGrants) { - MPartitionPrivilege partGrant = new MPartitionPrivilege(tab - .getPrincipalName(), tab.getPrincipalType(), - mpart, tab.getPrivilege(), now, tab.getGrantor(), tab - .getGrantorType(), tab.getGrantOption(), tab.getAuthorizer()); - toPersist.add(partGrant); - } - } - - if (tabColumnGrants != null) { - for (MTableColumnPrivilege col : tabColumnGrants) { - MPartitionColumnPrivilege partColumn = new MPartitionColumnPrivilege(col - .getPrincipalName(), col.getPrincipalType(), mpart, col - .getColumnName(), col.getPrivilege(), now, col.getGrantor(), col - .getGrantorType(), col.getGrantOption(), col.getAuthorizer()); - toPersist.add(partColumn); - } - - if (CollectionUtils.isNotEmpty(toPersist)) { - pm.makePersistentAll(toPersist); - } - } - - commited = commitTransaction(); - success = true; + addPartitionsInternal(catName, part.getDbName(), part.getTableName(), Arrays.asList(part)); + committed = commitTransaction(); } finally { - if (!commited) { + if (!committed) { rollbackTransaction(); } } - return success; + return committed; } @Override diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java index 025b39339bc1..90672bf483db 100644 --- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java +++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java @@ -274,7 +274,7 @@ private void runNonAcidBenchmarks() { () -> benchmarkDeleteWithPartitions(bench, bData, 1, nParameters[0])) .add("dropTableMetadataWithPartitions", () -> benchmarkDeleteMetaOnlyWithPartitions(bench, bData, 1, nParameters[0])) - .add("addPartition", () -> benchmarkCreatePartition(bench, bData)) + .add("addPartition", () -> benchmarkCreatePartition(bench, bData, 1)) .add("dropPartition", () -> benchmarkDropPartition(bench, bData, 1)) .add("listPartition", () -> benchmarkListPartition(bench, bData)) .add("getPartition", @@ -315,6 +315,8 @@ private void runNonAcidBenchmarks() { () -> benchmarkRenameTable(bench, bData, howMany)) .add("dropDatabase" + '.' + howMany, () -> benchmarkDropDatabase(bench, bData, howMany)) + .add("addPartition" + '.' + howMany, + () -> benchmarkCreatePartition(bench, bData, howMany)) .add("dropPartition" + '.' + howMany, () -> benchmarkDropPartition(bench, bData, howMany)) .add("openTxns" + '.' + howMany, diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java index c48ab7d2b175..214e9e1cd6bb 100644 --- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java +++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java @@ -44,6 +44,7 @@ import static org.apache.hadoop.hive.metastore.tools.Util.addManyPartitions; import static org.apache.hadoop.hive.metastore.tools.Util.addManyPartitionsNoException; +import static org.apache.hadoop.hive.metastore.tools.Util.createManyPartitions; import static org.apache.hadoop.hive.metastore.tools.Util.createSchema; import static org.apache.hadoop.hive.metastore.tools.Util.throwingSupplierWrapper; @@ -181,22 +182,23 @@ static DescriptiveStatistics benchmarkListTables(@NotNull MicroBenchmark bench, } static DescriptiveStatistics benchmarkCreatePartition(@NotNull MicroBenchmark bench, - @NotNull BenchData data) { + @NotNull BenchData data, + int howMany) { final HMSClient client = data.getClient(); String dbName = data.dbName; String tableName = data.tableName; BenchmarkUtils.createPartitionedTable(client, dbName, tableName); - final List values = Collections.singletonList("d1"); try { Table t = client.getTable(dbName, tableName); - Partition partition = new Util.PartitionBuilder(t) - .withValues(values) - .build(); + List parts = createManyPartitions(t, null, Collections.singletonList("d"), howMany); return bench.measure(null, - () -> throwingSupplierWrapper(() -> client.addPartition(partition)), - () -> throwingSupplierWrapper(() -> client.dropPartition(dbName, tableName, values))); + () -> throwingSupplierWrapper(() -> { + parts.forEach(part -> throwingSupplierWrapper(() -> client.addPartition(part))); + return null; + }), + () -> throwingSupplierWrapper(() -> client.dropPartitions(dbName, tableName, null))); } catch (TException e) { e.printStackTrace(); return new DescriptiveStatistics(); From 07c5e18549b1f5e380b178816b7c6dec6bca322d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 18 Oct 2023 08:10:44 +0530 Subject: [PATCH 022/179] HIVE-27786: Iceberg: Eliminate engine.hive.enabled table property. (#4793). (Ayush Saxena, reviewed by Denys Kuzmenko) --- .../java/org/apache/iceberg/hive/HiveTableOperations.java | 3 +-- .../test/java/org/apache/iceberg/hive/HiveTableTest.java | 5 ++--- .../org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java | 6 ------ .../mr/hive/TestHiveIcebergStorageHandlerNoScan.java | 8 ++------ .../positive/alter_multi_part_table_to_iceberg.q.out | 3 --- .../results/positive/alter_part_table_to_iceberg.q.out | 3 --- .../test/results/positive/alter_table_to_iceberg.q.out | 3 --- .../src/test/results/positive/col_stats.q.out | 2 -- .../src/test/results/positive/create_iceberg_table.q.out | 1 - .../create_iceberg_table_stored_as_fileformat.q.out | 5 ----- .../positive/create_iceberg_table_stored_by_iceberg.q.out | 1 - ...erg_table_stored_by_iceberg_with_serdeproperties.q.out | 1 - .../results/positive/ctas_iceberg_partitioned_orc.q.out | 1 - .../src/test/results/positive/ctlt_iceberg.q.out | 4 ---- .../src/test/results/positive/delete_all_iceberg.q.out | 1 - .../test/results/positive/describe_iceberg_table.q.out | 4 ---- .../results/positive/iceberg_insert_into_partition.q.out | 6 ------ .../iceberg_insert_into_partition_transforms.q.out | 5 ----- .../iceberg_insert_into_partition_with_evolution.q.out | 1 - .../positive/iceberg_insert_overwrite_partition.q.out | 6 ------ .../iceberg_insert_overwrite_partition_transforms.q.out | 4 ---- .../src/test/results/positive/iceberg_v2_deletes.q.out | 3 --- .../results/positive/mv_iceberg_partitioned_orc.q.out | 2 -- .../results/positive/mv_iceberg_partitioned_orc2.q.out | 2 -- .../src/test/results/positive/row_count.q.out | 2 -- .../test/results/positive/show_create_iceberg_table.q.out | 5 ----- .../positive/show_iceberg_materialized_views.q.out | 4 ---- .../results/positive/truncate_force_iceberg_table.q.out | 2 -- .../test/results/positive/truncate_iceberg_table.q.out | 5 ----- .../positive/truncate_partitioned_iceberg_table.q.out | 2 -- .../results/positive/use_basic_stats_from_iceberg.q.out | 2 -- 31 files changed, 5 insertions(+), 97 deletions(-) diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 4b747ac7cfe8..c5adae435fb8 100644 --- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -552,8 +552,7 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c return metadata.propertyAsBoolean(TableProperties.ENGINE_HIVE_ENABLED, false); } - return conf.getBoolean( - ConfigProperties.ENGINE_HIVE_ENABLED, TableProperties.ENGINE_HIVE_ENABLED_DEFAULT); + return conf.getBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, true); } /** diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index e23d03effed1..64c6bc3d330b 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -517,7 +517,7 @@ public void testEngineHiveEnabledDefault() throws TException { catalog.dropTable(TABLE_IDENTIFIER); // Unset in hive-conf - catalog.getConf().unset(ConfigProperties.ENGINE_HIVE_ENABLED); + catalog.getConf().setBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, false); catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = metastoreClient.getTable(DB_NAME, TABLE_NAME); @@ -531,7 +531,6 @@ public void testEngineHiveEnabledConfig() throws TException { catalog.dropTable(TABLE_IDENTIFIER); // Enable by hive-conf - catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "true"); catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = metastoreClient.getTable(DB_NAME, TABLE_NAME); @@ -541,7 +540,7 @@ public void testEngineHiveEnabledConfig() throws TException { catalog.dropTable(TABLE_IDENTIFIER); // Disable by hive-conf - catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "false"); + catalog.getConf().setBoolean(ConfigProperties.ENGINE_HIVE_ENABLED, false); catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); hmsTable = metastoreClient.getTable(DB_NAME, TABLE_NAME); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 14ccd429ce19..ad711d16c40e 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -245,9 +245,6 @@ public void rollbackCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTa @Override public void commitCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) { if (icebergTable == null) { - if (Catalogs.hiveCatalog(conf, catalogProperties)) { - catalogProperties.put(TableProperties.ENGINE_HIVE_ENABLED, true); - } setFileFormat(catalogProperties.getProperty(TableProperties.DEFAULT_FILE_FORMAT)); @@ -544,9 +541,6 @@ public void commitAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable catalogProperties.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(preAlterTableProperties.schema)); catalogProperties.put(InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(preAlterTableProperties.spec)); setFileFormat(preAlterTableProperties.format); - if (Catalogs.hiveCatalog(conf, catalogProperties)) { - catalogProperties.put(TableProperties.ENGINE_HIVE_ENABLED, true); - } HiveTableUtil.importFiles(preAlterTableProperties.tableLocation, preAlterTableProperties.format, preAlterTableProperties.partitionSpecProxy, preAlterTableProperties.partitionKeys, catalogProperties, conf); } else if (currentAlterTableOp != null) { diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 225aefb92509..dc17e472b9c3 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -1000,19 +1000,15 @@ public void testIcebergAndHmsTableProperties() throws Exception { Properties tableProperties = new Properties(); tableProperties.putAll(hmsParams); - if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - expectedIcebergProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); - } if (HiveVersion.min(HiveVersion.HIVE_3)) { expectedIcebergProperties.put("bucketing_version", "2"); } Assert.assertEquals(expectedIcebergProperties, icebergTable.properties()); if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - Assert.assertEquals(13, hmsParams.size()); + Assert.assertEquals(12, hmsParams.size()); Assert.assertEquals("initial_val", hmsParams.get("custom_property")); Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); - Assert.assertEquals("true", hmsParams.get(TableProperties.ENGINE_HIVE_ENABLED)); Assert.assertEquals(HiveIcebergStorageHandler.class.getName(), hmsParams.get(hive_metastoreConstants.META_TABLE_STORAGE)); Assert.assertEquals(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), @@ -1046,7 +1042,7 @@ public void testIcebergAndHmsTableProperties() throws Exception { .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - Assert.assertEquals(16, hmsParams.size()); // 2 newly-added properties + previous_metadata_location prop + Assert.assertEquals(15, hmsParams.size()); // 2 newly-added properties + previous_metadata_location prop Assert.assertEquals("true", hmsParams.get("new_prop_1")); Assert.assertEquals("false", hmsParams.get("new_prop_2")); Assert.assertEquals("new_val", hmsParams.get("custom_property")); diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out index 449b4459c044..9e57537aaa87 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out @@ -200,7 +200,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} - engine.hive.enabled true iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### @@ -466,7 +465,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} - engine.hive.enabled true iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### @@ -732,7 +730,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} - engine.hive.enabled true iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out index 5f2da382900b..7bcde7ebb973 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out @@ -158,7 +158,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### @@ -373,7 +372,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### @@ -588,7 +586,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out index 41bcffb2ddb3..1c9578068c49 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out @@ -112,7 +112,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### @@ -278,7 +277,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### @@ -444,7 +442,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out b/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out index 94708408f3ca..dbf77956935e 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out @@ -513,7 +513,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### @@ -557,7 +556,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"int\"}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out index 4f2cbfcd4e7b..fb2eeff31c3b 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out @@ -29,7 +29,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out index 4928ce600907..9d53be2c0cb8 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out @@ -35,7 +35,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -103,7 +102,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -171,7 +169,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -239,7 +236,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -302,7 +298,6 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} dummy dummy_value - engine.hive.enabled true iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out index 4f2cbfcd4e7b..fb2eeff31c3b 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out @@ -29,7 +29,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out index 927651fef476..46de7cdd213a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out @@ -29,7 +29,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out index 5f2c545d3af8..607426a6890a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out @@ -300,7 +300,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"a_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000},{\"name\":\"b_trunc\",\"transform\":\"truncate[3]\",\"source-id\":2,\"field-id\":1001}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out index e853cfc6bd4c..e93f606a141c 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out @@ -41,7 +41,6 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"a","required":false,"type":"int"}]}', - 'engine.hive.enabled'='true', 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', @@ -126,7 +125,6 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', @@ -165,7 +163,6 @@ TBLPROPERTIES ( 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'snapshot-count'='0', @@ -232,7 +229,6 @@ TBLPROPERTIES ( 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'snapshot-count'='0', diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out index 0715f8a59776..40f2e0d4ddf0 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out @@ -115,7 +115,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"deleted-data-files\":\"5\",\"deleted-records\":\"20\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true format-version 2 iceberg.delete.skiprowdata false iceberg.orc.files.only true diff --git a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out index 3c5a6069c376..84dc9677e2c7 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out @@ -77,7 +77,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -134,7 +133,6 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"year_field\",\"required\":false,\"type\":\"date\"},{\"id\":2,\"name\":\"month_field\",\"required\":false,\"type\":\"date\"},{\"id\":3,\"name\":\"day_field\",\"required\":false,\"type\":\"date\"},{\"id\":4,\"name\":\"hour_field\",\"required\":false,\"type\":\"timestamp\"},{\"id\":5,\"name\":\"truncate_field\",\"required\":false,\"type\":\"string\"},{\"id\":6,\"name\":\"bucket_field\",\"required\":false,\"type\":\"int\"},{\"id\":7,\"name\":\"identity_field\",\"required\":false,\"type\":\"int\"}]} default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"year_field_year\",\"transform\":\"year\",\"source-id\":1,\"field-id\":1000},{\"name\":\"month_field_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1001},{\"name\":\"day_field_day\",\"transform\":\"day\",\"source-id\":3,\"field-id\":1002},{\"name\":\"hour_field_hour\",\"transform\":\"hour\",\"source-id\":4,\"field-id\":1003},{\"name\":\"truncate_field_trunc\",\"transform\":\"truncate[2]\",\"source-id\":5,\"field-id\":1004},{\"name\":\"bucket_field_bucket\",\"transform\":\"bucket[2]\",\"source-id\":6,\"field-id\":1005},{\"name\":\"identity_field\",\"transform\":\"identity\",\"source-id\":7,\"field-id\":1006}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -192,7 +190,6 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"year_field\",\"required\":false,\"type\":\"date\"},{\"id\":3,\"name\":\"month_field\",\"required\":false,\"type\":\"date\"},{\"id\":4,\"name\":\"day_field\",\"required\":false,\"type\":\"date\"},{\"id\":5,\"name\":\"hour_field\",\"required\":false,\"type\":\"timestamp\"},{\"id\":6,\"name\":\"truncate_field\",\"required\":false,\"type\":\"string\"},{\"id\":7,\"name\":\"bucket_field\",\"required\":false,\"type\":\"int\"},{\"id\":8,\"name\":\"identity_field\",\"required\":false,\"type\":\"int\"}]} default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"year_field_year\",\"transform\":\"year\",\"source-id\":2,\"field-id\":1000},{\"name\":\"month_field_month\",\"transform\":\"month\",\"source-id\":3,\"field-id\":1001},{\"name\":\"day_field_day\",\"transform\":\"day\",\"source-id\":4,\"field-id\":1002},{\"name\":\"hour_field_hour\",\"transform\":\"hour\",\"source-id\":5,\"field-id\":1003},{\"name\":\"truncate_field_trunc\",\"transform\":\"truncate[2]\",\"source-id\":6,\"field-id\":1004},{\"name\":\"bucket_field_bucket\",\"transform\":\"bucket[2]\",\"source-id\":7,\"field-id\":1005},{\"name\":\"identity_field\",\"transform\":\"identity\",\"source-id\":8,\"field-id\":1006}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 @@ -238,7 +235,6 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]} default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out index 3ed994a25a9a..de2ce3805c87 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out @@ -482,7 +482,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1620,7 +1619,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"192\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"384\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"country\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000},{\"name\":\"state\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1001}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2636,7 +2634,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"192\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"384\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"country\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000},{\"name\":\"state\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1001}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -3290,7 +3287,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -3944,7 +3940,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -4598,7 +4593,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out index d7403efe1907..35d37f3367dd 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out @@ -601,7 +601,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_year\",\"transform\":\"year\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1257,7 +1256,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1913,7 +1911,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_day\",\"transform\":\"day\",\"source-id\":1,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2340,7 +2337,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_trunc\",\"transform\":\"truncate[2]\",\"source-id\":1,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2751,7 +2747,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out index 10b06657329a..8f2e8ee412ca 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out @@ -177,7 +177,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"1\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"7\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":1,\"fields\":[{\"name\":\"b_trunc_2\",\"transform\":\"truncate[2]\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out index 277e92a65f65..31e7b1b42b56 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out @@ -276,7 +276,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1202,7 +1201,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"2\",\"deleted-data-files\":\"2\",\"added-records\":\"20\",\"deleted-records\":\"10\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"30\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"country\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000},{\"name\":\"state\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1001}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1654,7 +1652,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2094,7 +2091,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2534,7 +2530,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2974,7 +2969,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out index 0914b606b01c..91ff6028e34f 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out @@ -597,7 +597,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_year\",\"transform\":\"year\",\"source-id\":3,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1227,7 +1226,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -1861,7 +1859,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_day\",\"transform\":\"day\",\"source-id\":1,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# @@ -2288,7 +2285,6 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_trunc\",\"transform\":\"truncate[2]\",\"source-id\":1,\"field-id\":1000}]} - engine.hive.enabled true iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out index 324f77df0ba7..d21ccfe732b4 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out @@ -26,7 +26,6 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"}]}', - 'engine.hive.enabled'='true', 'format-version'='2', 'iceberg.delete.skiprowdata'='false', 'iceberg.orc.files.only'='true', @@ -137,7 +136,6 @@ TBLPROPERTIES ( 'current-snapshot-id'='#Masked#', 'current-snapshot-summary'='{"added-position-delete-files":"1","added-delete-files":"1","added-files-size":"#Masked#","added-position-deletes":"1","changed-partition-count":"1","total-records":"8","total-files-size":"#Masked#","total-data-files":"2","total-delete-files":"2","total-position-deletes":"3","total-equality-deletes":"0"}', 'current-snapshot-timestamp-ms'='#Masked#', - 'engine.hive.enabled'='true', 'format-version'='2', 'iceberg.delete.skiprowdata'='true', 'iceberg.orc.files.only'='true', @@ -284,7 +282,6 @@ TBLPROPERTIES ( 'current-snapshot-summary'='{"added-position-delete-files":"1","added-delete-files":"1","added-files-size":"#Masked#","added-position-deletes":"1","changed-partition-count":"1","total-records":"8","total-files-size":"#Masked#","total-data-files":"4","total-delete-files":"3","total-position-deletes":"3","total-equality-deletes":"0"}', 'current-snapshot-timestamp-ms'='#Masked#', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"part","transform":"identity","source-id":2,"field-id":1000}]}', - 'engine.hive.enabled'='true', 'format-version'='2', 'iceberg.delete.skiprowdata'='true', 'iceberg.orc.files.only'='true', diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out index 0db129f22aef..2a9eee7b0f07 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out @@ -68,7 +68,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true format-version 1 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### @@ -153,7 +152,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out index 2f194ac55ae4..a035a1cc962a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out @@ -69,7 +69,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000},{\"name\":\"c_trunc\",\"transform\":\"truncate[3]\",\"source-id\":2,\"field-id\":1001}]} - engine.hive.enabled true format-version 1 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### @@ -155,7 +154,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000},{\"name\":\"c_trunc\",\"transform\":\"truncate[3]\",\"source-id\":2,\"field-id\":1001}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out index b5d9cadc9853..481336ab61d0 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out @@ -105,7 +105,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"10\",\"added-records\":\"21\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"10\",\"total-records\":\"21\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"10\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"p1\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"p2\",\"transform\":\"identity\",\"source-id\":6,\"field-id\":1001}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### @@ -190,7 +189,6 @@ Table Parameters: current-snapshot-summary {\"added-position-delete-files\":\"5\",\"added-delete-files\":\"5\",\"added-files-size\":\"#Masked#\",\"added-position-deletes\":\"6\",\"changed-partition-count\":\"5\",\"total-records\":\"21\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"10\",\"total-delete-files\":\"9\",\"total-position-deletes\":\"10\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"p1\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"p2\",\"transform\":\"identity\",\"source-id\":6,\"field-id\":1001}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out index d1f1344eb616..c6e0cddcdef3 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out @@ -33,7 +33,6 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"i","required":false,"type":"int"},{"id":2,"name":"s","required":false,"type":"string"},{"id":3,"name":"ts","required":false,"type":"timestamp"},{"id":4,"name":"d","required":false,"type":"date"}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', @@ -88,7 +87,6 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"year_field","required":false,"type":"date"},{"id":2,"name":"month_field","required":false,"type":"date"},{"id":3,"name":"day_field","required":false,"type":"date"},{"id":4,"name":"hour_field","required":false,"type":"timestamp"},{"id":5,"name":"truncate_field","required":false,"type":"string"},{"id":6,"name":"bucket_field","required":false,"type":"int"},{"id":7,"name":"identity_field","required":false,"type":"int"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"year_field_year","transform":"year","source-id":1,"field-id":1000},{"name":"month_field_month","transform":"month","source-id":2,"field-id":1001},{"name":"day_field_day","transform":"day","source-id":3,"field-id":1002},{"name":"hour_field_hour","transform":"hour","source-id":4,"field-id":1003},{"name":"truncate_field_trunc","transform":"truncate[2]","source-id":5,"field-id":1004},{"name":"bucket_field_bucket","transform":"bucket[2]","source-id":6,"field-id":1005},{"name":"identity_field","transform":"identity","source-id":7,"field-id":1006}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', @@ -144,7 +142,6 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"year_field","required":false,"type":"date"},{"id":3,"name":"month_field","required":false,"type":"date"},{"id":4,"name":"day_field","required":false,"type":"date"},{"id":5,"name":"hour_field","required":false,"type":"timestamp"},{"id":6,"name":"truncate_field","required":false,"type":"string"},{"id":7,"name":"bucket_field","required":false,"type":"int"},{"id":8,"name":"identity_field","required":false,"type":"int"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"year_field_year","transform":"year","source-id":2,"field-id":1000},{"name":"month_field_month","transform":"month","source-id":3,"field-id":1001},{"name":"day_field_day","transform":"day","source-id":4,"field-id":1002},{"name":"hour_field_hour","transform":"hour","source-id":5,"field-id":1003},{"name":"truncate_field_trunc","transform":"truncate[2]","source-id":6,"field-id":1004},{"name":"bucket_field_bucket","transform":"bucket[2]","source-id":7,"field-id":1005},{"name":"identity_field","transform":"identity","source-id":8,"field-id":1006}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', @@ -188,7 +185,6 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"a","required":false,"type":"int"},{"id":2,"name":"b","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"b","transform":"identity","source-id":2,"field-id":1000}]}', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', @@ -240,7 +236,6 @@ TBLPROPERTIES ( 'current-snapshot-id'='#SnapshotId#', 'current-snapshot-summary'='{"added-data-files":"1","added-records":"3","added-files-size":"638","changed-partition-count":"1","total-records":"3","total-files-size":"638","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"}', 'current-snapshot-timestamp-ms'='#Masked#', - 'engine.hive.enabled'='true', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', 'previous_metadata_location'='hdfs://### HDFS PATH ###', diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out index 7efe8f4aa96f..cb796611a99d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out @@ -365,7 +365,6 @@ Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ds\":\"true\",\"key\":\"true\",\"value\":\"true\"}} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"key\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ds\",\"required\":false,\"type\":\"string\"}]} - engine.hive.enabled true format-version 1 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### @@ -417,7 +416,6 @@ Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ds\":\"true\",\"key\":\"true\",\"value\":\"true\"}} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"key\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ds\",\"required\":false,\"type\":\"string\"}]} - engine.hive.enabled true format-version 1 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### @@ -468,7 +466,6 @@ Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ds\":\"true\",\"key\":\"true\",\"value\":\"true\"}} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"key\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ds\",\"required\":false,\"type\":\"string\"}]} - engine.hive.enabled true format-version 1 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### @@ -573,7 +570,6 @@ Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\"}} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"}]} - engine.hive.enabled true format-version 1 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out index 7e21f4ae71a4..0dee689afc2e 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out @@ -94,7 +94,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"3\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"3\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge false iceberg.orc.files.only false #### A masked pattern was here #### @@ -165,7 +164,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"10\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge false iceberg.orc.files.only false #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out index 44f5aa07e4fe..95a97d8074f1 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out @@ -94,7 +94,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"3\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"3\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge true iceberg.orc.files.only true #### A masked pattern was here #### @@ -165,7 +164,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"10\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge true iceberg.orc.files.only true #### A masked pattern was here #### @@ -234,7 +232,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge true iceberg.orc.files.only true #### A masked pattern was here #### @@ -305,7 +302,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"deleted-data-files\":\"1\",\"deleted-records\":\"5\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge true iceberg.orc.files.only true #### A masked pattern was here #### @@ -392,7 +388,6 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"deleted-data-files\":\"1\",\"deleted-records\":\"5\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# - engine.hive.enabled true external.table.purge false iceberg.orc.files.only true #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out index 2a4257cacc0f..309cbee993ca 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out @@ -101,7 +101,6 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true external.table.purge true iceberg.orc.files.only false #### A masked pattern was here #### @@ -200,7 +199,6 @@ Table Parameters: current-snapshot-summary {\"deleted-data-files\":\"4\",\"deleted-records\":\"9\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} - engine.hive.enabled true external.table.purge true iceberg.orc.files.only false #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out index 0469f1b0c75b..257ed797965e 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out @@ -160,7 +160,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### @@ -204,7 +203,6 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"int\"}]} - engine.hive.enabled true format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### From ed98e1cd01c937e202666bb5e7fbd00e45088164 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Wed, 18 Oct 2023 16:13:43 +0300 Subject: [PATCH 023/179] HIVE-27783: Iceberg: Implement Copy-On-Write for Update queries (Denys Kuzmenko, reviewed by Krisztian Kasa, Butao Zhang) Closes #4781 --- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 2 + .../iceberg/mr/hive/HiveIcebergSerDe.java | 9 +- .../mr/hive/HiveIcebergStorageHandler.java | 30 +- .../iceberg/mr/hive/writer/WriterBuilder.java | 19 +- ...update_iceberg_copy_on_write_partitioned.q | 37 + ...date_iceberg_copy_on_write_unpartitioned.q | 37 + ...te_iceberg_copy_on_write_partitioned.q.out | 2184 +++++++++++++++++ ..._iceberg_copy_on_write_unpartitioned.q.out | 2132 ++++++++++++++++ .../apache/hadoop/hive/ql/io/AcidUtils.java | 9 + .../hive/ql/metadata/HiveStorageHandler.java | 5 +- .../hive/ql/parse/SemanticAnalyzer.java | 5 +- .../ql/parse/SplitUpdateSemanticAnalyzer.java | 6 + .../parse/UpdateDeleteSemanticAnalyzer.java | 40 +- 13 files changed, 4480 insertions(+), 35 deletions(-) create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out create mode 100644 iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out diff --git a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 4ba32734649a..5b1f68050c67 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -486,6 +486,8 @@ public enum ErrorMsg { NON_NATIVE_ACID_UPDATE(10435, "Update and Merge into non-native ACID table is only supported when " + HiveConf.ConfVars.SPLIT_UPDATE.varname + " is true."), READ_ONLY_DATABASE(10436, "Database {0} is read-only", true), + NON_NATIVE_ACID_COW_UPDATE(10437, "Update and Merge into non-native ACID table in copy-on-write mode is only supported when " + + HiveConf.ConfVars.SPLIT_UPDATE.varname + " is false."), //========================== 20000 range starts here ========================// diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java index 9acb26003a8f..694e7e9b605e 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java @@ -41,11 +41,13 @@ import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.hadoop.HadoopFileIO; @@ -163,7 +165,12 @@ private static Schema projectedSchema(Configuration configuration, String tableN case DELETE: return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); case UPDATE: - return IcebergAcidUtil.createSerdeSchemaForUpdate(tableSchema.columns()); + if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( + configuration.get(TableProperties.UPDATE_MODE))) { + return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); + } else { + return IcebergAcidUtil.createSerdeSchemaForUpdate(tableSchema.columns()); + } case OTHER: return tableSchema; default: diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 8214c9e68034..8c8b54256f43 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -712,7 +712,8 @@ private void addCustomSortExpr(Table table, org.apache.hadoop.hive.ql.metadata. fieldOrderMap.put(fields.get(i).name(), i); } - int offset = acidSelectColumns(hmsTable, writeOperation).size(); + int offset = (shouldOverwrite(hmsTable, writeOperation) ? + ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA : acidSelectColumns(hmsTable, writeOperation)).size(); for (TransformSpec spec : transformSpecs) { int order = fieldOrderMap.get(spec.getColumnName()); @@ -1048,8 +1049,7 @@ public AcidSupportType supportsAcidOperations(org.apache.hadoop.hive.ql.metadata // TODO: remove the checks as copy-on-write mode implementation for these DML ops get added private static void checkDMLOperationMode(org.apache.hadoop.hive.ql.metadata.Table table) { Map opTypes = ImmutableMap.of( - TableProperties.MERGE_MODE, TableProperties.MERGE_MODE_DEFAULT, - TableProperties.UPDATE_MODE, TableProperties.UPDATE_MODE_DEFAULT); + TableProperties.MERGE_MODE, TableProperties.MERGE_MODE_DEFAULT); for (Map.Entry opType : opTypes.entrySet()) { String mode = table.getParameters().get(opType.getKey()); @@ -1074,10 +1074,13 @@ public List acidVirtualColumns() { public List acidSelectColumns(org.apache.hadoop.hive.ql.metadata.Table table, Operation operation) { switch (operation) { case DELETE: - case UPDATE: // TODO: make it configurable whether we want to include the table columns in the select query. // It might make delete writes faster if we don't have to write out the row object return ListUtils.union(ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA, table.getCols()); + case UPDATE: + return shouldOverwrite(table, operation) ? + ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA : + ListUtils.union(ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA, table.getCols()); case MERGE: return ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA; default: @@ -1587,13 +1590,20 @@ public Map getNativeProperties(org.apache.hadoop.hive.ql.metadat } @Override - public boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, String operationName) { + public boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, Context.Operation operation) { String mode = null; - String formatVersion = mTable.getTTable().getParameters().get(TableProperties.FORMAT_VERSION); - // As of now only delete mode is supported, for all others return false - if ("2".equals(formatVersion) && operationName.equalsIgnoreCase(Context.Operation.DELETE.toString())) { - mode = mTable.getTTable().getParameters() - .getOrDefault(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_DEFAULT); + // As of now only update & delete modes are supported, for all others return false + if (IcebergTableUtil.isV2Table(mTable.getParameters())) { + switch (operation) { + case DELETE: + mode = mTable.getTTable().getParameters().getOrDefault(TableProperties.DELETE_MODE, + TableProperties.DELETE_MODE_DEFAULT); + break; + case UPDATE: + mode = mTable.getTTable().getParameters().getOrDefault(TableProperties.UPDATE_MODE, + TableProperties.UPDATE_MODE_DEFAULT); + break; + } } return COPY_ON_WRITE.equalsIgnoreCase(mode); } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java index 530309172d34..77f022d17109 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java @@ -121,13 +121,11 @@ public HiveIcebergWriter build() { new HiveFileWriterFactory(table, dataFileFormat, dataSchema, null, deleteFileFormat, null, null, null, skipRowData ? null : dataSchema); - boolean copyOnWriteMode = RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( - properties.get(TableProperties.DELETE_MODE)) && operation == Operation.DELETE; - HiveIcebergWriter writer; switch (operation) { case DELETE: - if (copyOnWriteMode) { + if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( + properties.get(TableProperties.DELETE_MODE))) { writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, io, targetFileSize); } else { @@ -135,6 +133,17 @@ public HiveIcebergWriter build() { io, targetFileSize, skipRowData); } break; + case UPDATE: + if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( + properties.get(TableProperties.UPDATE_MODE))) { + writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, + outputFileFactory, io, targetFileSize); + } else { + // Update and Merge should be splitted to inserts and deletes + throw new IllegalArgumentException("Unsupported operation when creating IcebergRecordWriter: " + + operation.name()); + } + break; case OTHER: writer = new HiveIcebergRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, io, targetFileSize); @@ -142,7 +151,7 @@ public HiveIcebergWriter build() { default: // Update and Merge should be splitted to inserts and deletes throw new IllegalArgumentException("Unsupported operation when creating IcebergRecordWriter: " + - operation.name()); + operation.name()); } WriterRegistry.registerWriter(attemptID, tableName, writer); diff --git a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q new file mode 100644 index 000000000000..f5355d5ca9a8 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q @@ -0,0 +1,37 @@ +set hive.split.update=false; +set hive.explain.user=false; + +drop table if exists tbl_ice; +create external table tbl_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.update.mode'='copy-on-write'); + +-- update using simple predicates +insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56); +explain update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22; + +update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22; +select * from tbl_ice order by a, b, c; + +-- update using subqueries referencing the same table +insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801); +explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); + +update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); +select * from tbl_ice order by a, b, c; + +-- update using a join subquery between the same table & another table +drop table if exists tbl_ice_other; +create external table tbl_ice_other(a int, b string) stored by iceberg; +insert into tbl_ice_other values (10, 'ten'), (333, 'hundred'); +explain update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); + +update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); +select * from tbl_ice order by a, b, c; + +-- update using a join subquery between the same table & a non-iceberg table +drop table if exists tbl_standard_other; +create external table tbl_standard_other(a int, b string) stored as orc; +insert into tbl_standard_other values (10, 'ten'), (444, 'tutu'); +explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); + +update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); +select * from tbl_ice order by a, b, c; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q new file mode 100644 index 000000000000..85b21f9c45fb --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q @@ -0,0 +1,37 @@ +set hive.split.update=false; +set hive.explain.user=false; + +drop table if exists tbl_ice; +create external table tbl_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.update.mode'='copy-on-write'); + +-- update using simple predicates +insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56); +explain update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22; + +update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22; +select * from tbl_ice order by a, b, c; + +-- update using subqueries referencing the same table +insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801); +explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); + +update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800); +select * from tbl_ice order by a, b, c; + +-- update using a join subquery between the same table & another table +drop table if exists tbl_ice_other; +create external table tbl_ice_other(a int, b string) stored by iceberg; +insert into tbl_ice_other values (10, 'ten'), (333, 'hundred'); +explain update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); + +update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a); +select * from tbl_ice order by a, b, c; + +-- update using a join subquery between the same table & a non-iceberg table +drop table if exists tbl_standard_other; +create external table tbl_standard_other(a int, b string) stored as orc; +insert into tbl_standard_other values (10, 'ten'), (444, 'tutu'); +explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); + +update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); +select * from tbl_ice order by a, b, c; diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out new file mode 100644 index 000000000000..985c32f342ea --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out @@ -0,0 +1,2184 @@ +PREHOOK: query: drop table if exists tbl_ice +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.update.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: create external table tbl_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.update.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: explain update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 1 <- Union 2 (CONTAINS) + Reducer 3 <- Union 2 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE), Union 2 (CONTAINS) + Reducer 7 <- Map 6 (SIMPLE_EDGE), Union 2 (CONTAINS) + Reducer 8 <- Map 6 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), 'Changed' (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Execution mode: vectorized + Map 4 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a <> 22) and (b <> 'one') and (b <> 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Execution mode: vectorized + Map 6 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint) + Filter Operator + predicate: (((b) IN ('one', 'four') or (a = 22)) and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 3 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 584 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 8 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col5 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col5: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col5 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Union 2 + Vertex: Union 2 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed 50 +2 two 51 +3 three 52 +4 Changed 53 +5 five 54 +111 Changed 55 +333 two 56 +PREHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[283][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[288][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 16' is a cross product +Warning: Shuffle Join MERGEJOIN[303][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product +Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product +PREHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 30 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) + Reducer 13 <- Reducer 12 (XPROD_EDGE), Reducer 25 (XPROD_EDGE), Union 7 (CONTAINS) + Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 32 (XPROD_EDGE) + Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) + Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 24 (XPROD_EDGE) + Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) + Reducer 18 <- Reducer 17 (SIMPLE_EDGE) + Reducer 19 <- Map 1 (XPROD_EDGE), Reducer 31 (XPROD_EDGE) + Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 28 (XPROD_EDGE) + Reducer 20 <- Reducer 19 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) + Reducer 21 <- Reducer 20 (XPROD_EDGE), Reducer 26 (XPROD_EDGE) + Reducer 22 <- Reducer 21 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 23 <- Map 1 (SIMPLE_EDGE) + Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 26 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 27 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 28 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 29 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) + Reducer 30 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 31 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 32 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Reducer 27 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) + Reducer 5 <- Reducer 23 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 8 <- Union 7 (SIMPLE_EDGE) + Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Select Operator + expressions: a (type: int), c (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + Select Operator + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Filter Operator + predicate: (c > 800) (type: boolean) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: c (type: int) + minReductionHashAggr: 0.7777778 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + aggregations: count(), count(c) + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Filter Operator + predicate: (a <= 5) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + keys: a (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(a) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 12 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 + Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) + Reducer 13 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 + Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 + Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 14 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) + Reducer 15 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1, _col2, _col3, _col5 + Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) + Reducer 16 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col5, _col6 + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) + Reducer 17 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col2, _col3, _col5, _col6, _col8 + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col3 <> 0L) and _col5 is not null) or ((_col6 <> 0L) and _col8 is not null)) (type: boolean) + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 18 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 19 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 1980 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1980 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 20 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2004 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2004 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col8 (type: boolean) + Reducer 21 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9 + Statistics: Num rows: 9 Data size: 2076 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 9 Data size: 2076 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) + Reducer 22 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col11 + Statistics: Num rows: 9 Data size: 2112 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col6 <> 0L) and _col8 is not null) or ((_col9 <> 0L) and _col11 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2112 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed again' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2781 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 23 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 24 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 25 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 26 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 27 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 28 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 29 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 30 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 31 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 32 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9 + Statistics: Num rows: 9 Data size: 2805 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 9 Data size: 2805 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col11 + Statistics: Num rows: 9 Data size: 2841 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col6 <> 0L) and _col8 is not null) or ((_col9 <> 0L) and _col11 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2841 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 9 Data size: 2841 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 4 Data size: 1172 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 8 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Union 7 + Vertex: Union 7 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[283][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[288][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 16' is a cross product +Warning: Shuffle Join MERGEJOIN[303][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product +Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product +PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed again 50 +2 Changed again 51 +3 Changed again 52 +4 Changed again 53 +5 Changed again 54 +111 Changed 55 +333 two 56 +444 hola 800 +555 Changed again 801 +PREHOOK: query: drop table if exists tbl_ice_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice_other +PREHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice_other +Warning: Shuffle Join MERGEJOIN[269][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Map 1 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE) + Reducer 12 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) + Reducer 13 <- Reducer 12 (SIMPLE_EDGE) + Reducer 14 <- Reducer 10 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE) + Reducer 3 <- Reducer 14 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) + Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Union 5 (SIMPLE_EDGE) + Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) + Reducer 8 <- Reducer 7 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 5 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Map 15 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 11 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 12 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 13 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 14 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 7 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 8 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Union 5 + Vertex: Union 5 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[269][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed again 50 +2 Changed again 51 +3 Changed again 52 +4 Changed again 53 +5 Changed again 54 +111 Changed 55 +333 Changed forever 56 +444 hola 800 +555 Changed again 801 +PREHOOK: query: drop table if exists tbl_standard_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_standard_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_standard_other +PREHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_standard_other +POSTHOOK: Lineage: tbl_standard_other.a SCRIPT [] +POSTHOOK: Lineage: tbl_standard_other.b SCRIPT [] +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 8' is a cross product +PREHOOK: query: explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 11 <- Map 15 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE) + Reducer 13 <- Map 15 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE) + Reducer 3 <- Map 15 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 5 <- Union 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 2 (SIMPLE_EDGE) + Reducer 7 <- Map 15 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Reducer 10 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 9 <- Reducer 12 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE), Union 4 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Map 15 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Reducer 10 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 13 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 14 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'The last one' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 308 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 5 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Union 4 + Vertex: Union 4 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 8' is a cross product +PREHOOK: query: update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed again 50 +2 Changed again 51 +3 Changed again 52 +4 Changed again 53 +5 Changed again 54 +111 Changed 55 +333 Changed forever 56 +444 The last one 800 +555 Changed again 801 diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out new file mode 100644 index 000000000000..da2c5e0292d1 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out @@ -0,0 +1,2132 @@ +PREHOOK: query: drop table if exists tbl_ice +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.update.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: create external table tbl_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.update.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55), (333, 'two', 56) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: explain update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 1 <- Union 2 (CONTAINS) + Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE), Union 2 (CONTAINS) + Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 2 (CONTAINS) + Reducer 7 <- Map 5 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), 'Changed' (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Execution mode: vectorized + Map 3 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a <> 22) and (b <> 'one') and (b <> 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Execution mode: vectorized + Map 5 + Map Operator Tree: + TableScan + alias: tbl_ice + filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint) + Filter Operator + predicate: (((b) IN ('one', 'four') or (a = 22)) and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: FILE__PATH (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: FILE__PATH (type: string) + Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 584 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col5 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col5: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col5 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Union 2 + Vertex: Union 2 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='Changed' where b in ('one', 'four') or a = 22 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed 50 +2 two 51 +3 three 52 +4 Changed 53 +5 five 54 +111 Changed 55 +333 two 56 +PREHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', 801) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[301][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 12' is a cross product +Warning: Shuffle Join MERGEJOIN[278][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 18' is a cross product +Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 20' is a cross product +PREHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 17 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (XPROD_EDGE), Reducer 24 (XPROD_EDGE), Union 7 (CONTAINS) + Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 31 (XPROD_EDGE) + Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) + Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 23 (XPROD_EDGE) + Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 17 <- Reducer 16 (SIMPLE_EDGE) + Reducer 18 <- Map 1 (XPROD_EDGE), Reducer 30 (XPROD_EDGE) + Reducer 19 <- Reducer 18 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 27 (XPROD_EDGE) + Reducer 20 <- Reducer 19 (XPROD_EDGE), Reducer 25 (XPROD_EDGE) + Reducer 21 <- Reducer 20 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 22 <- Map 1 (SIMPLE_EDGE) + Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 26 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 27 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 28 <- Map 1 (SIMPLE_EDGE) + Reducer 29 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) + Reducer 30 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 31 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Reducer 26 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) + Reducer 5 <- Reducer 22 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) + Reducer 9 <- Reducer 29 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Select Operator + expressions: a (type: int), c (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + Select Operator + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Filter Operator + predicate: (c > 800) (type: boolean) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: c (type: int) + minReductionHashAggr: 0.7777778 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + aggregations: count(), count(c) + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Filter Operator + predicate: (a <= 5) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Group By Operator + keys: a (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(a) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 + Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) + Reducer 12 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 + Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 + Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 13 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) + Reducer 14 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1, _col2, _col3, _col5 + Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) + Reducer 15 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col5, _col6 + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) + Reducer 16 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col2, _col3, _col5, _col6, _col8 + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col3 <> 0L) and _col5 is not null) or ((_col6 <> 0L) and _col8 is not null)) (type: boolean) + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 17 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 18 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 1980 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1980 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: bigint) + Reducer 19 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2004 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2004 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col8 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Reducer 20 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9 + Statistics: Num rows: 9 Data size: 2076 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 9 Data size: 2076 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) + Reducer 21 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col11 + Statistics: Num rows: 9 Data size: 2112 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col6 <> 0L) and _col8 is not null) or ((_col9 <> 0L) and _col11 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2112 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed again' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2781 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 22 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 23 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 24 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 25 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 26 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 27 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 28 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 29 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 30 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 31 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9 + Statistics: Num rows: 9 Data size: 2805 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 9 Data size: 2805 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col11 + Statistics: Num rows: 9 Data size: 2841 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((_col6 <> 0L) and _col8 is not null) or ((_col9 <> 0L) and _col11 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2841 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 9 Data size: 2841 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 4 Data size: 1172 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Union 7 + Vertex: Union 7 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[301][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 12' is a cross product +Warning: Shuffle Join MERGEJOIN[278][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 18' is a cross product +Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 20' is a cross product +PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed again 50 +2 Changed again 51 +3 Changed again 52 +4 Changed again 53 +5 Changed again 54 +111 Changed 55 +333 two 56 +444 hola 800 +555 Changed again 801 +PREHOOK: query: drop table if exists tbl_ice_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: create external table tbl_ice_other(a int, b string) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice_other +PREHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice_other +POSTHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice_other +Warning: Shuffle Join MERGEJOIN[267][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Reducer 9 (SIMPLE_EDGE) + Reducer 11 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE) + Reducer 13 <- Reducer 9 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) + Reducer 3 <- Reducer 13 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) + Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Map 14 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 10 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 11 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 13 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 9 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Union 5 + Vertex: Union 5 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[267][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_ice_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='Changed forever' where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_ice_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed again 50 +2 Changed again 51 +3 Changed again 52 +4 Changed again 53 +5 Changed again 54 +111 Changed 55 +333 Changed forever 56 +444 hola 800 +555 Changed again 801 +PREHOOK: query: drop table if exists tbl_standard_other +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_standard_other +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: create external table tbl_standard_other(a int, b string) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_standard_other +PREHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_standard_other +POSTHOOK: query: insert into tbl_standard_other values (10, 'ten'), (444, 'tutu') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_standard_other +POSTHOOK: Lineage: tbl_standard_other.a SCRIPT [] +POSTHOOK: Lineage: tbl_standard_other.b SCRIPT [] +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 7' is a cross product +PREHOOK: query: explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Map 14 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE) + Reducer 12 <- Map 14 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE) + Reducer 13 <- Reducer 12 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE) + Reducer 3 <- Map 14 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 5 <- Reducer 2 (SIMPLE_EDGE) + Reducer 6 <- Map 14 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 8 <- Reducer 11 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 9 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Map 14 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Select Operator + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), FILE__PATH (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 11 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 12 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Reducer 13 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col5 ASC NULLS FIRST + partition by: _col5 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'The last one' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 308 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 5 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 10 Data size: 3018 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 7 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8, _col9, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 10 Data size: 3010 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Union 4 + Vertex: Union 4 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 7' is a cross product +PREHOOK: query: update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Input: default@tbl_standard_other +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Input: default@tbl_standard_other +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Changed again 50 +2 Changed again 51 +3 Changed again 52 +4 Changed again 53 +5 Changed again 54 +111 Changed 55 +333 Changed forever 56 +444 The last one 800 +555 Changed again 801 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java index 772947c25d59..0a367c8f8377 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java @@ -3383,6 +3383,15 @@ public static boolean isNonNativeAcidTable(Table table, boolean isWriteOperation table.getStorageHandler().supportsAcidOperations(table, isWriteOperation) != HiveStorageHandler.AcidSupportType.NONE; } + public static boolean isCopyOnWriteMode(Table table, Context.Operation operation) { + boolean copyOnWriteMode = false; + HiveStorageHandler storageHandler = table.getStorageHandler(); + if (storageHandler != null) { + copyOnWriteMode = storageHandler.shouldOverwrite(table, operation); + } + return copyOnWriteMode; + } + /** * Returns the virtual columns needed for update queries. For ACID queries it is a single ROW__ID, for non-native * tables the list is provided by the {@link HiveStorageHandler#acidVirtualColumns()}. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 8037403f3acc..b52e9d36c794 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; import org.apache.hadoop.hive.ql.ErrorMsg; @@ -386,10 +387,10 @@ default Map getNativeProperties(org.apache.hadoop.hive.ql.metada /** * Returns whether the data should be overwritten for the specific operation. * @param mTable the table. - * @param operationName operationName of the operation. + * @param operation operation type. * @return if the data should be overwritten for the specified operation. */ - default boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, String operationName) { + default boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, Context.Operation operation) { return false; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 6315d74885c3..5be52904b5fd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -8893,9 +8893,10 @@ private Operator genConversionSelectOperator(String dest, QB qb, Operator input, // For Non-Native ACID tables we should convert the new values as well rowFieldsOffset = expressions.size(); - if (updating(dest) && AcidUtils.isNonNativeAcidTable(table, true)) { + if (updating(dest) && AcidUtils.isNonNativeAcidTable(table, true) + && rowFields.size() >= rowFieldsOffset + columnNumber) { for (int i = 0; i < columnNumber; i++) { - ExprNodeDesc column = handleConversion(tableFields.get(i), rowFields.get(rowFieldsOffset + i), converted, dest, i); + ExprNodeDesc column = handleConversion(tableFields.get(i), rowFields.get(rowFieldsOffset-columnNumber + i), converted, dest, i); expressions.add(column); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java index 7bf93b819aa9..d36b1129067a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java @@ -19,7 +19,9 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; @@ -72,6 +74,10 @@ protected ASTNode getTargetTableNode(ASTNode tree) { protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws SemanticException { switch (tree.getToken().getType()) { case HiveParser.TOK_UPDATE_TABLE: + boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); + if (nonNativeAcid && AcidUtils.isCopyOnWriteMode(table, Context.Operation.UPDATE)) { + throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_COW_UPDATE.getErrorCodedMsg()); + } analyzeUpdate(tree, table, tabNameNode); break; default: diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java index 86ded055e71c..3d8050217821 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.ParseUtils.ReparseResult; @@ -66,7 +65,7 @@ protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws Se break; case HiveParser.TOK_UPDATE_TABLE: boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); - if (nonNativeAcid) { + if (nonNativeAcid && !AcidUtils.isCopyOnWriteMode(table, Context.Operation.UPDATE)) { throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); } operation = Context.Operation.UPDATE; @@ -118,12 +117,6 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN updateOutputs(mTable); return; } - - boolean copyOnWriteMode = false; - HiveStorageHandler storageHandler = mTable.getStorageHandler(); - if (storageHandler != null) { - copyOnWriteMode = storageHandler.shouldOverwrite(mTable, operation.name()); - } StringBuilder rewrittenQueryStr = new StringBuilder(); rewrittenQueryStr.append("insert into table "); @@ -136,6 +129,7 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN columnAppender.appendAcidSelectColumns(rewrittenQueryStr, operation); rewrittenQueryStr.setLength(rewrittenQueryStr.length() - 1); + boolean copyOnWriteMode = AcidUtils.isCopyOnWriteMode(mTable, operation); Map setColExprs = null; Map setCols = null; // Must be deterministic order set for consistent q-test output across Java versions @@ -154,7 +148,11 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN rewrittenQueryStr.append(','); String name = nonPartCols.get(i).getName(); ASTNode setCol = setCols.get(name); - rewrittenQueryStr.append(HiveUtils.unparseIdentifier(name, this.conf)); + String identifier = HiveUtils.unparseIdentifier(name, this.conf); + rewrittenQueryStr.append(identifier); + if (copyOnWriteMode) { + rewrittenQueryStr.append(" AS ").append(identifier); + } if (setCol != null) { // This is one of the columns we're setting, record it's position so we can come back // later and patch it up. @@ -166,7 +164,7 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN rewrittenQueryStr.append(" from "); rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); - + ASTNode where = null; int whereIndex = deleting() ? 1 : 2; @@ -179,7 +177,15 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN String whereClause = ctx.getTokenRewriteStream().toString( where.getChild(0).getTokenStartIndex(), where.getChild(0).getTokenStopIndex()); String filePathCol = HiveUtils.unparseIdentifier("FILE__PATH", conf); - + + if (updating()) { + rewrittenQueryStr.append("\nunion all"); + rewrittenQueryStr.append("\nselect "); + columnAppender.appendAcidSelectColumns(rewrittenQueryStr, Context.Operation.DELETE); + rewrittenQueryStr.setLength(rewrittenQueryStr.length() - 1); + rewrittenQueryStr.append(" from "); + rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); + } // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. rewrittenQueryStr.append("\nwhere NOT (").append(whereClause).append(")"); rewrittenQueryStr.append("\n").append(INDENT); @@ -193,12 +199,12 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN withQueryStr.append("WITH t AS ("); withQueryStr.append("\n").append(INDENT); withQueryStr.append("select "); - columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, operation); + columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, Context.Operation.DELETE); withQueryStr.setLength(withQueryStr.length() - 1); withQueryStr.append(" from ("); withQueryStr.append("\n").append(INDENT).append(INDENT); withQueryStr.append("select "); - columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, operation); + columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, Context.Operation.DELETE); withQueryStr.append(" row_number() OVER (partition by ").append(filePathCol).append(") rn"); withQueryStr.append(" from ").append(getFullTableNameForSQL(tabNameNode)); withQueryStr.append("\n").append(INDENT).append(INDENT); @@ -220,7 +226,11 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN Context rewrittenCtx = rr.rewrittenCtx; ASTNode rewrittenTree = rr.rewrittenTree; - ASTNode rewrittenInsert = (ASTNode)rewrittenTree.getChildren().get(1); + ASTNode rewrittenInsert = (ASTNode) (copyOnWriteMode && updating() ? + new ASTSearcher().simpleBreadthFirstSearch(rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, + HiveParser.TOK_UNIONALL).getChild(0).getChild(0) : rewrittenTree) + .getChild(1); + if (updating()) { rewrittenCtx.setOperation(Context.Operation.UPDATE); rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); @@ -229,7 +239,7 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE); } - if (where != null && !copyOnWriteMode) { + if (where != null && (!copyOnWriteMode || updating())) { assert rewrittenInsert.getToken().getType() == HiveParser.TOK_INSERT : "Expected TOK_INSERT as second child of TOK_QUERY but found " + rewrittenInsert.getName(); // The structure of the AST for the rewritten insert statement is: From 162da6d861897761aa0255c43b4c1016e1bb9d1b Mon Sep 17 00:00:00 2001 From: "Daniel (Hongdan) Zhu" <50390050+DanielZhu58@users.noreply.github.com> Date: Wed, 18 Oct 2023 18:27:07 -0700 Subject: [PATCH 024/179] HIVE-27346: Getting exception for wildcard search for database and table name (#4326) (Hongdan Zhu, reviewed by Attila Turoczy, Zhihua Deng) --- .../hadoop/hive/metastore/ObjectStore.java | 2 +- .../metastore/client/TestTablesGetExists.java | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index e7df10673b5a..e8996cb24983 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -2163,7 +2163,7 @@ public List
getTableObjectsByName(String catName, String db, List StringBuilder filterBuilder = new StringBuilder(); List parameterVals = new ArrayList<>(); - appendSimpleCondition(filterBuilder, "database.name", new String[] {db}, parameterVals); + appendPatternCondition(filterBuilder, "database.name", new String[] {db}, parameterVals); appendSimpleCondition(filterBuilder, "database.catalogName", new String[] {catName}, parameterVals); if(tbl_names != null){ appendSimpleCondition(filterBuilder, "tableName", lowered_tbl_names.toArray(new String[0]), parameterVals); diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java index 273054c6dab6..f2937bc85760 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java @@ -271,6 +271,27 @@ public void testGetTables() throws Exception { Assert.assertEquals("Found functions size", 1, tables.size()); Assert.assertTrue("Comparing tablenames", tables.contains(testTables[6].getTableName())); + // Find tables by using the wildcard sign "*" + tables = client.getTables(DEFAULT_DATABASE, "*"); + Assert.assertEquals("All tables size", 5, tables.size()); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[0].getTableName())); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[1].getTableName())); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[2].getTableName())); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[3].getTableName())); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[4].getTableName())); + + tables = client.getTables(OTHER_DATABASE, "*"); + Assert.assertEquals("All tables size", 2, tables.size()); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[5].getTableName())); + Assert.assertTrue("Comparing tablenames", tables.contains(testTables[6].getTableName())); + + tables = client.getTables("*", "*"); + Assert.assertEquals("All tables size", 7, tables.size()); + tables = client.getTables("d*", "*"); + Assert.assertEquals("All tables size", 7, tables.size()); + tables = client.getTables("def*", "*"); + Assert.assertEquals("All tables size", 5, tables.size()); + // Look for tables but do not find any tables = client.getTables(DEFAULT_DATABASE, "*_not_such_function_*"); Assert.assertEquals("No such table size", 0, tables.size()); From b02cef4fe943b9aba597dcdfd3b8f3d3a5efca3e Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Thu, 19 Oct 2023 12:42:35 +0530 Subject: [PATCH 025/179] HIVE-27731: Iceberg: Perform metadata delete for queries with static filters (#4748) (Sourabh Badhya reviewed by Denys Kuzmenko, Krisztian Kasa) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 2 + .../mr/hive/HiveIcebergStorageHandler.java | 54 ++ .../iceberg/mr/hive/IcebergTableUtil.java | 14 + ...delete_iceberg_copy_on_write_partitioned.q | 1 + .../test/queries/positive/metadata_delete.q | 82 +++ .../positive/vectorized_iceberg_merge_mixed.q | 4 +- ...te_iceberg_copy_on_write_partitioned.q.out | 249 ++------- ..._iceberg_copy_on_write_unpartitioned.q.out | 68 +-- .../delete_iceberg_partitioned_avro.q.out | 2 + .../delete_iceberg_partitioned_orc.q.out | 6 +- .../delete_iceberg_partitioned_parquet.q.out | 6 +- ...delete_iceberg_unpartitioned_parquet.q.out | 2 + .../iceberg_atomic_merge_update.q.out | 6 +- .../positive/iceberg_copy_on_write.q.out | 8 +- ...rg_truncate_partition_with_evolution.q.out | 96 ++-- .../positive/llap/llap_iceberg_read_orc.q.out | 4 +- .../results/positive/metadata_delete.q.out | 501 ++++++++++++++++++ .../vectorized_iceberg_merge_mixed.q.out | 8 +- .../positive/write_iceberg_branch.q.out | 16 +- .../ql/io/sarg/ConvertAstToSearchArg.java | 39 +- .../hive/ql/metadata/HiveStorageHandler.java | 5 + .../hive/ql/parse/AlterTableExecuteSpec.java | 22 +- .../parse/UpdateDeleteSemanticAnalyzer.java | 86 ++- .../ql/io/sarg/TestConvertAstToSearchArg.java | 91 ++-- 24 files changed, 1004 insertions(+), 368 deletions(-) create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/metadata_delete.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index adc6503debeb..a8378f097d39 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2672,6 +2672,8 @@ public static enum ConfVars { HIVE_OPTIMIZE_REPLACE_DELETE_WITH_TRUNCATE("hive.optimize.delete.all", false, "Optimize delete the entire data from table, use truncate instead"), + HIVE_OPTIMIZE_METADATA_DELETE("hive.optimize.delete.metadata.only", true, + "Optimize delete the entire data from table, use truncate instead"), HIVE_OPTIMIZE_LIMIT("hive.optimize.limit", true, "Optimize limit by pushing through Left Outer Joins and Selects"), HIVE_OPTIMIZE_TOPNKEY("hive.optimize.topnkey", true, "Whether to enable top n key optimizer."), diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 8c8b54256f43..c11663e3a659 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -155,10 +155,15 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.expressions.StrictMetricsEvaluator; import org.apache.iceberg.hadoop.HadoopConfigurable; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.mr.Catalogs; @@ -175,6 +180,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Throwables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -185,6 +191,7 @@ import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.SerializationUtil; +import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.StructProjection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -832,6 +839,12 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable, (AlterTableExecuteSpec.CherryPickSpec) executeSpec.getOperationParams(); IcebergTableUtil.cherryPick(icebergTable, cherryPickSpec.getSnapshotId()); break; + case DELETE_METADATA: + AlterTableExecuteSpec.DeleteMetadataSpec deleteMetadataSpec = + (AlterTableExecuteSpec.DeleteMetadataSpec) executeSpec.getOperationParams(); + IcebergTableUtil.performMetadataDelete(icebergTable, deleteMetadataSpec.getBranchName(), + deleteMetadataSpec.getSarg()); + break; default: throw new UnsupportedOperationException( String.format("Operation type %s is not supported", executeSpec.getOperationType().name())); @@ -1878,4 +1891,45 @@ public ColumnInfo getColumnInfo(org.apache.hadoop.hive.ql.metadata.Table hmsTabl throw new SemanticException(String.format("Unable to find a column with the name: %s", colName)); } } + + @Override + public boolean canPerformMetadataDelete(org.apache.hadoop.hive.ql.metadata.Table hmsTable, + String branchName, SearchArgument sarg) { + Expression exp; + try { + exp = HiveIcebergFilterFactory.generateFilterExpression(sarg); + } catch (UnsupportedOperationException e) { + LOG.warn("Unable to create Iceberg filter," + + " continuing without metadata delete: ", e); + return false; + } + Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + + // The following code is inspired & copied from Iceberg's SparkTable.java#canDeleteUsingMetadata + if (ExpressionUtil.selectsPartitions(exp, table, false)) { + return true; + } + + TableScan scan = table.newScan().filter(exp).caseSensitive(false).includeColumnStats().ignoreResiduals(); + if (branchName != null) { + scan.useRef(HiveUtils.getTableSnapshotRef(branchName)); + } + + try (CloseableIterable tasks = scan.planFiles()) { + Map evaluators = Maps.newHashMap(); + StrictMetricsEvaluator metricsEvaluator = + new StrictMetricsEvaluator(SnapshotUtil.schemaFor(table, branchName), exp); + + return Iterables.all(tasks, task -> { + DataFile file = task.file(); + PartitionSpec spec = task.spec(); + Evaluator evaluator = evaluators.computeIfAbsent(spec.specId(), specId -> + new Evaluator(spec.partitionType(), Projections.strict(spec).project(exp))); + return evaluator.eval(file.partition()) || metricsEvaluator.eval(file); + }); + } catch (IOException ioe) { + LOG.warn("Failed to close task iterable", ioe); + return false; + } + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java index 8527e25cbfe6..aacbf4c4e3d7 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java @@ -24,13 +24,17 @@ import java.util.Optional; import java.util.Properties; import java.util.function.Function; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; import org.apache.hadoop.hive.ql.parse.TransformSpec; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.session.SessionStateUtil; +import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.ManageSnapshots; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -39,6 +43,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdatePartitionSpec; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; @@ -278,4 +283,13 @@ public static boolean isV2Table(Map props) { return props != null && "2".equals(props.get(TableProperties.FORMAT_VERSION)); } + + public static void performMetadataDelete(Table icebergTable, String branchName, SearchArgument sarg) { + Expression exp = HiveIcebergFilterFactory.generateFilterExpression(sarg); + DeleteFiles deleteFiles = icebergTable.newDelete(); + if (StringUtils.isNotEmpty(branchName)) { + deleteFiles = deleteFiles.toBranch(HiveUtils.getTableSnapshotRef(branchName)); + } + deleteFiles.deleteFromRowFilter(exp).commit(); + } } diff --git a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q index 86c1b59ce23c..84114c48dedb 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_partitioned.q @@ -1,3 +1,4 @@ +--! qt:replace:/DeleteMetadataSpec(\S*)/#Masked#/ set hive.explain.user=false; drop table if exists tbl_ice; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/metadata_delete.q b/iceberg/iceberg-handler/src/test/queries/positive/metadata_delete.q new file mode 100644 index 000000000000..ca3ff2566067 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/metadata_delete.q @@ -0,0 +1,82 @@ +-- SORT_QUERY_RESULTS +--! qt:replace:/DeleteMetadataSpec(\S*)/#Masked#/ +set hive.explain.user=false; + +create table ice_date (a int, b date) stored by iceberg stored as orc tblproperties ('format-version'='2'); + +insert into table ice_date values (1, '2021-01-01'); +insert into table ice_date values (2, '2022-02-02'), (3, '2022-03-03'); + +delete from ice_date where b = '2022-02-02'; +delete from ice_date where a = 1 and b = '2021-01-01'; + +select * from ice_date; + +create table ice_date_year (a int, b date) stored by iceberg stored as orc tblproperties ('format-version'='2'); +insert into table ice_date_year values (1, '2021-01-01'); +insert into table ice_date_year values (2, '2022-02-02'), (3, '2022-03-03'); + +delete from ice_date_year where year(b) = 2022; + +select * from ice_date_year; + +-- Metadata delete should not be done here and fallback to normal delete. +create table ice_str_name (first_name string, last_name string) stored by iceberg stored as orc tblproperties ('format-version'='2'); +insert into table ice_str_name values ('Alex', 'Clark'); +insert into table ice_str_name values ('Bob', 'Bob'); + +delete from ice_str_name where first_name = last_name; + +select * from ice_str_name; + +-- Metadata delete should not be done here and fallback to normal delete. +create table ice_int_id (first_id int, last_id int) stored by iceberg stored as orc tblproperties ('format-version'='2'); +insert into table ice_int_id values (7, 9); +insert into table ice_int_id values (8, 8); + +delete from ice_int_id where first_id = last_id; + +select * from ice_int_id; + +-- Check if delete on a branch also uses the metadata delete whenever possible. +create table ice_branch_metadata_delete (a int, b string) stored by iceberg stored as orc tblproperties ('format-version'='2'); +insert into table ice_branch_metadata_delete values (1, 'ABC'); +insert into table ice_branch_metadata_delete values (2, 'DEF'); +insert into table ice_branch_metadata_delete values (3, 'GHI'); +insert into table ice_branch_metadata_delete values (4, 'JKL'); + +alter table ice_branch_metadata_delete create branch test01; +delete from default.ice_branch_metadata_delete.branch_test01 where a = 1; + +select * from default.ice_branch_metadata_delete.branch_test01; + +alter table ice_branch_metadata_delete drop branch test01; + +-- Metadata delete must not be applied for multi-table scans which have subquery and fallback to normal delete logic. +create table ice_delete_multiple_table1 (a int, b string) stored by iceberg stored as orc tblproperties ('format-version' = '2'); +create table ice_delete_multiple_table2 (a int, b string) stored by iceberg stored as orc tblproperties ('format-version' = '2'); +insert into table ice_delete_multiple_table1 values (1, 'ABC'), (2, 'DEF'), (3, 'GHI'); +insert into table ice_delete_multiple_table1 values (4, 'GHI'), (5, 'JKL'), (6, 'PQR'); +insert into table ice_delete_multiple_table2 values (1, 'ABC'), (2, 'DEF'), (3, 'GHI'); +insert into table ice_delete_multiple_table2 values (4, 'GHI'), (5, 'JKL'), (6, 'PQR'); + +delete from ice_delete_multiple_table2 where ice_delete_multiple_table2.a in (select ice_delete_multiple_table1.a from ice_delete_multiple_table1 where ice_delete_multiple_table1.b = 'GHI'); + +select * from ice_delete_multiple_table2; + +create table test_delete_config (a int, b int) stored by iceberg stored as orc tblproperties ('format-version'='2'); +insert into table test_delete_config values (1,2), (3,4); + +explain delete from test_delete_config where b < 5; + +set hive.optimize.delete.metadata.only=false; +explain delete from test_delete_config where b < 5; + +drop table ice_date; +drop table ice_date_year; +drop table ice_str_name; +drop table ice_int_id; +drop table ice_branch_metadata_delete; +drop table ice_delete_multiple_table1; +drop table ice_delete_multiple_table2; +drop table test_delete_config; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q index 3d14c5134ee5..080857ef1380 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q @@ -3,9 +3,9 @@ -- Mask neededVirtualColumns due to non-strict order --! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ -- Mask width ---! qt:replace:/(width=17)\d+/$1####/ +--! qt:replace:/(width=58)\d+/$1###/ -- Mask total data size ---! qt:replace:/(Data size: 35)\d+/$1####/ +--! qt:replace:/(Data size: 11)\d+/$1####/ set hive.vectorized.execution.enabled=true; set hive.llap.io.enabled=false; diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out index 0b84e42f45cc..91fd18b711f7 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out @@ -23,211 +23,28 @@ POSTHOOK: Output: default@tbl_ice PREHOOK: query: explain delete from tbl_ice where b in ('one', 'four') or a = 22 PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice -PREHOOK: Output: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### POSTHOOK: query: explain delete from tbl_ice where b in ('one', 'four') or a = 22 POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice -POSTHOOK: Output: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 - Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 3 (CONTAINS) - Reducer 4 <- Map 1 (SIMPLE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 3 (CONTAINS) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: tbl_ice - filterExpr: (((a <> 22) and (b <> 'one') and (b <> 'four')) or (b) IN ('one', 'four') or (a = 22)) (type: boolean) - Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col5 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 7 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) - Filter Operator - predicate: (((b) IN ('one', 'four') or (a = 22)) and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: FILE__PATH (type: string) - null sort order: a - sort order: + - Map-reduce partition columns: FILE__PATH (type: string) - Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized - Map 5 - Map Operator Tree: - TableScan - alias: tbl_ice - filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) - Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) - Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: FILE__PATH (type: string) - null sort order: a - sort order: + - Map-reduce partition columns: FILE__PATH (type: string) - Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint) - Execution mode: vectorized - Reducer 2 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col5 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 4 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice - Reducer 4 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: string) - outputColumnNames: _col5 - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE - PTF Operator - Function definitions: - Input definition - input alias: ptf_0 - output shape: _col5: string - type: WINDOWING - Windowing table definition - input alias: ptf_1 - name: windowingtablefunction - order by: _col5 ASC NULLS FIRST - partition by: _col5 - raw input shape: - window functions: - window function definition - alias: row_number_window_0 - name: row_number - window function: GenericUDAFRowNumberEvaluator - window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) - isPivotResult: true - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col5 (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - keys: _col0 (type: string) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 6 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE - PTF Operator - Function definitions: - Input definition - input alias: ptf_0 - type: WINDOWING - Windowing table definition - input alias: ptf_1 - name: windowingtablefunction - order by: _col5 ASC NULLS FIRST - partition by: _col5 - raw input shape: - window functions: - window function definition - alias: row_number_window_0 - name: row_number - window function: GenericUDAFRowNumberEvaluator - window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) - isPivotResult: true - Statistics: Num rows: 4 Data size: 1168 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 2 Data size: 584 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 4 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice - Union 3 - Vertex: Union 3 - - Stage: Stage-2 - Dependency Collection - - Stage: Stage-0 - Move Operator - tables: - replace: false - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice - - Stage: Stage-3 - Stats Work - Basic Stats Work: + Execute operation + table name: default.tbl_ice + spec: AlterTableExecuteSpec{operationType=DELETE_METADATA, operationParams=org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec$#Masked# PREHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice -PREHOOK: Output: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### POSTHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice -POSTHOOK: Output: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### PREHOOK: query: select * from tbl_ice order by a PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -248,6 +65,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product @@ -263,13 +82,13 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-4 Tez #### A masked pattern was here #### Edges: @@ -871,10 +690,10 @@ STAGE PLANS: Union 7 Vertex: Union 7 - Stage: Stage-2 + Stage: Stage-5 Dependency Collection - Stage: Stage-0 + Stage: Stage-3 Move Operator tables: replace: false @@ -884,10 +703,12 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-6 Stats Work Basic Stats Work: +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product @@ -946,13 +767,13 @@ POSTHOOK: Input: default@tbl_ice POSTHOOK: Input: default@tbl_ice_other POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-4 Tez #### A masked pattern was here #### Edges: @@ -1311,10 +1132,10 @@ STAGE PLANS: Union 5 Vertex: Union 5 - Stage: Stage-2 + Stage: Stage-5 Dependency Collection - Stage: Stage-0 + Stage: Stage-3 Move Operator tables: replace: false @@ -1324,7 +1145,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-6 Stats Work Basic Stats Work: @@ -1384,13 +1205,13 @@ POSTHOOK: Input: default@tbl_ice POSTHOOK: Input: default@tbl_ice_other POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-4 Tez #### A masked pattern was here #### Edges: @@ -1769,10 +1590,10 @@ STAGE PLANS: Union 5 Vertex: Union 5 - Stage: Stage-2 + Stage: Stage-5 Dependency Collection - Stage: Stage-0 + Stage: Stage-3 Move Operator tables: replace: false @@ -1782,7 +1603,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-6 Stats Work Basic Stats Work: diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out index d9f95eb36f63..22256bab0106 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out @@ -29,13 +29,13 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -203,10 +203,10 @@ STAGE PLANS: Union 3 Vertex: Union 3 - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -216,7 +216,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: @@ -248,6 +248,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product @@ -263,13 +265,13 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-4 Tez #### A masked pattern was here #### Edges: @@ -871,10 +873,10 @@ STAGE PLANS: Union 7 Vertex: Union 7 - Stage: Stage-2 + Stage: Stage-5 Dependency Collection - Stage: Stage-0 + Stage: Stage-3 Move Operator tables: replace: false @@ -884,10 +886,12 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-6 Stats Work Basic Stats Work: +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product @@ -946,13 +950,13 @@ POSTHOOK: Input: default@tbl_ice POSTHOOK: Input: default@tbl_ice_other POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-4 Tez #### A masked pattern was here #### Edges: @@ -1311,10 +1315,10 @@ STAGE PLANS: Union 5 Vertex: Union 5 - Stage: Stage-2 + Stage: Stage-5 Dependency Collection - Stage: Stage-0 + Stage: Stage-3 Move Operator tables: replace: false @@ -1324,7 +1328,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-6 Stats Work Basic Stats Work: @@ -1384,13 +1388,13 @@ POSTHOOK: Input: default@tbl_ice POSTHOOK: Input: default@tbl_standard_other POSTHOOK: Output: default@tbl_ice STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + Stage-5 depends on stages: Stage-4 + Stage-3 depends on stages: Stage-5 + Stage-6 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-4 Tez #### A masked pattern was here #### Edges: @@ -1769,10 +1773,10 @@ STAGE PLANS: Union 5 Vertex: Union 5 - Stage: Stage-2 + Stage: Stage-5 Dependency Collection - Stage: Stage-0 + Stage: Stage-3 Move Operator tables: replace: false @@ -1782,7 +1786,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Stage: Stage-3 + Stage: Stage-6 Stats Work Basic Stats Work: diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out index cbcb6e1f893a..38d0bc838386 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out @@ -48,6 +48,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out index 7efc58b0e6f7..2111826b9bf8 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out @@ -23,11 +23,11 @@ POSTHOOK: Output: default@tbl_ice PREHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice -PREHOOK: Output: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### POSTHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice -POSTHOOK: Output: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### PREHOOK: query: select * from tbl_ice order by a PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -48,6 +48,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out index 065128ce711c..43e01378a2cb 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out @@ -23,11 +23,11 @@ POSTHOOK: Output: default@tbl_ice PREHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice -PREHOOK: Output: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### POSTHOOK: query: delete from tbl_ice where b in ('one', 'four') or a = 22 POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice -POSTHOOK: Output: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### PREHOOK: query: select * from tbl_ice order by a PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -48,6 +48,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out index fbdb9ca29da1..3672e210f2e1 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out @@ -48,6 +48,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out index 7bc1a8db1352..403fb63f2c09 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out @@ -218,11 +218,11 @@ POSTHOOK: Output: default@calls_v2 PREHOOK: query: delete from calls_v2 where year=2024 PREHOOK: type: QUERY PREHOOK: Input: default@calls_v2 -PREHOOK: Output: default@calls_v2 +PREHOOK: Output: hdfs://### HDFS PATH ### POSTHOOK: query: delete from calls_v2 where year=2024 POSTHOOK: type: QUERY POSTHOOK: Input: default@calls_v2 -POSTHOOK: Output: default@calls_v2 +POSTHOOK: Output: hdfs://### HDFS PATH ### PREHOOK: query: select s.operation, s.summary['added-records'], s.summary['deleted-records'] from default.calls_v2.snapshots s order by s.snapshot_id PREHOOK: type: QUERY @@ -234,8 +234,8 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@calls_v2 POSTHOOK: Output: hdfs://### HDFS PATH ### append 10 NULL +delete NULL 4 overwrite 2 NULL -overwrite NULL NULL PREHOOK: query: DROP TABLE calls_v2 PREHOOK: type: DROPTABLE PREHOOK: Input: default@calls_v2 diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out index 83ee890d7b75..4d80795bc156 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out @@ -72,14 +72,14 @@ Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 3 (CONTAINS) Reducer 4 <- Map 1 (SIMPLE_EDGE) Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 3 (CONTAINS) -Stage-3 +Stage-4 Stats Work{} - Stage-0 + Stage-1 Move Operator table:{"name:":"default.ice01"} - Stage-2 + Stage-3 Dependency Collection{} - Stage-1 + Stage-2 Union 3 <-Reducer 2 [CONTAINS] File Output Operator [FS_46] diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out index 25d1246361e8..3217a146827a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out @@ -70,13 +70,13 @@ POSTHOOK: query: explain truncate table test_ice_int partition (a = 22) POSTHOOK: type: QUERY POSTHOOK: Output: default@test_ice_int@a=22 STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -119,10 +119,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_int - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -132,7 +132,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_int - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: @@ -281,13 +281,13 @@ POSTHOOK: query: explain truncate table test_ice_bigint partition (a = 226784902 POSTHOOK: type: QUERY POSTHOOK: Output: default@test_ice_bigint@a=226784902765739 STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -330,10 +330,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_bigint - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -343,7 +343,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_bigint - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: @@ -518,13 +518,13 @@ POSTHOOK: query: explain truncate table test_ice_str partition (b = 'ddd') POSTHOOK: type: QUERY POSTHOOK: Output: default@test_ice_str@b=ddd STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -567,10 +567,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_str - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -580,7 +580,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_str - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: @@ -775,13 +775,13 @@ POSTHOOK: query: explain truncate table test_ice_date partition (b = '2022-02-07 POSTHOOK: type: QUERY POSTHOOK: Output: default@test_ice_date@b=2022-02-07 STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -824,10 +824,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_date - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -837,7 +837,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_date - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: @@ -979,13 +979,13 @@ POSTHOOK: query: explain truncate table test_ice_double partition (a = 115674892 POSTHOOK: type: QUERY POSTHOOK: Output: default@test_ice_double@a=115674892756.67590946 STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -1028,10 +1028,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_double - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -1041,7 +1041,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_double - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: @@ -1177,13 +1177,13 @@ POSTHOOK: query: explain truncate table test_ice_double_date partition (a = 1156 POSTHOOK: type: QUERY POSTHOOK: Output: default@test_ice_double_date@a=115674892756.67590946/b=2022-02-07 STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -1225,10 +1225,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_double_date - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -1238,7 +1238,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.test_ice_double_date - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out index 442257dd5400..46bf3a38bcba 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out @@ -212,11 +212,11 @@ POSTHOOK: Output: default@target_ice PREHOOK: query: DELETE FROM target_ice WHERE a = 1 PREHOOK: type: QUERY PREHOOK: Input: default@target_ice -PREHOOK: Output: default@target_ice +#### A masked pattern was here #### POSTHOOK: query: DELETE FROM target_ice WHERE a = 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@target_ice -POSTHOOK: Output: default@target_ice +#### A masked pattern was here #### PREHOOK: query: SELECT * FROM target_ice PREHOOK: type: QUERY PREHOOK: Input: default@target_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out b/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out new file mode 100644 index 000000000000..e2b343c9d042 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out @@ -0,0 +1,501 @@ +PREHOOK: query: create table ice_date (a int, b date) stored by iceberg stored as orc tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date +POSTHOOK: query: create table ice_date (a int, b date) stored by iceberg stored as orc tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date +PREHOOK: query: insert into table ice_date values (1, '2021-01-01') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date +POSTHOOK: query: insert into table ice_date values (1, '2021-01-01') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date +PREHOOK: query: insert into table ice_date values (2, '2022-02-02'), (3, '2022-03-03') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date +POSTHOOK: query: insert into table ice_date values (2, '2022-02-02'), (3, '2022-03-03') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date +PREHOOK: query: delete from ice_date where b = '2022-02-02' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date +PREHOOK: Output: default@ice_date +POSTHOOK: query: delete from ice_date where b = '2022-02-02' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: default@ice_date +PREHOOK: query: delete from ice_date where a = 1 and b = '2021-01-01' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: delete from ice_date where a = 1 and b = '2021-01-01' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: select * from ice_date +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +3 2022-03-03 +PREHOOK: query: create table ice_date_year (a int, b date) stored by iceberg stored as orc tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_year +POSTHOOK: query: create table ice_date_year (a int, b date) stored by iceberg stored as orc tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_year +PREHOOK: query: insert into table ice_date_year values (1, '2021-01-01') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_year +POSTHOOK: query: insert into table ice_date_year values (1, '2021-01-01') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_year +PREHOOK: query: insert into table ice_date_year values (2, '2022-02-02'), (3, '2022-03-03') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_year +POSTHOOK: query: insert into table ice_date_year values (2, '2022-02-02'), (3, '2022-03-03') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_year +PREHOOK: query: delete from ice_date_year where year(b) = 2022 +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date_year +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: delete from ice_date_year where year(b) = 2022 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date_year +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: select * from ice_date_year +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date_year +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_date_year +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date_year +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 2021-01-01 +PREHOOK: query: create table ice_str_name (first_name string, last_name string) stored by iceberg stored as orc tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_str_name +POSTHOOK: query: create table ice_str_name (first_name string, last_name string) stored by iceberg stored as orc tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_str_name +PREHOOK: query: insert into table ice_str_name values ('Alex', 'Clark') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str_name +POSTHOOK: query: insert into table ice_str_name values ('Alex', 'Clark') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str_name +PREHOOK: query: insert into table ice_str_name values ('Bob', 'Bob') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str_name +POSTHOOK: query: insert into table ice_str_name values ('Bob', 'Bob') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str_name +PREHOOK: query: delete from ice_str_name where first_name = last_name +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_str_name +PREHOOK: Output: default@ice_str_name +POSTHOOK: query: delete from ice_str_name where first_name = last_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_str_name +POSTHOOK: Output: default@ice_str_name +PREHOOK: query: select * from ice_str_name +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_str_name +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_str_name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_str_name +POSTHOOK: Output: hdfs://### HDFS PATH ### +Alex Clark +PREHOOK: query: create table ice_int_id (first_id int, last_id int) stored by iceberg stored as orc tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_int_id +POSTHOOK: query: create table ice_int_id (first_id int, last_id int) stored by iceberg stored as orc tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_int_id +PREHOOK: query: insert into table ice_int_id values (7, 9) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_id +POSTHOOK: query: insert into table ice_int_id values (7, 9) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_id +PREHOOK: query: insert into table ice_int_id values (8, 8) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_id +POSTHOOK: query: insert into table ice_int_id values (8, 8) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_id +PREHOOK: query: delete from ice_int_id where first_id = last_id +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_int_id +PREHOOK: Output: default@ice_int_id +POSTHOOK: query: delete from ice_int_id where first_id = last_id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_int_id +POSTHOOK: Output: default@ice_int_id +PREHOOK: query: select * from ice_int_id +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_int_id +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_int_id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_int_id +POSTHOOK: Output: hdfs://### HDFS PATH ### +7 9 +PREHOOK: query: create table ice_branch_metadata_delete (a int, b string) stored by iceberg stored as orc tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_branch_metadata_delete +POSTHOOK: query: create table ice_branch_metadata_delete (a int, b string) stored by iceberg stored as orc tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_branch_metadata_delete +PREHOOK: query: insert into table ice_branch_metadata_delete values (1, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_branch_metadata_delete +POSTHOOK: query: insert into table ice_branch_metadata_delete values (1, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_branch_metadata_delete +PREHOOK: query: insert into table ice_branch_metadata_delete values (2, 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_branch_metadata_delete +POSTHOOK: query: insert into table ice_branch_metadata_delete values (2, 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_branch_metadata_delete +PREHOOK: query: insert into table ice_branch_metadata_delete values (3, 'GHI') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_branch_metadata_delete +POSTHOOK: query: insert into table ice_branch_metadata_delete values (3, 'GHI') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_branch_metadata_delete +PREHOOK: query: insert into table ice_branch_metadata_delete values (4, 'JKL') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_branch_metadata_delete +POSTHOOK: query: insert into table ice_branch_metadata_delete values (4, 'JKL') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_branch_metadata_delete +PREHOOK: query: alter table ice_branch_metadata_delete create branch test01 +PREHOOK: type: ALTERTABLE_CREATEBRANCH +PREHOOK: Input: default@ice_branch_metadata_delete +POSTHOOK: query: alter table ice_branch_metadata_delete create branch test01 +POSTHOOK: type: ALTERTABLE_CREATEBRANCH +POSTHOOK: Input: default@ice_branch_metadata_delete +PREHOOK: query: delete from default.ice_branch_metadata_delete.branch_test01 where a = 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_branch_metadata_delete +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: delete from default.ice_branch_metadata_delete.branch_test01 where a = 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_branch_metadata_delete +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: select * from default.ice_branch_metadata_delete.branch_test01 +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_branch_metadata_delete +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from default.ice_branch_metadata_delete.branch_test01 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_branch_metadata_delete +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 DEF +3 GHI +4 JKL +PREHOOK: query: alter table ice_branch_metadata_delete drop branch test01 +PREHOOK: type: ALTERTABLE_DROPBRANCH +PREHOOK: Input: default@ice_branch_metadata_delete +POSTHOOK: query: alter table ice_branch_metadata_delete drop branch test01 +POSTHOOK: type: ALTERTABLE_DROPBRANCH +POSTHOOK: Input: default@ice_branch_metadata_delete +PREHOOK: query: create table ice_delete_multiple_table1 (a int, b string) stored by iceberg stored as orc tblproperties ('format-version' = '2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_delete_multiple_table1 +POSTHOOK: query: create table ice_delete_multiple_table1 (a int, b string) stored by iceberg stored as orc tblproperties ('format-version' = '2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_delete_multiple_table1 +PREHOOK: query: create table ice_delete_multiple_table2 (a int, b string) stored by iceberg stored as orc tblproperties ('format-version' = '2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_delete_multiple_table2 +POSTHOOK: query: create table ice_delete_multiple_table2 (a int, b string) stored by iceberg stored as orc tblproperties ('format-version' = '2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_delete_multiple_table2 +PREHOOK: query: insert into table ice_delete_multiple_table1 values (1, 'ABC'), (2, 'DEF'), (3, 'GHI') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_delete_multiple_table1 +POSTHOOK: query: insert into table ice_delete_multiple_table1 values (1, 'ABC'), (2, 'DEF'), (3, 'GHI') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_delete_multiple_table1 +PREHOOK: query: insert into table ice_delete_multiple_table1 values (4, 'GHI'), (5, 'JKL'), (6, 'PQR') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_delete_multiple_table1 +POSTHOOK: query: insert into table ice_delete_multiple_table1 values (4, 'GHI'), (5, 'JKL'), (6, 'PQR') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_delete_multiple_table1 +PREHOOK: query: insert into table ice_delete_multiple_table2 values (1, 'ABC'), (2, 'DEF'), (3, 'GHI') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_delete_multiple_table2 +POSTHOOK: query: insert into table ice_delete_multiple_table2 values (1, 'ABC'), (2, 'DEF'), (3, 'GHI') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_delete_multiple_table2 +PREHOOK: query: insert into table ice_delete_multiple_table2 values (4, 'GHI'), (5, 'JKL'), (6, 'PQR') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_delete_multiple_table2 +POSTHOOK: query: insert into table ice_delete_multiple_table2 values (4, 'GHI'), (5, 'JKL'), (6, 'PQR') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_delete_multiple_table2 +PREHOOK: query: delete from ice_delete_multiple_table2 where ice_delete_multiple_table2.a in (select ice_delete_multiple_table1.a from ice_delete_multiple_table1 where ice_delete_multiple_table1.b = 'GHI') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_delete_multiple_table1 +PREHOOK: Input: default@ice_delete_multiple_table2 +PREHOOK: Output: default@ice_delete_multiple_table2 +POSTHOOK: query: delete from ice_delete_multiple_table2 where ice_delete_multiple_table2.a in (select ice_delete_multiple_table1.a from ice_delete_multiple_table1 where ice_delete_multiple_table1.b = 'GHI') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_delete_multiple_table1 +POSTHOOK: Input: default@ice_delete_multiple_table2 +POSTHOOK: Output: default@ice_delete_multiple_table2 +PREHOOK: query: select * from ice_delete_multiple_table2 +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_delete_multiple_table2 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_delete_multiple_table2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_delete_multiple_table2 +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 ABC +2 DEF +5 JKL +6 PQR +PREHOOK: query: create table test_delete_config (a int, b int) stored by iceberg stored as orc tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_delete_config +POSTHOOK: query: create table test_delete_config (a int, b int) stored by iceberg stored as orc tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_delete_config +PREHOOK: query: insert into table test_delete_config values (1,2), (3,4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@test_delete_config +POSTHOOK: query: insert into table test_delete_config values (1,2), (3,4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@test_delete_config +PREHOOK: query: explain delete from test_delete_config where b < 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@test_delete_config +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain delete from test_delete_config where b < 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_delete_config +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Execute operation + table name: default.test_delete_config + spec: AlterTableExecuteSpec{operationType=DELETE_METADATA, operationParams=org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec$#Masked# + +PREHOOK: query: explain delete from test_delete_config where b < 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@test_delete_config +PREHOOK: Output: default@test_delete_config +POSTHOOK: query: explain delete from test_delete_config where b < 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_delete_config +POSTHOOK: Output: default@test_delete_config +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: test_delete_config + filterExpr: (b < 5) (type: boolean) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (b < 5) (type: boolean) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), b (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + null sort order: zzzz + sort order: ++++ + Statistics: Num rows: 2 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col4 (type: int), _col5 (type: int) + Execution mode: vectorized + Reducer 2 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 2 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_delete_config + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_delete_config + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: drop table ice_date +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_date +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date +POSTHOOK: query: drop table ice_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date +PREHOOK: query: drop table ice_date_year +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_date_year +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_year +POSTHOOK: query: drop table ice_date_year +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_date_year +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_year +PREHOOK: query: drop table ice_str_name +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_str_name +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_str_name +POSTHOOK: query: drop table ice_str_name +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_str_name +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_str_name +PREHOOK: query: drop table ice_int_id +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_int_id +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_int_id +POSTHOOK: query: drop table ice_int_id +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_int_id +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_int_id +PREHOOK: query: drop table ice_branch_metadata_delete +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_branch_metadata_delete +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_branch_metadata_delete +POSTHOOK: query: drop table ice_branch_metadata_delete +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_branch_metadata_delete +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_branch_metadata_delete +PREHOOK: query: drop table ice_delete_multiple_table1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_delete_multiple_table1 +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_delete_multiple_table1 +POSTHOOK: query: drop table ice_delete_multiple_table1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_delete_multiple_table1 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_delete_multiple_table1 +PREHOOK: query: drop table ice_delete_multiple_table2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_delete_multiple_table2 +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_delete_multiple_table2 +POSTHOOK: query: drop table ice_delete_multiple_table2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_delete_multiple_table2 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_delete_multiple_table2 +PREHOOK: query: drop table test_delete_config +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@test_delete_config +PREHOOK: Output: database:default +PREHOOK: Output: default@test_delete_config +POSTHOOK: query: drop table test_delete_config +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@test_delete_config +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_delete_config diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out index 44d5690808e4..c031030ee561 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out @@ -83,11 +83,11 @@ POSTHOOK: Output: default@store_sales PREHOOK: query: delete from store_sales where ss_customer_sk > 2 PREHOOK: type: QUERY PREHOOK: Input: default@store_sales -PREHOOK: Output: default@store_sales +PREHOOK: Output: hdfs://### HDFS PATH ### POSTHOOK: query: delete from store_sales where ss_customer_sk > 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@store_sales -POSTHOOK: Output: default@store_sales +POSTHOOK: Output: hdfs://### HDFS PATH ### PREHOOK: query: select count(*) from store_sales PREHOOK: type: QUERY PREHOOK: Input: default@store_sales @@ -533,7 +533,7 @@ STAGE PLANS: TableScan alias: store_sales filterExpr: ((ss_sold_date_sk = 2451181) and ss_item_sk is not null and ss_customer_sk is not null) (type: boolean) - Statistics: Num rows: 2 Data size: 35#### Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 11#### Basic stats: COMPLETE Column stats: COMPLETE TableScan Vectorization: native: true vectorizationSchemaColumns: [0:ss_sold_date_sk:int, 1:ss_sold_time_sk:int, 2:ss_item_sk:int, 3:ss_customer_sk:int, 4:ss_cdemo_sk:int, 5:ss_hdemo_sk:int, 6:ss_addr_sk:int, 7:ss_store_sk:int, 8:ss_promo_sk:int, 9:ss_ticket_number:int, 10:ss_quantity:int, 11:ss_wholesale_cost:decimal(7,2), 12:ss_list_price:decimal(7,2), 13:ss_sales_price:decimal(7,2), 14:ss_ext_discount_amt:decimal(7,2), 15:ss_ext_sales_price:decimal(7,2), 16:ss_ext_wholesale_cost:decimal(7,2), 17:ss_ext_list_price:decimal(7,2), 18:ss_ext_tax:decimal(7,2), 19:ss_coupon_amt:decimal(7,2), 20:ss_net_paid:decimal(7,2), 21:ss_net_paid_inc_tax:decimal(7,2), 22:ss_net_profit:decimal(7,2), 23:PARTITION__SPEC__ID:int, 24:PARTITION__HASH:bigint, 25:FILE__PATH:string, 26:ROW__POSITION:bigint] @@ -957,7 +957,7 @@ Stage-6 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26"] Filter Operator [FIL_46] (rows=2 width=700) predicate:((ss_sold_date_sk = 2451181) and ss_item_sk is not null and ss_customer_sk is not null) - TableScan [TS_2] (rows=2 width=17####) + TableScan [TS_2] (rows=2 width=58###) default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_sold_time_sk","ss_item_sk","ss_customer_sk","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_quantity","ss_wholesale_cost","ss_list_price","ss_sales_price","ss_ext_discount_amt","ss_ext_sales_price","ss_ext_wholesale_cost","ss_ext_list_price","ss_ext_tax","ss_coupon_amt","ss_net_paid","ss_net_paid_inc_tax","ss_net_profit"] <-Select Operator [SEL_49] (rows=5 width=380) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"] diff --git a/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out b/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out index dc6fcbfbf7ef..dbafa73aa522 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out @@ -218,13 +218,13 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@ice01 POSTHOOK: Output: default@ice01 STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-2 depends on stages: Stage-1 - Stage-0 depends on stages: Stage-2 - Stage-3 depends on stages: Stage-0 + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-2 Tez #### A masked pattern was here #### Edges: @@ -268,10 +268,10 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.ice01 - Stage: Stage-2 + Stage: Stage-3 Dependency Collection - Stage: Stage-0 + Stage: Stage-1 Move Operator tables: replace: false @@ -281,7 +281,7 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.ice01 - Stage: Stage-3 + Stage: Stage-4 Stats Work Basic Stats Work: diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java index 0de48cca843f..de6f88b932c7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java @@ -21,7 +21,6 @@ import java.sql.Date; import java.sql.Timestamp; import java.time.LocalDate; -import java.util.GregorianCalendar; import java.util.List; import java.util.concurrent.ExecutionException; @@ -52,7 +51,6 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr; -import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; @@ -74,6 +72,7 @@ public class ConvertAstToSearchArg { private final SearchArgument.Builder builder; private final Configuration conf; + private boolean partial = false; /* * Create a new type for handling precision conversions from Decimal -> Double/Float @@ -106,6 +105,14 @@ private static enum BoxType { parse(expression); } + /** + * Returns whether the given expression is partially converted to a search argument from the hive filter. + * @return True if the expression is partially converted, otherwise false. + */ + public boolean isPartial() { + return partial; + } + /** * Build the search argument from the expression. * @return the search argument @@ -316,11 +323,13 @@ private void createLeaf(PredicateLeaf.Operator operator, String columnName = getColumnName(expression, variable); if (columnName == null) { builder.literal(SearchArgument.TruthValue.YES_NO_NULL); + partial = true; return; } BoxType boxType = getType(expression.getChildren().get(variable)); if (boxType == null) { builder.literal(SearchArgument.TruthValue.YES_NO_NULL); + partial = true; return; } @@ -370,6 +379,7 @@ private void createLeaf(PredicateLeaf.Operator operator, LOG.warn("Exception thrown during SARG creation. Returning YES_NO_NULL." + " Exception: " + e.getMessage()); builder.literal(SearchArgument.TruthValue.YES_NO_NULL); + partial = true; } if (needSwap) { @@ -438,6 +448,7 @@ private void parse(ExprNodeDesc expression) { // otherwise, we don't know what to do so make it a maybe builder.literal(SearchArgument.TruthValue.YES_NO_NULL); + partial = true; return; } @@ -499,6 +510,7 @@ private void parse(ExprNodeDesc expression) { // otherwise, we didn't understand it, so mark it maybe } else { builder.literal(SearchArgument.TruthValue.YES_NO_NULL); + partial = true; } } @@ -556,6 +568,11 @@ public static SearchArgument create(Configuration conf, ExprNodeGenericFuncDesc return new ConvertAstToSearchArg(conf, expression).buildSearchArgument(); } + public static ConvertAstToSearchArg.Result createSearchArgument(Configuration conf, ExprNodeGenericFuncDesc expression) { + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, expression); + return new ConvertAstToSearchArg.Result(convertAstToSearchArg.buildSearchArgument(), convertAstToSearchArg.isPartial()); + } + private final static ThreadLocal kryo = new ThreadLocal() { protected Kryo initialValue() { return SerializationUtilities.createNewKryo(); } }; @@ -591,4 +608,22 @@ public static String sargToKryo(SearchArgument sarg) { } } + public static final class Result { + private final SearchArgument sarg; + private final boolean partial; + + Result(SearchArgument sarg, boolean partial) { + this.sarg = sarg; + this.partial = partial; + } + + public SearchArgument getSearchArgument() { + return sarg; + } + + public boolean isPartial() { + return partial; + } + } + } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index b52e9d36c794..e05ecfb5a50b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.StorageFormatDescriptor; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; import org.apache.hadoop.hive.ql.parse.StorageFormat.StorageHandlerTypes; @@ -723,4 +724,8 @@ default ColumnInfo getColumnInfo(org.apache.hadoop.hive.ql.metadata.Table hmsTab "for a specific column."); } + default boolean canPerformMetadataDelete(org.apache.hadoop.hive.ql.metadata.Table hmsTable, String branchName, + SearchArgument searchArgument) { + return false; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java index 5102959f087b..b3c8edd9d876 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.parse; import com.google.common.base.MoreObjects; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import java.util.Arrays; @@ -38,7 +39,8 @@ public enum ExecuteOperationType { EXPIRE_SNAPSHOT, SET_CURRENT_SNAPSHOT, FAST_FORWARD, - CHERRY_PICK; + CHERRY_PICK, + DELETE_METADATA; } private final ExecuteOperationType operationType; @@ -234,4 +236,22 @@ public String toString() { return MoreObjects.toStringHelper(this).add("snapshotId", snapshotId).toString(); } } + + public static class DeleteMetadataSpec { + private final String branchName; + private final SearchArgument sarg; + + public DeleteMetadataSpec(String branchName, SearchArgument sarg) { + this.branchName = branchName; + this.sarg = sarg; + } + + public String getBranchName() { + return branchName; + } + + public SearchArgument getSarg() { + return sarg; + } + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java index 3d8050217821..9ed1cc2db034 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java @@ -17,22 +17,31 @@ */ package org.apache.hadoop.hive.ql.parse; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.ddl.DDLWork; +import org.apache.hadoop.hive.ql.ddl.table.execute.AlterTableExecuteDesc; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.ParseUtils.ReparseResult; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; /** * A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles @@ -61,7 +70,9 @@ protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws Se switch (tree.getToken().getType()) { case HiveParser.TOK_DELETE_FROM: operation = Context.Operation.DELETE; - reparseAndSuperAnalyze(tree, table, tabNameNode); + if (!tryMetadataUpdate(tree, table, tabNameNode)) { + reparseAndSuperAnalyze(tree, table, tabNameNode); + } break; case HiveParser.TOK_UPDATE_TABLE: boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); @@ -99,7 +110,7 @@ protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws Se private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException { List children = tree.getChildren(); - boolean shouldTruncate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_REPLACE_DELETE_WITH_TRUNCATE) + boolean shouldTruncate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_REPLACE_DELETE_WITH_TRUNCATE) && children.size() == 1 && deleting(); if (shouldTruncate) { StringBuilder rewrittenQueryStr = new StringBuilder("truncate ").append(getFullTableNameForSQL(tabNameNode)); @@ -164,15 +175,15 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN rewrittenQueryStr.append(" from "); rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); - + ASTNode where = null; int whereIndex = deleting() ? 1 : 2; - + if (children.size() > whereIndex) { where = (ASTNode)children.get(whereIndex); assert where.getToken().getType() == HiveParser.TOK_WHERE : "Expected where clause, but found " + where.getName(); - + if (copyOnWriteMode) { String whereClause = ctx.getTokenRewriteStream().toString( where.getChild(0).getTokenStartIndex(), where.getChild(0).getTokenStopIndex()); @@ -213,11 +224,11 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN withQueryStr.append(") q"); withQueryStr.append("\n").append(INDENT); withQueryStr.append("where rn=1\n)\n"); - + rewrittenQueryStr.insert(0, withQueryStr.toString()); } } - + if (!copyOnWriteMode) { // Add a sort by clause so that the row ids come out in the correct order appendSortBy(rewrittenQueryStr, columnAppender.getSortKeys()); @@ -230,7 +241,7 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN new ASTSearcher().simpleBreadthFirstSearch(rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_UNIONALL).getChild(0).getChild(0) : rewrittenTree) .getChild(1); - + if (updating()) { rewrittenCtx.setOperation(Context.Operation.UPDATE); rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); @@ -295,6 +306,65 @@ private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameN } } + private boolean tryMetadataUpdate(ASTNode tree, Table table, ASTNode tabNameNode) throws SemanticException { + // A feature flag on Hive to perform metadata delete on the source table. + if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_DELETE)) { + return false; + } + TableName tableName = getQualifiedTableName(tabNameNode); + if (!deleting() || table.getStorageHandler() == null) { + return false; + } + int whereIndex = 1; + List children = tree.getChildren(); + if (children.size() <= whereIndex) { + return false; + } + ASTNode whereNode = (ASTNode) children.get(whereIndex); + String whereClause = ctx.getTokenRewriteStream().toString( + whereNode.getChild(0).getTokenStartIndex(), whereNode.getChild(0).getTokenStopIndex()); + StringBuilder sb = new StringBuilder("select * from ").append(getFullTableNameForSQL(tabNameNode)) + .append(" where ").append(whereClause); + Context context = new Context(conf); + ASTNode rewrittenTree; + try { + rewrittenTree = ParseUtils.parse(sb.toString(), context); + } catch (ParseException pe) { + throw new SemanticException(pe); + } + BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, rewrittenTree); + sem.analyze(rewrittenTree, context); + + Map topOps = sem.getParseContext().getTopOps(); + if (!topOps.containsKey(table.getTableName())) { + return false; + } + ExprNodeGenericFuncDesc hiveFilter = topOps.get(table.getTableName()).getConf().getFilterExpr(); + if (hiveFilter == null) { + return false; + } + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(ctx.getConf(), hiveFilter); + if (result.isPartial()) { + return false; + } + SearchArgument sarg = result.getSearchArgument(); + if (!table.getStorageHandler().canPerformMetadataDelete(table, tableName.getTableMetaRef(), sarg)) { + return false; + } + + AlterTableExecuteSpec.DeleteMetadataSpec deleteMetadataSpec = + new AlterTableExecuteSpec.DeleteMetadataSpec(tableName.getTableMetaRef(), sarg); + AlterTableExecuteSpec executeSpec = + new AlterTableExecuteSpec<>(AlterTableExecuteSpec.ExecuteOperationType.DELETE_METADATA, deleteMetadataSpec); + AlterTableExecuteDesc desc = new AlterTableExecuteDesc(tableName, null, executeSpec); + DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), desc); + rootTasks = Collections.singletonList(TaskFactory.get(ddlWork)); + inputs = sem.getInputs(); + outputs = sem.getOutputs(); + updateOutputs(table); + return true; + } + private boolean updating() { return operation == Context.Operation.UPDATE; } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java index d02c57fd1d24..f2e13c53a3ed 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.io.sarg; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -555,8 +556,10 @@ public void testExpression1() throws Exception { " \n" + " \n"; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertFalse(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(9, leaves.size()); @@ -848,8 +851,10 @@ public void testExpression2() throws Exception { " \n" + " \n"; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertFalse(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(4, leaves.size()); @@ -1285,8 +1290,10 @@ public void testExpression3() throws Exception { " \n" + " \n"; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertTrue(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(3, leaves.size()); @@ -1513,8 +1520,10 @@ id in (34,50) */ " \n" + "\n"; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertFalse(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(3, leaves.size()); @@ -1787,8 +1796,10 @@ public void testExpression5() throws Exception { " \n" + " \n"; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertTrue(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(1, leaves.size()); @@ -2275,8 +2286,10 @@ public void testExpression7() throws Exception { " \n" + ""; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertFalse(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(9, leaves.size()); @@ -2438,8 +2451,10 @@ public void testExpression8() throws Exception { " \n" + " "; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertTrue(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(0, leaves.size()); @@ -2575,8 +2590,10 @@ public void testExpression9() throws Exception { " \n" + " "; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertFalse(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(0, leaves.size()); @@ -2700,8 +2717,10 @@ public void testExpression10() throws Exception { " \n" + ""; + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(conf, getFuncDesc(exprStr)); + assertFalse(result.isPartial()); SearchArgumentImpl sarg = - (SearchArgumentImpl) ConvertAstToSearchArg.create(conf, getFuncDesc(exprStr)); + (SearchArgumentImpl) result.getSearchArgument(); List leaves = sarg.getLeaves(); assertEquals(1, leaves.size()); @@ -2745,9 +2764,9 @@ public void testTimestampSarg() throws Exception { TypeInfoFactory.timestampTypeInfo, "ts", Timestamp.ofEpochMilli(1426595696000L)); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2761,9 +2780,9 @@ public void testDateSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.dateTypeInfo, "dt", "2015-05-05"); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2777,9 +2796,9 @@ public void testDecimalSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.decimalTypeInfo, "dec", 123); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2793,9 +2812,9 @@ public void testCharSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.charTypeInfo, "ch", "char "); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2809,9 +2828,9 @@ public void testVarcharSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.varcharTypeInfo, "vc", "variable"); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument();; assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2825,9 +2844,9 @@ public void testBigintSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.intTypeInfo, "bi", 12345); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2856,9 +2875,9 @@ public void testBooleanSarg() throws Exception { new GenericUDFOPAnd(), children); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("(and leaf-0 leaf-1)", sarg.getExpression().toOldString()); assertEquals(2, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2875,9 +2894,9 @@ public void testFloatSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.floatTypeInfo, "flt", 1.1f); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); @@ -2891,9 +2910,9 @@ public void testDoubleSarg() throws Exception { getColumnEqualsConstantExpression(TypeInfoFactory.doubleTypeInfo, "dbl", 2.2); String serialAst = SerializationUtilities.serializeExpression(node); - SearchArgument sarg = - new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)) - .buildSearchArgument(); + ConvertAstToSearchArg convertAstToSearchArg = new ConvertAstToSearchArg(conf, SerializationUtilities.deserializeExpression(serialAst)); + assertFalse(convertAstToSearchArg.isPartial()); + SearchArgument sarg = convertAstToSearchArg.buildSearchArgument(); assertEquals("leaf-0", sarg.getExpression().toOldString()); assertEquals(1, sarg.getLeaves().size()); PredicateLeaf leaf = sarg.getLeaves().get(0); From 4cbd9579c6cb30a3e0dbe6f86464202d6b935b78 Mon Sep 17 00:00:00 2001 From: Steve Carlin Date: Sun, 8 Oct 2023 16:50:03 -0700 Subject: [PATCH 026/179] HIVE-27777: CBO fails on multi insert overwrites with common group expression (Steve Carlin reviewed by Stamatis Zampetakis) The following statement is failing at compilation time when CBO is enabled. FROM (select key, f1 FROM tbl1 where key=5) a INSERT OVERWRITE TABLE tbl2 partition(key=5) select f1 WHERE key > 0 GROUP by f1 INSERT OVERWRITE TABLE tbl2 partition(key=6) select f1 WHERE key > 0 GROUP by f1; The failure happens when there is a filter to a constant value in the FROM clause, the value is referenced in the filter in the INSERT OVERWRITE, and there is a common group existing across the insert overwrites. CBO is pulling up the key = 5 expression into the select clause as a constant (i.e. select 5 key, f1 FROM tbl1 where key = 5). After it gets converted back into AST and then re-compiled, there is code in the common group method that expects all columns to be non-constants which is causing the failiure. Close apache/hive#4783 --- .../hive/ql/parse/SemanticAnalyzer.java | 2 +- .../hive/ql/plan/ExprNodeConstantDesc.java | 9 + .../clientpositive/multi_insert_gby5.q | 6 + .../llap/multi_insert_gby5.q.out | 207 ++++++++++++++++++ 4 files changed, 223 insertions(+), 1 deletion(-) create mode 100644 ql/src/test/queries/clientpositive/multi_insert_gby5.q create mode 100644 ql/src/test/results/clientpositive/llap/multi_insert_gby5.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 5be52904b5fd..250f7c2fcbc9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -6097,7 +6097,7 @@ private ReduceSinkOperator genCommonGroupByPlanReduceSinkOperator(QB qb, List entry : nodeOutputs.entrySet()) { ASTNode parameter = entry.getKey(); ExprNodeDesc expression = entry.getValue(); - if (!(expression instanceof ExprNodeColumnDesc)) { + if (!(expression instanceof ExprNodeColumnDesc) && !ExprNodeConstantDesc.isFoldedFromCol(expression)) { continue; } if (ExprNodeDescUtils.indexOf(expression, reduceValues) >= 0) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java index 268aa1a2faa3..f5e3828e2cda 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.common.StringInternUtils; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; @@ -213,6 +214,14 @@ public boolean isSame(Object o) { return true; } + public static boolean isFoldedFromCol(ExprNodeDesc expr) { + if (!(expr instanceof ExprNodeConstantDesc)) { + return false; + } + ExprNodeConstantDesc constantExpr = (ExprNodeConstantDesc) expr; + return StringUtils.isNotEmpty(constantExpr.foldedFromCol); + } + @Override public int hashCode() { int superHashCode = super.hashCode(); diff --git a/ql/src/test/queries/clientpositive/multi_insert_gby5.q b/ql/src/test/queries/clientpositive/multi_insert_gby5.q new file mode 100644 index 000000000000..be856d7669b1 --- /dev/null +++ b/ql/src/test/queries/clientpositive/multi_insert_gby5.q @@ -0,0 +1,6 @@ +CREATE TABLE target1 (tc int); +CREATE TABLE target2 (tc int); + +EXPLAIN FROM (SELECT 100 as sa, 200 as sb) source +INSERT OVERWRITE TABLE target1 SELECT sa WHERE sb > 0 GROUP BY sa +INSERT OVERWRITE TABLE target2 SELECT sa GROUP BY sa; diff --git a/ql/src/test/results/clientpositive/llap/multi_insert_gby5.q.out b/ql/src/test/results/clientpositive/llap/multi_insert_gby5.q.out new file mode 100644 index 000000000000..1345395e0ee9 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/multi_insert_gby5.q.out @@ -0,0 +1,207 @@ +PREHOOK: query: CREATE TABLE target1 (tc int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@target1 +POSTHOOK: query: CREATE TABLE target1 (tc int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@target1 +PREHOOK: query: CREATE TABLE target2 (tc int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@target2 +POSTHOOK: query: CREATE TABLE target2 (tc int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@target2 +PREHOOK: query: EXPLAIN FROM (SELECT 100 as sa, 200 as sb) source +INSERT OVERWRITE TABLE target1 SELECT sa WHERE sb > 0 GROUP BY sa +INSERT OVERWRITE TABLE target2 SELECT sa GROUP BY sa +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@target1 +PREHOOK: Output: default@target2 +POSTHOOK: query: EXPLAIN FROM (SELECT 100 as sa, 200 as sb) source +INSERT OVERWRITE TABLE target1 SELECT sa WHERE sb > 0 GROUP BY sa +INSERT OVERWRITE TABLE target2 SELECT sa GROUP BY sa +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@target1 +POSTHOOK: Output: default@target2 +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-3 + Stage-4 depends on stages: Stage-0 + Stage-1 depends on stages: Stage-3 + Stage-5 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: _dummy_table + Row Limit Per Split: 1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: 100 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: 100 (type: int) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: 200 (type: int) + Execution mode: llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Forward + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (VALUE._col0 > 0) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: KEY._col0 (type: int) + mode: complete + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.target1 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: tc + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(tc), max(tc), count(1), count(tc), compute_bit_vector_hll(tc) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary) + Group By Operator + keys: KEY._col0 (type: int) + mode: complete + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.target2 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: tc + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(tc), max(tc), count(1), count(tc), compute_bit_vector_hll(tc) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'LONG' (type: string), UDFToLong(_col0) (type: bigint), UDFToLong(_col1) (type: bigint), (_col2 - _col3) (type: bigint), COALESCE(ndv_compute_bit_vector(_col4),0) (type: bigint), _col4 (type: binary) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'LONG' (type: string), UDFToLong(_col0) (type: bigint), UDFToLong(_col1) (type: bigint), (_col2 - _col3) (type: bigint), COALESCE(ndv_compute_bit_vector(_col4),0) (type: bigint), _col4 (type: binary) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-3 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.target1 + + Stage: Stage-4 + Stats Work + Basic Stats Work: + Column Stats Desc: + Columns: tc + Column Types: int + Table: default.target1 + + Stage: Stage-1 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.target2 + + Stage: Stage-5 + Stats Work + Basic Stats Work: + Column Stats Desc: + Columns: tc + Column Types: int + Table: default.target2 + From 089e2727cb9b2855d9c1ad2b7a789b7afb8fc5a5 Mon Sep 17 00:00:00 2001 From: rkirti Date: Wed, 11 Oct 2023 13:40:47 +0530 Subject: [PATCH 027/179] HIVE-27745: Add unit test to ensure short version is inline with full version (Kirti Ruge reviewed by Stamatis Zampetakis) Close apache/hive#4791 --- .../utils/TestMetastoreVersionInfo.java | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetastoreVersionInfo.java diff --git a/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetastoreVersionInfo.java b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetastoreVersionInfo.java new file mode 100644 index 000000000000..e256befa62b1 --- /dev/null +++ b/standalone-metastore/metastore-common/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetastoreVersionInfo.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.utils; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestMetastoreVersionInfo { + + private String hiveShortVersion; + private String hiveVersion; + + @Before + public void setUp() throws Exception { + hiveShortVersion = MetastoreVersionInfo.getShortVersion(); + hiveVersion = MetastoreVersionInfo.getVersion(); + } + + @Test + public void testValidateHiveShortVersionWithHiveVersion() { + Assert.assertEquals(hiveVersion.replace("-SNAPSHOT", ""), hiveShortVersion); + } + + @Test + public void testIfHiveVersionHasShortVersionAsPrefix() { + Assert.assertTrue(hiveVersion.startsWith(hiveShortVersion)); + } +} + From 0233dcc7f1f09198c093cb4b69bd2b2598c97303 Mon Sep 17 00:00:00 2001 From: Devaspati Date: Fri, 27 Oct 2023 11:07:42 +0530 Subject: [PATCH 028/179] HIVE-27824: Upgrade Ivy to 2.5.2 (#4828). (Devaspati Krishnatri , Reviewed by Ayush Saxena, Naveen Gangam) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f76569deb4c3..5e322a86a820 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 4.5.13 4.4.13 - 2.5.1 + 2.5.2 2.13.5 2.3.4 2.4.1 From c9b3a6b026de01252f13eac2ba857bf0e6e68c74 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sun, 29 Oct 2023 15:31:19 +0800 Subject: [PATCH 029/179] HIVE-27826: Upgrade to Parquet 1.13.1 (#4830). (zhangbutao, reviewed by Ayush Saxena) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5e322a86a820..b9b8339351d8 100644 --- a/pom.xml +++ b/pom.xml @@ -191,7 +191,7 @@ 4.5.5 2.8 - 1.13.0 + 1.13.1 0.16.0 1.5.6 3.21.7 From 2e6618efbe5c870873ec06cf8c4d4c9702e31f0c Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Mon, 30 Oct 2023 04:09:33 +0800 Subject: [PATCH 030/179] HIVE-27780: Implement direct SQL for get_all_functions (#4786). (zhangbutao, reviewed by Ayush Saxena) --- .../hive/metastore/MetaStoreDirectSql.java | 105 +++++++++++++++++- .../metastore/MetastoreDirectSqlUtils.java | 19 +++- .../hadoop/hive/metastore/ObjectStore.java | 27 +++++ .../hive/metastore/client/TestFunctions.java | 8 ++ 4 files changed, 155 insertions(+), 4 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java index 92865954bcdf..6f04fd03720d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java @@ -68,6 +68,8 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.DatabaseType; import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.FunctionType; import org.apache.hadoop.hive.metastore.api.GetPartitionsFilterSpec; import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; import org.apache.hadoop.hive.metastore.api.HiveObjectRef; @@ -92,6 +94,7 @@ import org.apache.hadoop.hive.metastore.model.MConstraint; import org.apache.hadoop.hive.metastore.model.MCreationMetadata; import org.apache.hadoop.hive.metastore.model.MDatabase; +import org.apache.hadoop.hive.metastore.model.MFunction; import org.apache.hadoop.hive.metastore.model.MNotificationLog; import org.apache.hadoop.hive.metastore.model.MNotificationNextId; import org.apache.hadoop.hive.metastore.model.MPartition; @@ -182,7 +185,7 @@ public static String getIdListForIn(Collection objectIds) throws MetaExce SDS, SERDES, SKEWED_STRING_LIST_VALUES, SKEWED_VALUES, BUCKETING_COLS, SKEWED_COL_NAMES, SKEWED_COL_VALUE_LOC_MAP, COLUMNS_V2, PARTITION_KEYS, SERDE_PARAMS, PART_COL_STATS, KEY_CONSTRAINTS, TAB_COL_STATS, PARTITION_KEY_VALS, PART_PRIVS, PART_COL_PRIVS, SKEWED_STRING_LIST, CDS, - TBL_COL_PRIVS; + TBL_COL_PRIVS, FUNCS, FUNC_RU; public MetaStoreDirectSql(PersistenceManager pm, Configuration conf, String schema) { this.pm = pm; @@ -3064,4 +3067,104 @@ public Map> updatePartitionColumnStatisticsBatch( long csId = updateStat.getNextCSIdForMPartitionColumnStatistics(numStats); return updateStat.updatePartitionColumnStatistics(partColStatsMap, tbl, csId, validWriteIds, writeId, listeners); } + + public List getFunctions(String catName) throws MetaException { + List funcIds = getFunctionIds(catName); + // Get full objects. For Oracle/etc. do it in batches. + return Batchable.runBatched(batchSize, funcIds, new Batchable() { + @Override + public List run(List input) throws MetaException { + return getFunctionsFromFunctionIds(input, catName); + } + }); + } + + private List getFunctionsFromFunctionIds(List funcIdList, String catName) throws MetaException { + String funcIds = getIdListForIn(funcIdList); + final int funcIdIndex = 0; + final int funcNameIndex = 1; + final int dbNameIndex = 2; + final int funcClassNameIndex = 3; + final int funcOwnerNameIndex = 4; + final int funcOwnerTypeIndex = 5; + final int funcCreateTimeIndex = 6; + final int funcTypeIndex = 7; + + String queryText = "SELECT " + + FUNCS + ".\"FUNC_ID\", " + + FUNCS + ".\"FUNC_NAME\", " + + DBS + ".\"NAME\", " + + FUNCS + ".\"CLASS_NAME\", " + + FUNCS + ".\"OWNER_NAME\", " + + FUNCS + ".\"OWNER_TYPE\", " + + FUNCS + ".\"CREATE_TIME\", " + + FUNCS + ".\"FUNC_TYPE\"" + + " FROM " + FUNCS + + " LEFT JOIN " + DBS + " ON " + FUNCS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\"" + + " where " + FUNCS +".\"FUNC_ID\" in (" + funcIds + ") order by " + FUNCS + ".\"FUNC_NAME\" asc"; + + List results = new ArrayList<>(); + TreeMap funcs = new TreeMap<>(); + final boolean doTrace = LOG.isDebugEnabled(); + long start = doTrace ? System.nanoTime() : 0; + try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { + List queryResult = executeWithArray(query.getInnerQuery(), null, queryText); + long end = doTrace ? System.nanoTime() : 0; + MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end); + + for (Object[] function : queryResult) { + Long funcId = MetastoreDirectSqlUtils.extractSqlLong(function[funcIdIndex]); + String funcName = MetastoreDirectSqlUtils.extractSqlString(function[funcNameIndex]); + String dbName = MetastoreDirectSqlUtils.extractSqlString(function[dbNameIndex]); + String funcClassName = MetastoreDirectSqlUtils.extractSqlString(function[funcClassNameIndex]); + String funcOwnerName = MetastoreDirectSqlUtils.extractSqlString(function[funcOwnerNameIndex]); + String funcOwnerType = MetastoreDirectSqlUtils.extractSqlString(function[funcOwnerTypeIndex]); + int funcCreateTime = MetastoreDirectSqlUtils.extractSqlInt(function[funcCreateTimeIndex]); + int funcType = MetastoreDirectSqlUtils.extractSqlInt(function[funcTypeIndex]); + + Function func = new Function(); + func.setFunctionName(funcName); + func.setDbName(dbName); + func.setCatName(catName); + func.setClassName(funcClassName); + func.setOwnerName(funcOwnerName); + func.setOwnerType(PrincipalType.valueOf(funcOwnerType)); + func.setCreateTime(funcCreateTime); + func.setFunctionType(FunctionType.findByValue(funcType)); + func.setResourceUris(new ArrayList<>()); + + results.add(func); + funcs.put(funcId, func); + } + } + MetastoreDirectSqlUtils.setFunctionResourceUris(FUNC_RU, pm, funcIds, funcs); + return results; + } + + private List getFunctionIds(String catName) throws MetaException { + boolean doTrace = LOG.isDebugEnabled(); + + String queryText = "select " + FUNCS + ".\"FUNC_ID\" from " + FUNCS + + " LEFT JOIN " + DBS + " ON " + FUNCS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\"" + + " where " + DBS + ".\"CTLG_NAME\" = ? "; + + long start = doTrace ? System.nanoTime() : 0; + Object[] params = new Object[1]; + params[0] = catName; + try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { + List sqlResult = executeWithArray(query.getInnerQuery(), params, queryText); + long queryTime = doTrace ? System.nanoTime() : 0; + MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, queryTime); + final List result; + if (sqlResult.isEmpty()) { + result = Collections.emptyList(); + } else { + result = new ArrayList<>(sqlResult.size()); + for (Object fields : sqlResult) { + result.add(MetastoreDirectSqlUtils.extractSqlLong(fields)); + } + } + return result; + } + } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java index e26ea2ee1a8a..b30999773128 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java @@ -19,17 +19,18 @@ package org.apache.hadoop.hive.metastore; -import com.google.common.base.Joiner; import org.apache.commons.lang3.BooleanUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.ResourceType; +import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.SkewedInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; -import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +47,6 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.stream.Collectors; /** * Helper utilities used by DirectSQL code in HiveMetastore. @@ -514,6 +514,19 @@ public void apply(SerDeInfo t, Object[] fields) { } } + static void setFunctionResourceUris(String FUNC_RU, PersistenceManager pm, String funcIds, + TreeMap functions) + throws MetaException { + String queryText; + queryText = "select \"FUNC_ID\", \"RESOURCE_TYPE\", \"RESOURCE_URI\" from " + FUNC_RU + + " where \"FUNC_ID\" in (" + funcIds + ")" + + " order by \"FUNC_ID\" asc, \"INTEGER_IDX\" asc"; + loopJoinOrderedResult(pm, functions, queryText, 0, (t, fields) -> { + ResourceUri resourceUri = new ResourceUri(ResourceType.findByValue((int)fields[1]), (String) fields[2]); + t.getResourceUris().add(resourceUri); + }); + } + /** * Convert a boolean value returned from the RDBMS to a Java Boolean object. * MySQL has booleans, but e.g. Derby uses 'Y'/'N' mapping and Oracle DB diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index e8996cb24983..b3e8bbb82c24 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -11541,6 +11541,33 @@ public Function getFunction(String catName, String dbName, String funcName) thro @Override public List getAllFunctions(String catName) throws MetaException { + try { + return getFunctionsInternal(catName); + } catch (NoSuchObjectException e) { + throw new RuntimeException(e); + } + } + + protected List getFunctionsInternal(String catalogName) + throws MetaException, NoSuchObjectException { + return new GetListHelper(catalogName, "", "", true, true) { + @Override + protected List getSqlResult(GetHelper> ctx) throws MetaException { + return directSql.getFunctions(catalogName); + } + @Override + protected List getJdoResult(GetHelper> ctx) throws MetaException { + try { + return getAllFunctionsViaJDO(catalogName); + } catch (Exception e) { + LOG.error("Failed to convert to functions", e); + throw new MetaException(e.getMessage()); + } + } + }.run(false); + } + + private List getAllFunctionsViaJDO (String catName) { boolean commited = false; Query query = null; try { diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java index aafbdd1bf51b..efa57840f1c4 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java @@ -377,12 +377,20 @@ public void testGetAllFunctions() throws Exception { for(Function function : allFunctions) { if (function.getDbName().equals(OTHER_DATABASE)) { Assert.assertEquals("Comparing functions", testFunctions[3], function); + Assert.assertEquals("Checking function's resourceUris", + testFunctions[3].getResourceUris().toString(), function.getResourceUris().toString()); } else if (function.getFunctionName().equals("test_function_hidden_1")) { Assert.assertEquals("Comparing functions", testFunctions[2], function); + Assert.assertEquals("Checking function's resourceUris", + testFunctions[2].getResourceUris().toString(), function.getResourceUris().toString()); } else if (function.getFunctionName().equals("test_function_to_find_2")) { Assert.assertEquals("Comparing functions", testFunctions[1], function); + Assert.assertEquals("Checking function's resourceUris", + testFunctions[1].getResourceUris().toString(), function.getResourceUris().toString()); } else { Assert.assertEquals("Comparing functions", testFunctions[0], function); + Assert.assertEquals("Checking function's resourceUris", + testFunctions[0].getResourceUris().toString(), function.getResourceUris().toString()); } } From 52011ce2aff57771f90345c6292da683836799e9 Mon Sep 17 00:00:00 2001 From: okumin Date: Mon, 30 Oct 2023 13:07:39 +0900 Subject: [PATCH 031/179] HIVE-27825: Better error message for an empty quoted identifier (#4829). (okumin, reviewed by Ayush Saxena) --- parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g | 2 +- .../java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g | 4 ++-- .../clientnegative/empty_quoted_identifier_column_name.q | 1 + .../empty_quoted_identifier_column_name_standard_backtick.q | 2 ++ ...mpty_quoted_identifier_column_name_standard_double_quote.q | 2 ++ .../clientnegative/empty_quoted_identifier_table_name.q | 1 + .../empty_quoted_identifier_table_name_standard_backtick.q | 2 ++ ...empty_quoted_identifier_table_name_standard_double_quote.q | 2 ++ .../clientnegative/empty_quoted_identifier_column_name.q.out | 1 + ...mpty_quoted_identifier_column_name_standard_backtick.q.out | 1 + ..._quoted_identifier_column_name_standard_double_quote.q.out | 1 + .../clientnegative/empty_quoted_identifier_table_name.q.out | 1 + ...empty_quoted_identifier_table_name_standard_backtick.q.out | 1 + ...y_quoted_identifier_table_name_standard_double_quote.q.out | 1 + 14 files changed, 19 insertions(+), 3 deletions(-) create mode 100644 ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name.q create mode 100644 ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q create mode 100644 ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q create mode 100644 ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name.q create mode 100644 ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q create mode 100644 ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q create mode 100644 ql/src/test/results/clientnegative/empty_quoted_identifier_column_name.q.out create mode 100644 ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q.out create mode 100644 ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q.out create mode 100644 ql/src/test/results/clientnegative/empty_quoted_identifier_table_name.q.out create mode 100644 ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q.out create mode 100644 ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q.out diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g index fff0fdade053..f3497d9edee7 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g @@ -35,5 +35,5 @@ StringLiteral fragment QuotedIdentifier : - ('`' ( '``' | ~('`') )* '`') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); } + ('`' ( '``' | ~('`') )+ '`') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); } ; diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g index 466b20014b69..b1b1e4aa65d0 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerStandard.g @@ -35,6 +35,6 @@ StringLiteral fragment QuotedIdentifier : - ('"' ( '""' | ~('"') )* '"') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "\"\"", "\"")); } - | ('`' ( '``' | ~('`') )* '`') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); } + ('"' ( '""' | ~('"') )+ '"') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "\"\"", "\"")); } + | ('`' ( '``' | ~('`') )+ '`') { setText(StringUtils.replace(getText().substring(1, getText().length() -1 ), "``", "`")); } ; diff --git a/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name.q b/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name.q new file mode 100644 index 000000000000..8dccc381acf2 --- /dev/null +++ b/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name.q @@ -0,0 +1 @@ +select `` from `test_database`.`test_table` diff --git a/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q b/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q new file mode 100644 index 000000000000..7c4ede22cc4a --- /dev/null +++ b/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q @@ -0,0 +1,2 @@ +set hive.support.quoted.identifiers=standard; +select `` from `test_database`.`test_table` diff --git a/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q b/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q new file mode 100644 index 000000000000..f6b66651d96e --- /dev/null +++ b/ql/src/test/queries/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q @@ -0,0 +1,2 @@ +set hive.support.quoted.identifiers=standard; +select "" from "test_database"."test_table" diff --git a/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name.q b/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name.q new file mode 100644 index 000000000000..245892d9bee8 --- /dev/null +++ b/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name.q @@ -0,0 +1 @@ +select * from `test_database`.``; diff --git a/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q b/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q new file mode 100644 index 000000000000..b1ba96fa64a3 --- /dev/null +++ b/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q @@ -0,0 +1,2 @@ +set hive.support.quoted.identifiers=standard; +select * from `test_database`.``; diff --git a/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q b/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q new file mode 100644 index 000000000000..3d5a8a4921f3 --- /dev/null +++ b/ql/src/test/queries/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q @@ -0,0 +1,2 @@ +set hive.support.quoted.identifiers=standard; +select * from "test_database".""; diff --git a/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name.q.out b/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name.q.out new file mode 100644 index 000000000000..0635ad3b29ea --- /dev/null +++ b/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name.q.out @@ -0,0 +1 @@ +FAILED: ParseException line 1:10 cannot recognize input near 'from' 'test_database' '.' in select clause diff --git a/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q.out b/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q.out new file mode 100644 index 000000000000..0da914f6b0a2 --- /dev/null +++ b/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_backtick.q.out @@ -0,0 +1 @@ +FAILED: ParseException line 2:10 cannot recognize input near 'from' 'test_database' '.' in select clause diff --git a/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q.out b/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q.out new file mode 100644 index 000000000000..0da914f6b0a2 --- /dev/null +++ b/ql/src/test/results/clientnegative/empty_quoted_identifier_column_name_standard_double_quote.q.out @@ -0,0 +1 @@ +FAILED: ParseException line 2:10 cannot recognize input near 'from' 'test_database' '.' in select clause diff --git a/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name.q.out b/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name.q.out new file mode 100644 index 000000000000..33f9b4b16922 --- /dev/null +++ b/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name.q.out @@ -0,0 +1 @@ +FAILED: ParseException line 1:32 cannot recognize input near 'test_database' '.' '' in table name diff --git a/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q.out b/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q.out new file mode 100644 index 000000000000..b9c938633b0d --- /dev/null +++ b/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_backtick.q.out @@ -0,0 +1 @@ +FAILED: ParseException line 2:32 cannot recognize input near 'test_database' '.' '' in table name diff --git a/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q.out b/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q.out new file mode 100644 index 000000000000..b9c938633b0d --- /dev/null +++ b/ql/src/test/results/clientnegative/empty_quoted_identifier_table_name_standard_double_quote.q.out @@ -0,0 +1 @@ +FAILED: ParseException line 2:32 cannot recognize input near 'test_database' '.' '' in table name From 40551ff4669325defa08ff2adc0b497296cf46fa Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Mon, 30 Oct 2023 12:22:34 +0800 Subject: [PATCH 032/179] HIVE-27651: Upgrade hbase version (#4649). (zhangbutao, reviewed by Ayush Saxena) --- hbase-handler/pom.xml | 7 +++++++ .../apache/hadoop/hive/hbase/TestHBaseStorageHandler.java | 2 +- .../src/test/results/negative/cascade_dbdrop.q.out | 3 +-- itests/hcatalog-unit/pom.xml | 5 +++++ itests/pom.xml | 6 ++++++ itests/util/pom.xml | 5 +++++ pom.xml | 2 +- 7 files changed, 26 insertions(+), 4 deletions(-) diff --git a/hbase-handler/pom.xml b/hbase-handler/pom.xml index 3a908a3bd03b..e0a4c8de19a0 100644 --- a/hbase-handler/pom.xml +++ b/hbase-handler/pom.xml @@ -184,6 +184,13 @@ + + org.apache.hbase + hbase-testing-util + ${hbase.version} + tests + test + org.apache.hbase hbase-mapreduce diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseStorageHandler.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseStorageHandler.java index 25ddd3219f9a..535d13860414 100644 --- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseStorageHandler.java +++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseStorageHandler.java @@ -61,7 +61,7 @@ public void testGetUriForAuthEmptyTableDefaultHostPort() throws URISyntaxExcepti // written out. At the time this test was written, this was the current // behavior, so I left this test as/is. Need to research if a null // table can be provided here. - Assert.assertEquals("hbase://localhost:2181/null", uri.toString()); + Assert.assertEquals("hbase://127.0.0.1:2181/null", uri.toString()); } @Test diff --git a/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out b/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out index 9f3d2c01a894..589efc6ab3c9 100644 --- a/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out +++ b/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out @@ -18,8 +18,7 @@ TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "t POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:hbasedb POSTHOOK: Output: hbaseDB@hbase_table_0 -Found 3 items -drwxr-xr-x - ### USER ### ### GROUP ### 0 ### HDFS DATE ### hdfs://### HDFS PATH ### +Found 2 items drwxr-xr-x - ### USER ### ### GROUP ### 0 ### HDFS DATE ### hdfs://### HDFS PATH ### drwxr-xr-x - ### USER ### ### GROUP ### 0 ### HDFS DATE ### hdfs://### HDFS PATH ### PREHOOK: query: DROP DATABASE IF EXISTS hbaseDB CASCADE diff --git a/itests/hcatalog-unit/pom.xml b/itests/hcatalog-unit/pom.xml index 4a049697b712..3f47452b7e32 100644 --- a/itests/hcatalog-unit/pom.xml +++ b/itests/hcatalog-unit/pom.xml @@ -71,6 +71,11 @@ hive-hbase-handler test + + org.apache.hbase + hbase-testing-util + tests + org.apache.hive hive-cli diff --git a/itests/pom.xml b/itests/pom.xml index f23bd28b97a2..6f2cdad408d4 100644 --- a/itests/pom.xml +++ b/itests/pom.xml @@ -447,6 +447,12 @@ ${hbase.version} tests + + org.apache.hbase + hbase-testing-util + ${hbase.version} + tests + org.apache.tez tez-tests diff --git a/itests/util/pom.xml b/itests/util/pom.xml index 7f0e25a31fc0..97ebb9c83aa2 100644 --- a/itests/util/pom.xml +++ b/itests/util/pom.xml @@ -219,6 +219,11 @@ hbase-server tests + + org.apache.hbase + hbase-testing-util + tests + org.apache.hbase hbase-mapreduce diff --git a/pom.xml b/pom.xml index b9b8339351d8..bf8a5d3e5fd9 100644 --- a/pom.xml +++ b/pom.xml @@ -143,7 +143,7 @@ 3.3.6 ${basedir}/${hive.path.to.root}/testutils/hadoop 1.3 - 2.0.0-alpha4 + 2.5.6-hadoop3 0.7.2 3.3.7 From d2d4b86433ef356bf738edd72bfef74511a15444 Mon Sep 17 00:00:00 2001 From: tarak271 Date: Tue, 31 Oct 2023 15:08:20 +0530 Subject: [PATCH 033/179] HIVE-27598 - Enhance alter table compact to work for partitioned tables (Taraka Rama Rao Lethavadla, reviewed by Laszlo Vegh) --- .../compact/AlterTableCompactOperation.java | 130 +++--- .../hive/ql/txn/compactor/Initiator.java | 249 +---------- .../hive/ql/txn/compactor/InitiatorBase.java | 309 ++++++++++++++ .../clientpositive/manual_compaction.q | 68 +++ .../llap/manual_compaction.q.out | 387 ++++++++++++++++++ 5 files changed, 843 insertions(+), 300 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java create mode 100644 ql/src/test/queries/clientpositive/manual_compaction.q create mode 100644 ql/src/test/results/clientpositive/llap/manual_compaction.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java index 9187101367b2..cc896331aff7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java @@ -18,27 +18,27 @@ package org.apache.hadoop.hive.ql.ddl.table.storage.compact; -import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.CompactionRequest; -import org.apache.hadoop.hive.metastore.utils.JavaUtils; -import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; -import org.apache.hadoop.hive.ql.io.AcidUtils; - -import java.util.List; -import java.util.Map; +import org.apache.hadoop.hive.metastore.api.CompactionRequest; import org.apache.hadoop.hive.metastore.api.CompactionResponse; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.utils.JavaUtils; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.ddl.DDLOperation; +import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; +import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.txn.compactor.InitiatorBase; + +import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.hadoop.hive.ql.io.AcidUtils.compactionTypeStr2ThriftType; @@ -50,80 +50,95 @@ public AlterTableCompactOperation(DDLOperationContext context, AlterTableCompact super(context, desc); } - @Override - public int execute() throws HiveException { + @Override public int execute() throws Exception { Table table = context.getDb().getTable(desc.getTableName()); if (!AcidUtils.isTransactionalTable(table)) { throw new HiveException(ErrorMsg.NONACID_COMPACTION_NOT_SUPPORTED, table.getDbName(), table.getTableName()); } - String partitionName = getPartitionName(table); + CompactionRequest compactionRequest = new CompactionRequest(table.getDbName(), table.getTableName(), + compactionTypeStr2ThriftType(desc.getCompactionType())); - CompactionResponse resp = compact(table, partitionName); - if (!resp.isAccepted()) { - String message = Constants.ERROR_MESSAGE_NO_DETAILS_AVAILABLE; - if (resp.isSetErrormessage()) { - message = resp.getErrormessage(); - } - throw new HiveException(ErrorMsg.COMPACTION_REFUSED, - table.getDbName(), table.getTableName(), partitionName == null ? "" : "(partition=" + partitionName + ")", message); - } + compactionRequest.setPoolName(desc.getPoolName()); + compactionRequest.setProperties(desc.getProperties()); + compactionRequest.setInitiatorId(JavaUtils.hostname() + "-" + HiveMetaStoreClient.MANUALLY_INITIATED_COMPACTION); + compactionRequest.setInitiatorVersion(HiveMetaStoreClient.class.getPackage().getImplementationVersion()); + compactionRequest.setOrderByClause(desc.getOrderByClause()); - if (desc.isBlocking() && resp.isAccepted()) { - waitForCompactionToFinish(resp); + if (desc.getNumberOfBuckets() > 0) { + compactionRequest.setNumberOfBuckets(desc.getNumberOfBuckets()); } + InitiatorBase initiatorBase = new InitiatorBase(); + initiatorBase.setConf(context.getConf()); + initiatorBase.init(new AtomicBoolean()); + + Map partitionMap = + convertPartitionsFromThriftToDB(getPartitions(table, desc, context)); + + if(desc.getPartitionSpec() != null){ + Optional partitionName = partitionMap.keySet().stream().findFirst(); + partitionName.ifPresent(compactionRequest::setPartitionname); + } + List compactionResponses = + initiatorBase.initiateCompactionForTable(compactionRequest, table.getTTable(), partitionMap); + for (CompactionResponse compactionResponse : compactionResponses) { + if (!compactionResponse.isAccepted()) { + String message; + if (compactionResponse.isSetErrormessage()) { + message = compactionResponse.getErrormessage(); + throw new HiveException(ErrorMsg.COMPACTION_REFUSED, table.getDbName(), table.getTableName(), + "CompactionId: " + compactionResponse.getId(), message); + } + context.getConsole().printInfo( + "Compaction already enqueued with id " + compactionResponse.getId() + "; State is " + + compactionResponse.getState()); + continue; + } + context.getConsole().printInfo("Compaction enqueued with id " + compactionResponse.getId()); + if (desc.isBlocking() && compactionResponse.isAccepted()) { + waitForCompactionToFinish(compactionResponse, context); + } + } return 0; } - private String getPartitionName(Table table) throws HiveException { - String partitionName = null; + private List getPartitions(Table table, AlterTableCompactDesc desc, DDLOperationContext context) + throws HiveException { + List partitions = new ArrayList<>(); + if (desc.getPartitionSpec() == null) { - if (table.isPartitioned()) { // Compaction can only be done on the whole table if the table is non-partitioned. - throw new HiveException(ErrorMsg.NO_COMPACTION_PARTITION); + if (table.isPartitioned()) { + // Compaction will get initiated for all the potential partitions that meets the criteria + partitions = context.getDb().getPartitions(table); } } else { Map partitionSpec = desc.getPartitionSpec(); - List partitions = context.getDb().getPartitions(table, partitionSpec); + partitions = context.getDb().getPartitions(table, partitionSpec); if (partitions.size() > 1) { throw new HiveException(ErrorMsg.TOO_MANY_COMPACTION_PARTITIONS); } else if (partitions.size() == 0) { throw new HiveException(ErrorMsg.INVALID_PARTITION_SPEC); } - partitionName = partitions.get(0).getName(); } - return partitionName; + return partitions; } - private CompactionResponse compact(Table table, String partitionName) throws HiveException { - CompactionRequest req = new CompactionRequest(table.getDbName(), table.getTableName(), - compactionTypeStr2ThriftType(desc.getCompactionType())); - req.setPartitionname(partitionName); - req.setPoolName(desc.getPoolName()); - req.setProperties(desc.getProperties()); - req.setInitiatorId(JavaUtils.hostname() + "-" + HiveMetaStoreClient.MANUALLY_INITIATED_COMPACTION); - req.setInitiatorVersion(HiveMetaStoreClient.class.getPackage().getImplementationVersion()); - req.setOrderByClause(desc.getOrderByClause()); - if (desc.getNumberOfBuckets() > 0) { - req.setNumberOfBuckets(desc.getNumberOfBuckets()); - } - CompactionResponse resp = context.getDb().compact(req); - if (resp.isAccepted()) { - context.getConsole().printInfo("Compaction enqueued with id " + resp.getId()); - } else { - context.getConsole().printInfo("Compaction already enqueued with id " + resp.getId() + "; State is " + - resp.getState()); - } - return resp; + private Map convertPartitionsFromThriftToDB( + List partitions) { + Map partitionMap = new LinkedHashMap<>(); + partitions.forEach(partition -> partitionMap.put(partition.getName(), partition.getTPartition())); + return partitionMap; } - private void waitForCompactionToFinish(CompactionResponse resp) throws HiveException { + private void waitForCompactionToFinish(CompactionResponse resp, DDLOperationContext context) throws HiveException { StringBuilder progressDots = new StringBuilder(); long waitTimeMs = 1000; long waitTimeOut = HiveConf.getLongVar(context.getConf(), HiveConf.ConfVars.HIVE_COMPACTOR_WAIT_TIMEOUT); - wait: while (true) { + wait: + while (true) { //double wait time until 5min - waitTimeMs = waitTimeMs*2; + waitTimeMs = waitTimeMs * 2; waitTimeMs = Math.min(waitTimeMs, waitTimeOut); try { Thread.sleep(waitTimeMs); @@ -133,10 +148,11 @@ private void waitForCompactionToFinish(CompactionResponse resp) throws HiveExcep } ShowCompactRequest request = new ShowCompactRequest(); request.setId(resp.getId()); - + ShowCompactResponse compaction = context.getDb().showCompactions(request); if (compaction.getCompactsSize() == 1) { ShowCompactResponseElement comp = compaction.getCompacts().get(0); + LOG.debug("Response for cid: "+comp.getId()+" is "+comp.getState()); switch (comp.getState()) { case TxnStore.WORKING_RESPONSE: case TxnStore.INITIATED_RESPONSE: @@ -146,11 +162,11 @@ private void waitForCompactionToFinish(CompactionResponse resp) throws HiveExcep continue wait; default: //done - context.getConsole().printInfo("Compaction with id " + resp.getId() + " finished with status: " + - comp.getState()); + context.getConsole() + .printInfo("Compaction with id " + resp.getId() + " finished with status: " + comp.getState()); break wait; } - }else { + } else { throw new HiveException("No suitable compaction found"); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java index a86c18baad92..bb48c8f219bf 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java @@ -20,33 +20,12 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.ServerUtils; -import org.apache.hadoop.hive.common.ValidReadTxnList; import org.apache.hadoop.hive.common.ValidTxnList; -import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.CompactionRequest; -import org.apache.hadoop.hive.metastore.api.CompactionResponse; -import org.apache.hadoop.hive.metastore.api.CompactionType; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; -import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.metrics.PerfLogger; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; @@ -54,28 +33,14 @@ import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.apache.hadoop.hive.ql.io.AcidDirectory; import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.AcidUtils.ParsedDirectory; -import org.apache.hadoop.hive.shims.HadoopShims.HdfsFileStatusWithId; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.common.util.Ref; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.LongSummaryStatistics; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.ExecutorService; +import java.util.*; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -86,12 +51,10 @@ * A class to initiate compactions. This will run in a separate thread. * It's critical that there exactly 1 of these in a given warehouse. */ -public class Initiator extends MetaStoreCompactorThread { +public class Initiator extends InitiatorBase { static final private String CLASS_NAME = Initiator.class.getName(); static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME); - static final private String COMPACTORTHRESHOLD_PREFIX = "compactorthreshold."; - private ExecutorService compactionExecutor; private boolean metricsEnabled; @@ -178,7 +141,7 @@ public void run() { } Table t = metadataCache.computeIfAbsent(ci.getFullTableName(), () -> resolveTable(ci)); - String poolName = getPoolName(ci, t); + ci.poolName = getPoolName(ci, t); Partition p = resolvePartition(ci); if (p == null && ci.partName != null) { LOG.info("Can't find partition " + ci.getFullPartitionName() + @@ -194,7 +157,7 @@ public void run() { CompletableFuture asyncJob = CompletableFuture.runAsync( CompactorUtil.ThrowingRunnable.unchecked(() -> - scheduleCompactionIfRequired(ci, t, p, poolName, runAs, metricsEnabled)), compactionExecutor) + scheduleCompactionIfRequired(ci, t, p, runAs, metricsEnabled)), compactionExecutor) .exceptionally(exc -> { LOG.error("Error while running scheduling the compaction on the table {} / partition {}.", tableName, partition, exc); return null; @@ -250,36 +213,6 @@ protected boolean isCacheEnabled() { MetastoreConf.ConfVars.COMPACTOR_INITIATOR_TABLECACHE_ON); } - private void scheduleCompactionIfRequired(CompactionInfo ci, Table t, Partition p, String poolName, - String runAs, boolean metricsEnabled) - throws MetaException { - StorageDescriptor sd = resolveStorageDescriptor(t, p); - try { - ValidWriteIdList validWriteIds = resolveValidWriteIds(t); - - checkInterrupt(); - - CompactionType type = checkForCompaction(ci, validWriteIds, sd, t.getParameters(), runAs); - if (type != null) { - ci.type = type; - ci.poolName = poolName; - requestCompaction(ci, runAs); - } - } catch (InterruptedException e) { - //Handle InterruptedException separately so the compactioninfo won't be marked as failed. - LOG.info("Initiator pool is being shut down, task received interruption."); - } catch (Throwable ex) { - String errorMessage = "Caught exception while trying to determine if we should compact " + ci + ". Marking " - + "failed to avoid repeated failures, " + ex; - LOG.error(errorMessage); - ci.errorMessage = errorMessage; - if (metricsEnabled) { - Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_INITIATOR_FAILURE_COUNTER).inc(); - } - txnHandler.markFailed(ci); - } - } - private String getPoolName(CompactionInfo ci, Table t) throws Exception { Map params = t.getParameters(); String poolName = params == null ? null : params.get(Constants.HIVE_COMPACTOR_WORKER_POOL); @@ -294,16 +227,7 @@ private Database resolveDatabase(CompactionInfo ci) throws MetaException, NoSuch return CompactorUtil.resolveDatabase(conf, ci.dbname); } - private ValidWriteIdList resolveValidWriteIds(Table t) throws NoSuchTxnException, MetaException { - ValidTxnList validTxnList = new ValidReadTxnList(conf.get(ValidTxnList.VALID_TXNS_KEY)); - // The response will have one entry per table and hence we get only one ValidWriteIdList - String fullTableName = TxnUtils.getFullTableName(t.getDbName(), t.getTableName()); - GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName)); - rqst.setValidTxnList(validTxnList.writeToString()); - return TxnUtils.createValidCompactWriteIdList( - txnHandler.getValidWriteIds(rqst).getTblValidWriteIds().get(0)); - } @VisibleForTesting protected String resolveUserToRunAs(Map cache, Table t, Partition p) @@ -394,160 +318,6 @@ private boolean foundCurrentOrFailedCompactions(ShowCompactResponse compactions, return false; } - private CompactionType checkForCompaction(final CompactionInfo ci, - final ValidWriteIdList writeIds, - final StorageDescriptor sd, - final Map tblproperties, - final String runAs) - throws IOException, InterruptedException { - // If it's marked as too many aborted, we already know we need to compact - if (ci.tooManyAborts) { - LOG.debug("Found too many aborted transactions for " + ci.getFullPartitionName() + ", " + - "initiating major compaction"); - return CompactionType.MAJOR; - } - - if (ci.hasOldAbort) { - HiveConf.ConfVars oldAbortedTimeoutProp = - HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD; - LOG.debug("Found an aborted transaction for " + ci.getFullPartitionName() - + " with age older than threshold " + oldAbortedTimeoutProp + ": " + conf - .getTimeVar(oldAbortedTimeoutProp, TimeUnit.HOURS) + " hours. " - + "Initiating minor compaction."); - return CompactionType.MINOR; - } - AcidDirectory acidDirectory = getAcidDirectory(sd, writeIds); - long baseSize = getBaseSize(acidDirectory); - FileSystem fs = acidDirectory.getFs(); - Map deltaSizes = new HashMap<>(); - for (AcidUtils.ParsedDelta delta : acidDirectory.getCurrentDirectories()) { - deltaSizes.put(delta.getPath(), getDirSize(fs, delta)); - } - long deltaSize = deltaSizes.values().stream().reduce(0L, Long::sum); - AcidMetricService.updateMetricsFromInitiator(ci.dbname, ci.tableName, ci.partName, conf, txnHandler, - baseSize, deltaSizes, acidDirectory.getObsolete()); - - if (runJobAsSelf(runAs)) { - return determineCompactionType(ci, acidDirectory, tblproperties, baseSize, deltaSize); - } else { - LOG.info("Going to initiate as user " + runAs + " for " + ci.getFullPartitionName()); - UserGroupInformation ugi = UserGroupInformation.createProxyUser(runAs, - UserGroupInformation.getLoginUser()); - CompactionType compactionType; - try { - compactionType = ugi.doAs( - (PrivilegedExceptionAction) () -> determineCompactionType(ci, acidDirectory, tblproperties, baseSize, deltaSize)); - } finally { - try { - FileSystem.closeAllForUGI(ugi); - } catch (IOException exception) { - LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " + - ci.getFullPartitionName(), exception); - } - } - return compactionType; - } - } - - private AcidDirectory getAcidDirectory(StorageDescriptor sd, ValidWriteIdList writeIds) throws IOException { - Path location = new Path(sd.getLocation()); - FileSystem fs = location.getFileSystem(conf); - return AcidUtils.getAcidState(fs, location, conf, writeIds, Ref.from(false), false); - } - - private CompactionType determineCompactionType(CompactionInfo ci, AcidDirectory dir, Map tblproperties, long baseSize, long deltaSize) { - boolean noBase = false; - List deltas = dir.getCurrentDirectories(); - if (baseSize == 0 && deltaSize > 0) { - noBase = true; - } else { - String deltaPctProp = tblproperties.get(COMPACTORTHRESHOLD_PREFIX + - HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD); - float deltaPctThreshold = deltaPctProp == null ? - HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD) : - Float.parseFloat(deltaPctProp); - boolean bigEnough = (float)deltaSize/(float)baseSize > deltaPctThreshold; - boolean multiBase = dir.getObsolete().stream() - .anyMatch(path -> path.getName().startsWith(AcidUtils.BASE_PREFIX)); - - boolean initiateMajor = bigEnough || (deltaSize == 0 && multiBase); - if (LOG.isDebugEnabled()) { - StringBuilder msg = new StringBuilder("delta size: "); - msg.append(deltaSize); - msg.append(" base size: "); - msg.append(baseSize); - msg.append(" multiBase "); - msg.append(multiBase); - msg.append(" deltaSize "); - msg.append(deltaSize); - msg.append(" threshold: "); - msg.append(deltaPctThreshold); - msg.append(" delta/base ratio > ").append(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD.varname) - .append(": "); - msg.append(bigEnough); - msg.append("."); - if (!initiateMajor) { - msg.append("not"); - } - msg.append(" initiating major compaction."); - LOG.debug(msg.toString()); - } - if (initiateMajor) return CompactionType.MAJOR; - } - - String deltaNumProp = tblproperties.get(COMPACTORTHRESHOLD_PREFIX + - HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD); - int deltaNumThreshold = deltaNumProp == null ? - HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD) : - Integer.parseInt(deltaNumProp); - boolean enough = deltas.size() > deltaNumThreshold; - if (!enough) { - LOG.debug("Not enough deltas to initiate compaction for table=" + ci.tableName + "partition=" + ci.partName - + ". Found: " + deltas.size() + " deltas, threshold is " + deltaNumThreshold); - return null; - } - // If there's no base file, do a major compaction - LOG.debug("Found " + deltas.size() + " delta files, and " + (noBase ? "no" : "has") + " base," + - "requesting " + (noBase ? "major" : "minor") + " compaction"); - - return noBase || !isMinorCompactionSupported(tblproperties, dir) ? - CompactionType.MAJOR : CompactionType.MINOR; - } - - private long getBaseSize(AcidDirectory dir) throws IOException { - long baseSize = 0; - if (dir.getBase() != null) { - baseSize = getDirSize(dir.getFs(), dir.getBase()); - } else { - for (HdfsFileStatusWithId origStat : dir.getOriginalFiles()) { - baseSize += origStat.getFileStatus().getLen(); - } - } - return baseSize; - } - - private long getDirSize(FileSystem fs, ParsedDirectory dir) throws IOException { - return dir.getFiles(fs, Ref.from(false)).stream() - .map(HdfsFileStatusWithId::getFileStatus) - .mapToLong(FileStatus::getLen) - .sum(); - } - - private void requestCompaction(CompactionInfo ci, String runAs) throws MetaException { - CompactionRequest rqst = new CompactionRequest(ci.dbname, ci.tableName, ci.type); - if (ci.partName != null) rqst.setPartitionname(ci.partName); - rqst.setRunas(runAs); - rqst.setInitiatorId(getInitiatorId(Thread.currentThread().getId())); - rqst.setInitiatorVersion(this.runtimeVersion); - rqst.setPoolName(ci.poolName); - LOG.info("Requesting compaction: " + rqst); - CompactionResponse resp = txnHandler.compact(rqst); - if(resp.isAccepted()) { - ci.id = resp.getId(); - } - } - // Check if it's a dynamic partitioning case. If so, do not initiate compaction for streaming ingest, only for aborts. private static boolean isDynPartIngest(Table t, CompactionInfo ci){ if (t.getPartitionKeys() != null && t.getPartitionKeys().size() > 0 && @@ -637,13 +407,6 @@ private boolean isEligibleForCompaction(CompactionInfo ci, return true; } - private String getInitiatorId(long threadId) { - StringBuilder name = new StringBuilder(this.hostName); - name.append("-"); - name.append(threadId); - return name.toString(); - } - private static class InitiatorCycleUpdater implements Runnable { private final String metric; private final long startedAt; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java new file mode 100644 index 000000000000..14dd2ebffe02 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.txn.compactor; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.common.ValidWriteIdList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.CompactionRequest; +import org.apache.hadoop.hive.metastore.api.CompactionResponse; +import org.apache.hadoop.hive.metastore.api.CompactionType; +import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; +import org.apache.hadoop.hive.metastore.metrics.Metrics; +import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.apache.hadoop.hive.metastore.utils.JavaUtils; +import org.apache.hadoop.hive.ql.io.AcidDirectory; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.shims.HadoopShims; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.common.util.Ref; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class InitiatorBase extends MetaStoreCompactorThread { + + static final private String COMPACTOR_THRESHOLD_PREFIX = "compactorthreshold."; + + private List initiateCompactionForMultiplePartitions(Table table, + Map partitions, CompactionRequest request) { + List compactionResponses = new ArrayList<>(); + partitions.entrySet().parallelStream().forEach(entry -> { + try { + StorageDescriptor sd = resolveStorageDescriptor(table, entry.getValue()); + String runAs = TxnUtils.findUserToRunAs(sd.getLocation(), table, conf); + CompactionInfo ci = + new CompactionInfo(table.getDbName(), table.getTableName(), entry.getKey(), request.getType()); + ci.initiatorId = request.getInitiatorId(); + ci.orderByClause = request.getOrderByClause(); + ci.initiatorVersion = request.getInitiatorVersion(); + if (request.getNumberOfBuckets() > 0) { + ci.numberOfBuckets = request.getNumberOfBuckets(); + } + ci.poolName = request.getPoolName(); + LOG.info( + "Checking to see if we should compact partition " + entry.getKey() + " of table " + table.getDbName() + "." + + table.getTableName()); + CollectionUtils.addIgnoreNull(compactionResponses, + scheduleCompactionIfRequired(ci, table, entry.getValue(), runAs, false)); + } catch (IOException | InterruptedException | MetaException e) { + LOG.error( + "Error occurred while Checking if we should compact partition " + entry.getKey() + " of table " + table.getDbName() + "." + + table.getTableName() + " Exception: " + e.getMessage()); + throw new RuntimeException(e); + } + }); + return compactionResponses; + } + + public List initiateCompactionForTable(CompactionRequest request, Table table, Map partitions) throws Exception { + ValidTxnList validTxnList = TxnCommonUtils.createValidReadTxnList(txnHandler.getOpenTxns(), 0); + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + + if (request.getPartitionname()!= null || partitions.isEmpty()) { + List responses = new ArrayList<>(); + responses.add(txnHandler.compact(request)); + return responses; + } else { + return initiateCompactionForMultiplePartitions(table, partitions, request); + } + } + + @Override protected boolean isCacheEnabled() { + return false; + } + + private String getInitiatorId(long threadId) { + return this.hostName + "-" + threadId; + } + + private CompactionResponse requestCompaction(CompactionInfo ci, String runAs) throws MetaException { + CompactionRequest compactionRequest = new CompactionRequest(ci.dbname, ci.tableName, ci.type); + if (ci.partName != null) + compactionRequest.setPartitionname(ci.partName); + compactionRequest.setRunas(runAs); + if (StringUtils.isEmpty(ci.initiatorId)) { + compactionRequest.setInitiatorId(getInitiatorId(Thread.currentThread().getId())); + } else { + compactionRequest.setInitiatorId(ci.initiatorId); + } + compactionRequest.setInitiatorVersion(this.runtimeVersion); + compactionRequest.setPoolName(ci.poolName); + LOG.info("Requesting compaction: " + compactionRequest); + CompactionResponse resp = txnHandler.compact(compactionRequest); + if (resp.isAccepted()) { + ci.id = resp.getId(); + } + return resp; + } + + private AcidDirectory getAcidDirectory(StorageDescriptor sd, ValidWriteIdList writeIds) throws IOException { + Path location = new Path(sd.getLocation()); + FileSystem fs = location.getFileSystem(conf); + return AcidUtils.getAcidState(fs, location, conf, writeIds, Ref.from(false), false); + } + + private CompactionType determineCompactionType(CompactionInfo ci, AcidDirectory dir, + Map tblProperties, long baseSize, long deltaSize) { + boolean noBase = false; + List deltas = dir.getCurrentDirectories(); + if (baseSize == 0 && deltaSize > 0) { + noBase = true; + } else { + String deltaPctProp = + tblProperties.get(COMPACTOR_THRESHOLD_PREFIX + HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD); + float deltaPctThreshold = deltaPctProp == null ? HiveConf.getFloatVar(conf, + HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD) : Float.parseFloat(deltaPctProp); + boolean bigEnough = (float) deltaSize / (float) baseSize > deltaPctThreshold; + boolean multiBase = dir.getObsolete().stream().anyMatch(path -> path.getName().startsWith(AcidUtils.BASE_PREFIX)); + + boolean initiateMajor = bigEnough || (deltaSize == 0 && multiBase); + if (LOG.isDebugEnabled()) { + StringBuilder msg = new StringBuilder("delta size: "); + msg.append(deltaSize); + msg.append(" base size: "); + msg.append(baseSize); + msg.append(" multiBase "); + msg.append(multiBase); + msg.append(" deltaSize "); + msg.append(deltaSize); + msg.append(" threshold: "); + msg.append(deltaPctThreshold); + msg.append(" delta/base ratio > ").append(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD.varname) + .append(": "); + msg.append(bigEnough); + msg.append("."); + if (!initiateMajor) { + msg.append("not"); + } + msg.append(" initiating major compaction."); + LOG.debug(msg.toString()); + } + if (initiateMajor) + return CompactionType.MAJOR; + } + + String deltaNumProp = + tblProperties.get(COMPACTOR_THRESHOLD_PREFIX + HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD); + int deltaNumThreshold = deltaNumProp == null ? HiveConf.getIntVar(conf, + HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD) : Integer.parseInt(deltaNumProp); + boolean enough = deltas.size() > deltaNumThreshold; + if (!enough) { + LOG.debug("Not enough deltas to initiate compaction for table=" + ci.tableName + "partition=" + ci.partName + + ". Found: " + deltas.size() + " deltas, threshold is " + deltaNumThreshold); + return null; + } + // If there's no base file, do a major compaction + LOG.debug("Found " + deltas.size() + " delta files, and " + (noBase ? "no" : "has") + " base," + "requesting " + + (noBase ? "major" : "minor") + " compaction"); + + return noBase || !isMinorCompactionSupported(tblProperties, dir) ? CompactionType.MAJOR : CompactionType.MINOR; + } + + private long getBaseSize(AcidDirectory dir) throws IOException { + long baseSize = 0; + if (dir.getBase() != null) { + baseSize = getDirSize(dir.getFs(), dir.getBase()); + } else { + for (HadoopShims.HdfsFileStatusWithId origStat : dir.getOriginalFiles()) { + baseSize += origStat.getFileStatus().getLen(); + } + } + return baseSize; + } + + private long getDirSize(FileSystem fs, AcidUtils.ParsedDirectory dir) throws IOException { + return dir.getFiles(fs, Ref.from(false)).stream().map(HadoopShims.HdfsFileStatusWithId::getFileStatus) + .mapToLong(FileStatus::getLen).sum(); + } + + private CompactionType checkForCompaction(final CompactionInfo ci, final ValidWriteIdList writeIds, + final StorageDescriptor sd, final Map tblProperties, final String runAs) + throws IOException, InterruptedException { + // If it's marked as too many aborted, we already know we need to compact + if (ci.tooManyAborts) { + LOG.debug("Found too many aborted transactions for " + ci.getFullPartitionName() + ", " + + "initiating major compaction"); + return CompactionType.MAJOR; + } + + if (ci.hasOldAbort) { + HiveConf.ConfVars oldAbortedTimeoutProp = HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD; + LOG.debug("Found an aborted transaction for " + ci.getFullPartitionName() + " with age older than threshold " + + oldAbortedTimeoutProp + ": " + conf.getTimeVar(oldAbortedTimeoutProp, TimeUnit.HOURS) + " hours. " + + "Initiating minor compaction."); + return CompactionType.MINOR; + } + AcidDirectory acidDirectory = getAcidDirectory(sd, writeIds); + long baseSize = getBaseSize(acidDirectory); + FileSystem fs = acidDirectory.getFs(); + Map deltaSizes = new HashMap<>(); + for (AcidUtils.ParsedDelta delta : acidDirectory.getCurrentDirectories()) { + deltaSizes.put(delta.getPath(), getDirSize(fs, delta)); + } + long deltaSize = deltaSizes.values().stream().reduce(0L, Long::sum); + AcidMetricService.updateMetricsFromInitiator(ci.dbname, ci.tableName, ci.partName, conf, txnHandler, baseSize, + deltaSizes, acidDirectory.getObsolete()); + + if (runJobAsSelf(runAs)) { + return determineCompactionType(ci, acidDirectory, tblProperties, baseSize, deltaSize); + } else { + LOG.info("Going to initiate as user " + runAs + " for " + ci.getFullPartitionName()); + UserGroupInformation ugi = UserGroupInformation.createProxyUser(runAs, UserGroupInformation.getLoginUser()); + CompactionType compactionType; + try { + compactionType = ugi.doAs( + (PrivilegedExceptionAction) () -> determineCompactionType(ci, acidDirectory, tblProperties, + baseSize, deltaSize)); + } finally { + try { + FileSystem.closeAllForUGI(ugi); + } catch (IOException exception) { + LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " + ci.getFullPartitionName(), + exception); + } + } + return compactionType; + } + } + + private ValidWriteIdList resolveValidWriteIds(Table t) + throws NoSuchTxnException, MetaException { + ValidTxnList validTxnList = new ValidReadTxnList(conf.get(ValidTxnList.VALID_TXNS_KEY)); + // The response will have one entry per table and hence we get only one ValidWriteIdList + String fullTableName = TxnUtils.getFullTableName(t.getDbName(), t.getTableName()); + GetValidWriteIdsRequest validWriteIdsRequest = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName)); + validWriteIdsRequest.setValidTxnList(validTxnList.writeToString()); + + return TxnUtils.createValidCompactWriteIdList(txnHandler.getValidWriteIds(validWriteIdsRequest).getTblValidWriteIds().get(0)); + } + + protected CompactionResponse scheduleCompactionIfRequired(CompactionInfo ci, Table t, + Partition p, String runAs, boolean metricsEnabled) + throws MetaException { + StorageDescriptor sd = resolveStorageDescriptor(t, p); + try { + ValidWriteIdList validWriteIds = resolveValidWriteIds(t); + + checkInterrupt(); + + CompactionType type = checkForCompaction(ci, validWriteIds, sd, t.getParameters(), runAs); + if (type != null) { + ci.type = type; + return requestCompaction(ci, runAs); + } + } catch (InterruptedException e) { + //Handle InterruptedException separately so the compactionInfo won't be marked as failed. + LOG.info("Initiator pool is being shut down, task received interruption."); + } catch (Throwable ex) { + String errorMessage = "Caught exception while trying to determine if we should compact " + ci + ". Marking " + + "failed to avoid repeated failures, " + ex; + LOG.error(errorMessage); + ci.errorMessage = errorMessage; + if (metricsEnabled) { + Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_INITIATOR_FAILURE_COUNTER).inc(); + } + txnHandler.markFailed(ci); + } + return null; + } + +} diff --git a/ql/src/test/queries/clientpositive/manual_compaction.q b/ql/src/test/queries/clientpositive/manual_compaction.q new file mode 100644 index 000000000000..a794ff7555af --- /dev/null +++ b/ql/src/test/queries/clientpositive/manual_compaction.q @@ -0,0 +1,68 @@ +-- Mask the enqueue time which is based on current time +--! qt:replace:/(initiated\s+---\s+---\s+)[0-9]*(\s+---)/$1#Masked#$2/ +--! qt:replace:/(---\s+)[a-zA-Z0-9\-]+(\s+manual)/$1#Masked#$2/ +-- Mask the hostname in show compaction +--! qt:replace:/(---\s+)[\S]*(\s+manual)/$1#Masked#$2/ +-- Mask compaction id as they will be allocated in parallel threads +--! qt:replace:/^[0-9]/#Masked#/ + +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table UN_PARTITIONED_T(key string, val string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true'); + +create table UN_PARTITIONED_T_MINOR(key string, val string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true'); + +create table PARTITIONED_T(key string, val string) partitioned by (dt string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true'); + +alter table UN_PARTITIONED_T compact 'major'; + +alter table UN_PARTITIONED_T_MINOR compact 'minor'; + +alter table PARTITIONED_T add partition(dt='2023'); + +insert into PARTITIONED_T partition(dt='2023') values ('k1','v1'); +insert into PARTITIONED_T partition(dt='2023') values ('k2','v2'); +insert into PARTITIONED_T partition(dt='2023') values ('k3','v3'); + +alter table PARTITIONED_T partition(dt='2023') compact 'minor'; + +SHOW COMPACTIONS ORDER BY 'PARTITION' DESC; + +alter table PARTITIONED_T add partition(dt='2024'); + +insert into PARTITIONED_T partition(dt='2024') values ('k1','v1'); +insert into PARTITIONED_T partition(dt='2024') values ('k2','v2'); +insert into PARTITIONED_T partition(dt='2024') values ('k3','v3'); +insert into PARTITIONED_T partition(dt='2024') values ('k4','v4'); +insert into PARTITIONED_T partition(dt='2024') values ('k5','v5'); +insert into PARTITIONED_T partition(dt='2024') values ('k6','v6'); +insert into PARTITIONED_T partition(dt='2024') values ('k7','v7'); +insert into PARTITIONED_T partition(dt='2024') values ('k8','v8'); +insert into PARTITIONED_T partition(dt='2024') values ('k9','v9'); +insert into PARTITIONED_T partition(dt='2024') values ('k10','v10'); +insert into PARTITIONED_T partition(dt='2024') values ('k11','v11'); + +insert into PARTITIONED_T partition(dt='2022') values ('k1','v1'); +insert into PARTITIONED_T partition(dt='2022') values ('k2','v2'); +insert into PARTITIONED_T partition(dt='2022') values ('k3','v3'); +insert into PARTITIONED_T partition(dt='2022') values ('k4','v4'); +insert into PARTITIONED_T partition(dt='2022') values ('k5','v5'); +insert into PARTITIONED_T partition(dt='2022') values ('k6','v6'); +insert into PARTITIONED_T partition(dt='2022') values ('k7','v7'); +insert into PARTITIONED_T partition(dt='2022') values ('k8','v8'); +insert into PARTITIONED_T partition(dt='2022') values ('k9','v9'); +insert into PARTITIONED_T partition(dt='2022') values ('k10','v10'); +insert into PARTITIONED_T partition(dt='2022') values ('k11','v11'); + +explain alter table PARTITIONED_T compact 'major'; + +alter table PARTITIONED_T compact 'major'; + +SHOW COMPACTIONS ORDER BY 'PARTITION' DESC; + +drop table UN_PARTITIONED_T; + +drop table UN_PARTITIONED_T_MINOR; + +drop table PARTITIONED_T; diff --git a/ql/src/test/results/clientpositive/llap/manual_compaction.q.out b/ql/src/test/results/clientpositive/llap/manual_compaction.q.out new file mode 100644 index 000000000000..90dddd709e15 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/manual_compaction.q.out @@ -0,0 +1,387 @@ +PREHOOK: query: create table UN_PARTITIONED_T(key string, val string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@UN_PARTITIONED_T +POSTHOOK: query: create table UN_PARTITIONED_T(key string, val string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@UN_PARTITIONED_T +PREHOOK: query: create table UN_PARTITIONED_T_MINOR(key string, val string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@UN_PARTITIONED_T_MINOR +POSTHOOK: query: create table UN_PARTITIONED_T_MINOR(key string, val string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@UN_PARTITIONED_T_MINOR +PREHOOK: query: create table PARTITIONED_T(key string, val string) partitioned by (dt string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@PARTITIONED_T +POSTHOOK: query: create table PARTITIONED_T(key string, val string) partitioned by (dt string) clustered by (val) into 2 buckets stored as ORC TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@PARTITIONED_T +PREHOOK: query: alter table UN_PARTITIONED_T compact 'major' +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@un_partitioned_t +PREHOOK: Output: default@un_partitioned_t +POSTHOOK: query: alter table UN_PARTITIONED_T compact 'major' +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@un_partitioned_t +POSTHOOK: Output: default@un_partitioned_t +PREHOOK: query: alter table UN_PARTITIONED_T_MINOR compact 'minor' +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@un_partitioned_t_minor +PREHOOK: Output: default@un_partitioned_t_minor +POSTHOOK: query: alter table UN_PARTITIONED_T_MINOR compact 'minor' +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@un_partitioned_t_minor +POSTHOOK: Output: default@un_partitioned_t_minor +PREHOOK: query: alter table PARTITIONED_T add partition(dt='2023') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: default@partitioned_t +POSTHOOK: query: alter table PARTITIONED_T add partition(dt='2023') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: default@partitioned_t +POSTHOOK: Output: default@partitioned_t@dt=2023 +PREHOOK: query: insert into PARTITIONED_T partition(dt='2023') values ('k1','v1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2023') values ('k1','v1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2023).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2023).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2023') values ('k2','v2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2023') values ('k2','v2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2023).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2023).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2023') values ('k3','v3') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2023') values ('k3','v3') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2023).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2023).val SCRIPT [] +PREHOOK: query: alter table PARTITIONED_T partition(dt='2023') compact 'minor' +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@partitioned_t +PREHOOK: Output: default@partitioned_t@dt=2023 +POSTHOOK: query: alter table PARTITIONED_T partition(dt='2023') compact 'minor' +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@partitioned_t +POSTHOOK: Output: default@partitioned_t@dt=2023 +PREHOOK: query: SHOW COMPACTIONS ORDER BY 'PARTITION' DESC +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: SHOW COMPACTIONS ORDER BY 'PARTITION' DESC +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default un_partitioned_t_minor --- MINOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +#Masked# default un_partitioned_t --- MAJOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +#Masked# default partitioned_t dt=2023 MINOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +PREHOOK: query: alter table PARTITIONED_T add partition(dt='2024') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: default@partitioned_t +POSTHOOK: query: alter table PARTITIONED_T add partition(dt='2024') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: default@partitioned_t +POSTHOOK: Output: default@partitioned_t@dt=2024 +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k1','v1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k1','v1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k2','v2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k2','v2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k3','v3') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k3','v3') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k4','v4') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k4','v4') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k5','v5') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k5','v5') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k6','v6') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k6','v6') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k7','v7') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k7','v7') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k8','v8') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k8','v8') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k9','v9') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k9','v9') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k10','v10') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k10','v10') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k11','v11') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2024') values ('k11','v11') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2024 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2024).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k1','v1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k1','v1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k2','v2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k2','v2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k3','v3') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k3','v3') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k4','v4') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k4','v4') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k5','v5') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k5','v5') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k6','v6') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k6','v6') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k7','v7') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k7','v7') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k8','v8') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k8','v8') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k9','v9') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k9','v9') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k10','v10') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k10','v10') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k11','v11') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: query: insert into PARTITIONED_T partition(dt='2022') values ('k11','v11') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@partitioned_t@dt=2022 +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).key SCRIPT [] +POSTHOOK: Lineage: partitioned_t PARTITION(dt=2022).val SCRIPT [] +PREHOOK: query: explain alter table PARTITIONED_T compact 'major' +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@partitioned_t +PREHOOK: Output: default@partitioned_t +POSTHOOK: query: explain alter table PARTITIONED_T compact 'major' +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@partitioned_t +POSTHOOK: Output: default@partitioned_t +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.PARTITIONED_T + numberOfBuckets: 0 + table name: default.PARTITIONED_T + +PREHOOK: query: alter table PARTITIONED_T compact 'major' +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@partitioned_t +PREHOOK: Output: default@partitioned_t +POSTHOOK: query: alter table PARTITIONED_T compact 'major' +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@partitioned_t +POSTHOOK: Output: default@partitioned_t +PREHOOK: query: SHOW COMPACTIONS ORDER BY 'PARTITION' DESC +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: SHOW COMPACTIONS ORDER BY 'PARTITION' DESC +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default un_partitioned_t_minor --- MINOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +#Masked# default un_partitioned_t --- MAJOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +#Masked# default partitioned_t dt=2024 MAJOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +#Masked# default partitioned_t dt=2023 MINOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +#Masked# default partitioned_t dt=2022 MAJOR initiated --- --- #Masked# --- --- --- --- #Masked# manual default 0 0 0 --- +PREHOOK: query: drop table UN_PARTITIONED_T +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@un_partitioned_t +PREHOOK: Output: database:default +PREHOOK: Output: default@un_partitioned_t +POSTHOOK: query: drop table UN_PARTITIONED_T +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@un_partitioned_t +POSTHOOK: Output: database:default +POSTHOOK: Output: default@un_partitioned_t +PREHOOK: query: drop table UN_PARTITIONED_T_MINOR +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@un_partitioned_t_minor +PREHOOK: Output: database:default +PREHOOK: Output: default@un_partitioned_t_minor +POSTHOOK: query: drop table UN_PARTITIONED_T_MINOR +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@un_partitioned_t_minor +POSTHOOK: Output: database:default +POSTHOOK: Output: default@un_partitioned_t_minor +PREHOOK: query: drop table PARTITIONED_T +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@partitioned_t +PREHOOK: Output: database:default +PREHOOK: Output: default@partitioned_t +POSTHOOK: query: drop table PARTITIONED_T +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@partitioned_t +POSTHOOK: Output: database:default +POSTHOOK: Output: default@partitioned_t From 58569c5947c18003ed28fde62218631618918984 Mon Sep 17 00:00:00 2001 From: Sai Hemanth Gantasala <68923650+saihemanth-cloudera@users.noreply.github.com> Date: Thu, 2 Nov 2023 09:46:05 -0700 Subject: [PATCH 034/179] HIVE-27499: Include optional db/table names in the NotificationEvent thrift object (#4569) (Sai Hemanth Gantasala, Reviewed by Zhihua Deng, Naveen Gangam, Henrib) --- .../listener/TestDbNotificationListener.java | 64 +- .../thrift/gen-cpp/ThriftHiveMetastore.cpp | 2560 ++++----- .../thrift/gen-cpp/hive_metastore_types.cpp | 4556 +++++++++-------- .../gen/thrift/gen-cpp/hive_metastore_types.h | 40 +- .../metastore/api/AlterPartitionsRequest.java | 72 +- .../hive/metastore/api/AlterTableRequest.java | 32 +- .../api/ClearFileMetadataRequest.java | 32 +- .../metastore/api/ClientCapabilities.java | 36 +- .../metastore/api/CreateDatabaseRequest.java | 44 +- .../metastore/api/CreateTableRequest.java | 248 +- .../hive/metastore/api/ExtendedTableInfo.java | 64 +- .../metastore/api/FindSchemasByColsResp.java | 36 +- .../hive/metastore/api/FireEventRequest.java | 76 +- .../metastore/api/FireEventRequestData.java | 36 +- .../hive/metastore/api/FireEventResponse.java | 32 +- .../api/GetAllFunctionsResponse.java | 36 +- .../metastore/api/GetDatabaseRequest.java | 32 +- .../hive/metastore/api/GetFieldsResponse.java | 36 +- .../api/GetFileMetadataByExprRequest.java | 32 +- .../api/GetFileMetadataByExprResult.java | 48 +- .../metastore/api/GetFileMetadataRequest.java | 32 +- .../metastore/api/GetFileMetadataResult.java | 44 +- .../metastore/api/GetOpenTxnsRequest.java | 36 +- .../api/GetPartitionNamesPsRequest.java | 32 +- .../api/GetPartitionNamesPsResponse.java | 32 +- .../metastore/api/GetPartitionRequest.java | 32 +- .../api/GetPartitionsFilterSpec.java | 32 +- .../api/GetPartitionsPsWithAuthRequest.java | 64 +- .../api/GetPartitionsPsWithAuthResponse.java | 36 +- .../metastore/api/GetPartitionsRequest.java | 64 +- .../metastore/api/GetPartitionsResponse.java | 36 +- .../metastore/api/GetProjectionsSpec.java | 32 +- .../hive/metastore/api/GetSchemaResponse.java | 36 +- .../hive/metastore/api/GetTableRequest.java | 32 +- .../metastore/api/GetTablesExtRequest.java | 32 +- .../hive/metastore/api/GetTablesRequest.java | 64 +- .../hive/metastore/api/GetTablesResult.java | 36 +- .../metastore/api/InsertEventRequestData.java | 128 +- .../api/NotificationEventRequest.java | 396 +- .../api/NotificationEventResponse.java | 36 +- .../api/NotificationEventsCountRequest.java | 164 +- .../metastore/api/PartitionsResponse.java | 36 +- .../metastore/api/PutFileMetadataRequest.java | 64 +- .../metastore/api/RenamePartitionRequest.java | 32 +- .../metastore/api/ReplicationMetricList.java | 36 +- .../hive/metastore/api/SchemaVersion.java | 36 +- .../metastore/api/ThriftHiveMetastore.java | 2780 +++++----- .../metastore/api/WMFullResourcePlan.java | 144 +- .../api/WMGetAllResourcePlanResponse.java | 36 +- .../WMGetTriggersForResourePlanResponse.java | 36 +- .../api/WMValidateResourcePlanResponse.java | 64 +- .../api/WriteNotificationLogBatchRequest.java | 36 +- .../api/WriteNotificationLogRequest.java | 32 +- .../metastore/AlterPartitionsRequest.php | 40 +- .../gen-php/metastore/AlterTableRequest.php | 18 +- .../metastore/ClearFileMetadataRequest.php | 18 +- .../gen-php/metastore/ClientCapabilities.php | 18 +- .../metastore/CreateDatabaseRequest.php | 26 +- .../gen-php/metastore/CreateTableRequest.php | 138 +- .../gen-php/metastore/ExtendedTableInfo.php | 36 +- .../metastore/FindSchemasByColsResp.php | 20 +- .../gen-php/metastore/FireEventRequest.php | 44 +- .../metastore/FireEventRequestData.php | 20 +- .../gen-php/metastore/FireEventResponse.php | 18 +- .../metastore/GetAllFunctionsResponse.php | 20 +- .../gen-php/metastore/GetDatabaseRequest.php | 18 +- .../gen-php/metastore/GetFieldsResponse.php | 20 +- .../GetFileMetadataByExprRequest.php | 18 +- .../metastore/GetFileMetadataByExprResult.php | 28 +- .../metastore/GetFileMetadataRequest.php | 18 +- .../metastore/GetFileMetadataResult.php | 26 +- .../gen-php/metastore/GetOpenTxnsRequest.php | 18 +- .../metastore/GetPartitionNamesPsRequest.php | 18 +- .../metastore/GetPartitionNamesPsResponse.php | 18 +- .../gen-php/metastore/GetPartitionRequest.php | 18 +- .../metastore/GetPartitionsFilterSpec.php | 18 +- .../GetPartitionsPsWithAuthRequest.php | 36 +- .../GetPartitionsPsWithAuthResponse.php | 20 +- .../metastore/GetPartitionsRequest.php | 36 +- .../metastore/GetPartitionsResponse.php | 20 +- .../gen-php/metastore/GetProjectionsSpec.php | 18 +- .../gen-php/metastore/GetSchemaResponse.php | 20 +- .../gen-php/metastore/GetTableRequest.php | 18 +- .../gen-php/metastore/GetTablesExtRequest.php | 18 +- .../gen-php/metastore/GetTablesRequest.php | 36 +- .../gen-php/metastore/GetTablesResult.php | 20 +- .../metastore/InsertEventRequestData.php | 72 +- .../metastore/NotificationEventRequest.php | 96 +- .../metastore/NotificationEventResponse.php | 20 +- .../NotificationEventsCountRequest.php | 44 + .../gen-php/metastore/PartitionsResponse.php | 20 +- .../metastore/PutFileMetadataRequest.php | 36 +- .../metastore/RenamePartitionRequest.php | 18 +- .../metastore/ReplicationMetricList.php | 20 +- .../gen-php/metastore/SchemaVersion.php | 20 +- ...hriftHiveMetastore_add_partitions_args.php | 20 +- ...iveMetastore_add_partitions_pspec_args.php | 20 +- ...tore_add_write_ids_to_min_history_args.php | 26 +- ...iftHiveMetastore_alter_partitions_args.php | 20 +- ...rtitions_with_environment_context_args.php | 20 +- ...iftHiveMetastore_append_partition_args.php | 18 +- ...artition_with_environment_context_args.php | 18 +- ...ore_create_table_with_constraints_args.php | 120 +- ...hriftHiveMetastore_drop_partition_args.php | 18 +- ...artition_with_environment_context_args.php | 18 +- ...tHiveMetastore_exchange_partition_args.php | 26 +- ...HiveMetastore_exchange_partitions_args.php | 26 +- ...veMetastore_exchange_partitions_result.php | 20 +- ...tastore_find_columns_with_stats_result.php | 18 +- ...HiveMetastore_get_all_databases_result.php | 18 +- ...ized_view_objects_for_rewriting_result.php | 20 +- ...tHiveMetastore_get_all_packages_result.php | 18 +- ...store_get_all_stored_procedures_result.php | 18 +- ...iftHiveMetastore_get_all_tables_result.php | 18 +- ...store_get_all_token_identifiers_result.php | 18 +- ...astore_get_all_write_event_info_result.php | 20 +- ...riftHiveMetastore_get_databases_result.php | 18 +- ...iveMetastore_get_dataconnectors_result.php | 18 +- .../ThriftHiveMetastore_get_fields_result.php | 20 +- ...fields_with_environment_context_result.php | 20 +- ...riftHiveMetastore_get_functions_result.php | 18 +- ...ftHiveMetastore_get_master_keys_result.php | 18 +- ...aterialized_views_for_rewriting_result.php | 18 +- ...astore_get_part_specs_by_filter_result.php | 20 +- ...ThriftHiveMetastore_get_partition_args.php | 18 +- ...eMetastore_get_partition_names_ps_args.php | 18 +- ...etastore_get_partition_names_ps_result.php | 18 +- ...tastore_get_partition_names_req_result.php | 18 +- ...veMetastore_get_partition_names_result.php | 18 +- ...Metastore_get_partition_with_auth_args.php | 36 +- ...re_get_partitions_by_filter_req_result.php | 20 +- ...astore_get_partitions_by_filter_result.php | 20 +- ...Metastore_get_partitions_by_names_args.php | 18 +- ...tastore_get_partitions_by_names_result.php | 20 +- ...ftHiveMetastore_get_partitions_ps_args.php | 18 +- ...HiveMetastore_get_partitions_ps_result.php | 20 +- ...store_get_partitions_ps_with_auth_args.php | 36 +- ...ore_get_partitions_ps_with_auth_result.php | 20 +- ...eMetastore_get_partitions_pspec_result.php | 20 +- ...iftHiveMetastore_get_partitions_result.php | 20 +- ...etastore_get_partitions_with_auth_args.php | 18 +- ...astore_get_partitions_with_auth_result.php | 20 +- ...ftHiveMetastore_get_privilege_set_args.php | 18 +- ...iftHiveMetastore_get_role_names_result.php | 18 +- ...HiveMetastore_get_runtime_stats_result.php | 20 +- ...tastore_get_schema_all_versions_result.php | 20 +- .../ThriftHiveMetastore_get_schema_result.php | 20 +- ...schema_with_environment_context_result.php | 20 +- ...hriftHiveMetastore_get_table_meta_args.php | 18 +- ...iftHiveMetastore_get_table_meta_result.php | 20 +- ...store_get_table_names_by_filter_result.php | 18 +- ...tastore_get_table_objects_by_name_args.php | 18 +- ...store_get_table_objects_by_name_result.php | 20 +- ...iveMetastore_get_tables_by_type_result.php | 18 +- ...iftHiveMetastore_get_tables_ext_result.php | 20 +- .../ThriftHiveMetastore_get_tables_result.php | 18 +- ...hriftHiveMetastore_get_type_all_result.php | 28 +- ...tastore_isPartitionMarkedForEvent_args.php | 26 +- ...ftHiveMetastore_list_privileges_result.php | 20 +- .../ThriftHiveMetastore_list_roles_result.php | 20 +- ...veMetastore_markPartitionForEvent_args.php | 26 +- ...rtition_name_has_valid_characters_args.php | 18 +- ...etastore_partition_name_to_spec_result.php | 26 +- ...etastore_partition_name_to_vals_result.php | 18 +- ...iftHiveMetastore_rename_partition_args.php | 18 +- .../ThriftHiveMetastore_set_ugi_args.php | 18 +- .../ThriftHiveMetastore_set_ugi_result.php | 18 +- ...hriftHiveMetastore_truncate_table_args.php | 18 +- .../gen-php/metastore/WMFullResourcePlan.php | 80 +- .../WMGetAllResourcePlanResponse.php | 20 +- .../WMGetTriggersForResourePlanResponse.php | 20 +- .../WMValidateResourcePlanResponse.php | 36 +- .../WriteNotificationLogBatchRequest.php | 20 +- .../metastore/WriteNotificationLogRequest.php | 18 +- .../hive_metastore/ThriftHiveMetastore.py | 1056 ++-- .../thrift/gen-py/hive_metastore/ttypes.py | 936 ++-- .../gen/thrift/gen-rb/hive_metastore_types.rb | 12 +- .../src/main/thrift/hive_metastore.thrift | 6 +- .../hadoop/hive/metastore/ObjectStore.java | 39 + 179 files changed, 9434 insertions(+), 8435 deletions(-) diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java index 46f4f2da8335..5be32602d24e 100644 --- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hive.metastore.api.InsertEventRequestData; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.NotificationEventRequest; import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest; import org.apache.hadoop.hive.metastore.api.Partition; @@ -120,7 +121,7 @@ public class TestDbNotificationListener { private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationListener.class - .getName()); + .getName()); private static final int EVENTS_TTL = 30; private static final int CLEANUP_SLEEP_TIME = 10; private static Map emptyParameters = new HashMap(); @@ -1581,5 +1582,66 @@ private void verifyInsert(NotificationEvent event, String dbName, String tblName assertTrue(files.hasNext()); } + /** + * The method creates some events in notification log and fetches the events + * based on the fields set on the NotificationEventRequest object. It includes + * setting database name, table name(s), event skip list (i.e., filter out events + * that are not required). These fields are optional. + * @throws Exception + */ + @Test + public void fetchNotificationEventBasedOnTables() throws Exception { + String dbName = "default"; + String table1 = "test_tbl1"; + String table2 = "test_tbl2"; + String table3 = "test_tbl3"; + // Generate some table events + generateSometableEvents(dbName, table1); + generateSometableEvents(dbName, table2); + generateSometableEvents(dbName, table3); + + // Verify events by table names + NotificationEventRequest request = new NotificationEventRequest(); + request.setLastEvent(firstEventId); + request.setMaxEvents(-1); + request.setDbName(dbName); + request.setTableNames(Arrays.asList(table1)); + NotificationEventResponse rsp1 = msClient.getNextNotification(request, true, null); + assertEquals(12, rsp1.getEventsSize()); + request.setTableNames(Arrays.asList(table1, table2)); + request.setEventTypeSkipList(Arrays.asList("CREATE_TABLE")); + NotificationEventResponse rsp2 = msClient.getNextNotification(request, true, null); + // The actual count of events should 24. Having CREATE_TABLE event in the event skip + // list will result in events count reduced 22 as it skips fetching 2 create_table events + // associated with two different tables. + assertEquals(22, rsp2.getEventsSize()); + request.unsetTableNames(); + request.unsetEventTypeSkipList(); + NotificationEventResponse rsp3 = msClient.getNextNotification(request, true, null); + assertEquals(36, rsp3.getEventsSize()); + + NotificationEventsCountRequest eventsReq = new NotificationEventsCountRequest(firstEventId, dbName); + eventsReq.setTableNames(Arrays.asList(table1)); + assertEquals(12, msClient.getNotificationEventsCount(eventsReq).getEventsCount()); + eventsReq.setTableNames(Arrays.asList(table1, table2)); + assertEquals(24, msClient.getNotificationEventsCount(eventsReq).getEventsCount()); + eventsReq.unsetTableNames(); + assertEquals(36, msClient.getNotificationEventsCount(eventsReq).getEventsCount()); + } + private void generateSometableEvents(String dbName, String tableName) throws Exception { + // CREATE_DATABASE event is generated but we filter this out while fetching events. + driver.run("create database if not exists "+dbName); + driver.run("use "+dbName); + // Event 1: CREATE_TABLE event + driver.run("create table " + tableName + " (c int) partitioned by (ds string)"); + // Event 2: ADD_PARTITION, 3: ALTER_PARTITION, 4: UPDATE_PART_COL_STAT_EVENT events + driver.run("insert into table " + tableName + " partition (ds = 'today') values (1)"); + // Event 5: INSERT, 6: ALTER_PARTITION, 7: UPDATE_PART_COL_STAT_EVENT events + driver.run("insert into table " + tableName + " partition (ds = 'today') values (2)"); + // Event 8: INSERT, 9: ALTER_PARTITION, 10: UPDATE_PART_COL_STAT_EVENT events + driver.run("insert into table " + tableName + " partition (ds) values (3, 'today')"); + // Event 11: ADD_PARTITION, Event 12: ALTER_PARTITION events + driver.run("alter table " + tableName + " add partition (ds = 'yesterday')"); + } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp index f1cf0975146d..9afe386894e1 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp @@ -2975,14 +2975,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1844; - ::apache::thrift::protocol::TType _etype1847; - xfer += iprot->readListBegin(_etype1847, _size1844); - this->success.resize(_size1844); - uint32_t _i1848; - for (_i1848 = 0; _i1848 < _size1844; ++_i1848) + uint32_t _size1856; + ::apache::thrift::protocol::TType _etype1859; + xfer += iprot->readListBegin(_etype1859, _size1856); + this->success.resize(_size1856); + uint32_t _i1860; + for (_i1860 = 0; _i1860 < _size1856; ++_i1860) { - xfer += iprot->readString(this->success[_i1848]); + xfer += iprot->readString(this->success[_i1860]); } xfer += iprot->readListEnd(); } @@ -3021,10 +3021,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1849; - for (_iter1849 = this->success.begin(); _iter1849 != this->success.end(); ++_iter1849) + std::vector ::const_iterator _iter1861; + for (_iter1861 = this->success.begin(); _iter1861 != this->success.end(); ++_iter1861) { - xfer += oprot->writeString((*_iter1849)); + xfer += oprot->writeString((*_iter1861)); } xfer += oprot->writeListEnd(); } @@ -3069,14 +3069,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1850; - ::apache::thrift::protocol::TType _etype1853; - xfer += iprot->readListBegin(_etype1853, _size1850); - (*(this->success)).resize(_size1850); - uint32_t _i1854; - for (_i1854 = 0; _i1854 < _size1850; ++_i1854) + uint32_t _size1862; + ::apache::thrift::protocol::TType _etype1865; + xfer += iprot->readListBegin(_etype1865, _size1862); + (*(this->success)).resize(_size1862); + uint32_t _i1866; + for (_i1866 = 0; _i1866 < _size1862; ++_i1866) { - xfer += iprot->readString((*(this->success))[_i1854]); + xfer += iprot->readString((*(this->success))[_i1866]); } xfer += iprot->readListEnd(); } @@ -3193,14 +3193,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1855; - ::apache::thrift::protocol::TType _etype1858; - xfer += iprot->readListBegin(_etype1858, _size1855); - this->success.resize(_size1855); - uint32_t _i1859; - for (_i1859 = 0; _i1859 < _size1855; ++_i1859) + uint32_t _size1867; + ::apache::thrift::protocol::TType _etype1870; + xfer += iprot->readListBegin(_etype1870, _size1867); + this->success.resize(_size1867); + uint32_t _i1871; + for (_i1871 = 0; _i1871 < _size1867; ++_i1871) { - xfer += iprot->readString(this->success[_i1859]); + xfer += iprot->readString(this->success[_i1871]); } xfer += iprot->readListEnd(); } @@ -3239,10 +3239,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1860; - for (_iter1860 = this->success.begin(); _iter1860 != this->success.end(); ++_iter1860) + std::vector ::const_iterator _iter1872; + for (_iter1872 = this->success.begin(); _iter1872 != this->success.end(); ++_iter1872) { - xfer += oprot->writeString((*_iter1860)); + xfer += oprot->writeString((*_iter1872)); } xfer += oprot->writeListEnd(); } @@ -3287,14 +3287,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1861; - ::apache::thrift::protocol::TType _etype1864; - xfer += iprot->readListBegin(_etype1864, _size1861); - (*(this->success)).resize(_size1861); - uint32_t _i1865; - for (_i1865 = 0; _i1865 < _size1861; ++_i1865) + uint32_t _size1873; + ::apache::thrift::protocol::TType _etype1876; + xfer += iprot->readListBegin(_etype1876, _size1873); + (*(this->success)).resize(_size1873); + uint32_t _i1877; + for (_i1877 = 0; _i1877 < _size1873; ++_i1877) { - xfer += iprot->readString((*(this->success))[_i1865]); + xfer += iprot->readString((*(this->success))[_i1877]); } xfer += iprot->readListEnd(); } @@ -4347,14 +4347,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1866; - ::apache::thrift::protocol::TType _etype1869; - xfer += iprot->readListBegin(_etype1869, _size1866); - this->success.resize(_size1866); - uint32_t _i1870; - for (_i1870 = 0; _i1870 < _size1866; ++_i1870) + uint32_t _size1878; + ::apache::thrift::protocol::TType _etype1881; + xfer += iprot->readListBegin(_etype1881, _size1878); + this->success.resize(_size1878); + uint32_t _i1882; + for (_i1882 = 0; _i1882 < _size1878; ++_i1882) { - xfer += iprot->readString(this->success[_i1870]); + xfer += iprot->readString(this->success[_i1882]); } xfer += iprot->readListEnd(); } @@ -4393,10 +4393,10 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::write(::apache::thrift:: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1871; - for (_iter1871 = this->success.begin(); _iter1871 != this->success.end(); ++_iter1871) + std::vector ::const_iterator _iter1883; + for (_iter1883 = this->success.begin(); _iter1883 != this->success.end(); ++_iter1883) { - xfer += oprot->writeString((*_iter1871)); + xfer += oprot->writeString((*_iter1883)); } xfer += oprot->writeListEnd(); } @@ -4441,14 +4441,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_presult::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1872; - ::apache::thrift::protocol::TType _etype1875; - xfer += iprot->readListBegin(_etype1875, _size1872); - (*(this->success)).resize(_size1872); - uint32_t _i1876; - for (_i1876 = 0; _i1876 < _size1872; ++_i1876) + uint32_t _size1884; + ::apache::thrift::protocol::TType _etype1887; + xfer += iprot->readListBegin(_etype1887, _size1884); + (*(this->success)).resize(_size1884); + uint32_t _i1888; + for (_i1888 = 0; _i1888 < _size1884; ++_i1888) { - xfer += iprot->readString((*(this->success))[_i1876]); + xfer += iprot->readString((*(this->success))[_i1888]); } xfer += iprot->readListEnd(); } @@ -5510,17 +5510,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_MAP) { { this->success.clear(); - uint32_t _size1877; - ::apache::thrift::protocol::TType _ktype1878; - ::apache::thrift::protocol::TType _vtype1879; - xfer += iprot->readMapBegin(_ktype1878, _vtype1879, _size1877); - uint32_t _i1881; - for (_i1881 = 0; _i1881 < _size1877; ++_i1881) + uint32_t _size1889; + ::apache::thrift::protocol::TType _ktype1890; + ::apache::thrift::protocol::TType _vtype1891; + xfer += iprot->readMapBegin(_ktype1890, _vtype1891, _size1889); + uint32_t _i1893; + for (_i1893 = 0; _i1893 < _size1889; ++_i1893) { - std::string _key1882; - xfer += iprot->readString(_key1882); - Type& _val1883 = this->success[_key1882]; - xfer += _val1883.read(iprot); + std::string _key1894; + xfer += iprot->readString(_key1894); + Type& _val1895 = this->success[_key1894]; + xfer += _val1895.read(iprot); } xfer += iprot->readMapEnd(); } @@ -5559,11 +5559,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::map ::const_iterator _iter1884; - for (_iter1884 = this->success.begin(); _iter1884 != this->success.end(); ++_iter1884) + std::map ::const_iterator _iter1896; + for (_iter1896 = this->success.begin(); _iter1896 != this->success.end(); ++_iter1896) { - xfer += oprot->writeString(_iter1884->first); - xfer += _iter1884->second.write(oprot); + xfer += oprot->writeString(_iter1896->first); + xfer += _iter1896->second.write(oprot); } xfer += oprot->writeMapEnd(); } @@ -5608,17 +5608,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_MAP) { { (*(this->success)).clear(); - uint32_t _size1885; - ::apache::thrift::protocol::TType _ktype1886; - ::apache::thrift::protocol::TType _vtype1887; - xfer += iprot->readMapBegin(_ktype1886, _vtype1887, _size1885); - uint32_t _i1889; - for (_i1889 = 0; _i1889 < _size1885; ++_i1889) + uint32_t _size1897; + ::apache::thrift::protocol::TType _ktype1898; + ::apache::thrift::protocol::TType _vtype1899; + xfer += iprot->readMapBegin(_ktype1898, _vtype1899, _size1897); + uint32_t _i1901; + for (_i1901 = 0; _i1901 < _size1897; ++_i1901) { - std::string _key1890; - xfer += iprot->readString(_key1890); - Type& _val1891 = (*(this->success))[_key1890]; - xfer += _val1891.read(iprot); + std::string _key1902; + xfer += iprot->readString(_key1902); + Type& _val1903 = (*(this->success))[_key1902]; + xfer += _val1903.read(iprot); } xfer += iprot->readMapEnd(); } @@ -5772,14 +5772,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1892; - ::apache::thrift::protocol::TType _etype1895; - xfer += iprot->readListBegin(_etype1895, _size1892); - this->success.resize(_size1892); - uint32_t _i1896; - for (_i1896 = 0; _i1896 < _size1892; ++_i1896) + uint32_t _size1904; + ::apache::thrift::protocol::TType _etype1907; + xfer += iprot->readListBegin(_etype1907, _size1904); + this->success.resize(_size1904); + uint32_t _i1908; + for (_i1908 = 0; _i1908 < _size1904; ++_i1908) { - xfer += this->success[_i1896].read(iprot); + xfer += this->success[_i1908].read(iprot); } xfer += iprot->readListEnd(); } @@ -5834,10 +5834,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1897; - for (_iter1897 = this->success.begin(); _iter1897 != this->success.end(); ++_iter1897) + std::vector ::const_iterator _iter1909; + for (_iter1909 = this->success.begin(); _iter1909 != this->success.end(); ++_iter1909) { - xfer += (*_iter1897).write(oprot); + xfer += (*_iter1909).write(oprot); } xfer += oprot->writeListEnd(); } @@ -5890,14 +5890,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1898; - ::apache::thrift::protocol::TType _etype1901; - xfer += iprot->readListBegin(_etype1901, _size1898); - (*(this->success)).resize(_size1898); - uint32_t _i1902; - for (_i1902 = 0; _i1902 < _size1898; ++_i1902) + uint32_t _size1910; + ::apache::thrift::protocol::TType _etype1913; + xfer += iprot->readListBegin(_etype1913, _size1910); + (*(this->success)).resize(_size1910); + uint32_t _i1914; + for (_i1914 = 0; _i1914 < _size1910; ++_i1914) { - xfer += (*(this->success))[_i1902].read(iprot); + xfer += (*(this->success))[_i1914].read(iprot); } xfer += iprot->readListEnd(); } @@ -6083,14 +6083,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1903; - ::apache::thrift::protocol::TType _etype1906; - xfer += iprot->readListBegin(_etype1906, _size1903); - this->success.resize(_size1903); - uint32_t _i1907; - for (_i1907 = 0; _i1907 < _size1903; ++_i1907) + uint32_t _size1915; + ::apache::thrift::protocol::TType _etype1918; + xfer += iprot->readListBegin(_etype1918, _size1915); + this->success.resize(_size1915); + uint32_t _i1919; + for (_i1919 = 0; _i1919 < _size1915; ++_i1919) { - xfer += this->success[_i1907].read(iprot); + xfer += this->success[_i1919].read(iprot); } xfer += iprot->readListEnd(); } @@ -6145,10 +6145,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1908; - for (_iter1908 = this->success.begin(); _iter1908 != this->success.end(); ++_iter1908) + std::vector ::const_iterator _iter1920; + for (_iter1920 = this->success.begin(); _iter1920 != this->success.end(); ++_iter1920) { - xfer += (*_iter1908).write(oprot); + xfer += (*_iter1920).write(oprot); } xfer += oprot->writeListEnd(); } @@ -6201,14 +6201,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1909; - ::apache::thrift::protocol::TType _etype1912; - xfer += iprot->readListBegin(_etype1912, _size1909); - (*(this->success)).resize(_size1909); - uint32_t _i1913; - for (_i1913 = 0; _i1913 < _size1909; ++_i1913) + uint32_t _size1921; + ::apache::thrift::protocol::TType _etype1924; + xfer += iprot->readListBegin(_etype1924, _size1921); + (*(this->success)).resize(_size1921); + uint32_t _i1925; + for (_i1925 = 0; _i1925 < _size1921; ++_i1925) { - xfer += (*(this->success))[_i1913].read(iprot); + xfer += (*(this->success))[_i1925].read(iprot); } xfer += iprot->readListEnd(); } @@ -6625,14 +6625,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1914; - ::apache::thrift::protocol::TType _etype1917; - xfer += iprot->readListBegin(_etype1917, _size1914); - this->success.resize(_size1914); - uint32_t _i1918; - for (_i1918 = 0; _i1918 < _size1914; ++_i1918) + uint32_t _size1926; + ::apache::thrift::protocol::TType _etype1929; + xfer += iprot->readListBegin(_etype1929, _size1926); + this->success.resize(_size1926); + uint32_t _i1930; + for (_i1930 = 0; _i1930 < _size1926; ++_i1930) { - xfer += this->success[_i1918].read(iprot); + xfer += this->success[_i1930].read(iprot); } xfer += iprot->readListEnd(); } @@ -6687,10 +6687,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1919; - for (_iter1919 = this->success.begin(); _iter1919 != this->success.end(); ++_iter1919) + std::vector ::const_iterator _iter1931; + for (_iter1931 = this->success.begin(); _iter1931 != this->success.end(); ++_iter1931) { - xfer += (*_iter1919).write(oprot); + xfer += (*_iter1931).write(oprot); } xfer += oprot->writeListEnd(); } @@ -6743,14 +6743,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1920; - ::apache::thrift::protocol::TType _etype1923; - xfer += iprot->readListBegin(_etype1923, _size1920); - (*(this->success)).resize(_size1920); - uint32_t _i1924; - for (_i1924 = 0; _i1924 < _size1920; ++_i1924) + uint32_t _size1932; + ::apache::thrift::protocol::TType _etype1935; + xfer += iprot->readListBegin(_etype1935, _size1932); + (*(this->success)).resize(_size1932); + uint32_t _i1936; + for (_i1936 = 0; _i1936 < _size1932; ++_i1936) { - xfer += (*(this->success))[_i1924].read(iprot); + xfer += (*(this->success))[_i1936].read(iprot); } xfer += iprot->readListEnd(); } @@ -6936,14 +6936,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1925; - ::apache::thrift::protocol::TType _etype1928; - xfer += iprot->readListBegin(_etype1928, _size1925); - this->success.resize(_size1925); - uint32_t _i1929; - for (_i1929 = 0; _i1929 < _size1925; ++_i1929) + uint32_t _size1937; + ::apache::thrift::protocol::TType _etype1940; + xfer += iprot->readListBegin(_etype1940, _size1937); + this->success.resize(_size1937); + uint32_t _i1941; + for (_i1941 = 0; _i1941 < _size1937; ++_i1941) { - xfer += this->success[_i1929].read(iprot); + xfer += this->success[_i1941].read(iprot); } xfer += iprot->readListEnd(); } @@ -6998,10 +6998,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1930; - for (_iter1930 = this->success.begin(); _iter1930 != this->success.end(); ++_iter1930) + std::vector ::const_iterator _iter1942; + for (_iter1942 = this->success.begin(); _iter1942 != this->success.end(); ++_iter1942) { - xfer += (*_iter1930).write(oprot); + xfer += (*_iter1942).write(oprot); } xfer += oprot->writeListEnd(); } @@ -7054,14 +7054,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1931; - ::apache::thrift::protocol::TType _etype1934; - xfer += iprot->readListBegin(_etype1934, _size1931); - (*(this->success)).resize(_size1931); - uint32_t _i1935; - for (_i1935 = 0; _i1935 < _size1931; ++_i1935) + uint32_t _size1943; + ::apache::thrift::protocol::TType _etype1946; + xfer += iprot->readListBegin(_etype1946, _size1943); + (*(this->success)).resize(_size1943); + uint32_t _i1947; + for (_i1947 = 0; _i1947 < _size1943; ++_i1947) { - xfer += (*(this->success))[_i1935].read(iprot); + xfer += (*(this->success))[_i1947].read(iprot); } xfer += iprot->readListEnd(); } @@ -7901,14 +7901,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->primaryKeys.clear(); - uint32_t _size1936; - ::apache::thrift::protocol::TType _etype1939; - xfer += iprot->readListBegin(_etype1939, _size1936); - this->primaryKeys.resize(_size1936); - uint32_t _i1940; - for (_i1940 = 0; _i1940 < _size1936; ++_i1940) + uint32_t _size1948; + ::apache::thrift::protocol::TType _etype1951; + xfer += iprot->readListBegin(_etype1951, _size1948); + this->primaryKeys.resize(_size1948); + uint32_t _i1952; + for (_i1952 = 0; _i1952 < _size1948; ++_i1952) { - xfer += this->primaryKeys[_i1940].read(iprot); + xfer += this->primaryKeys[_i1952].read(iprot); } xfer += iprot->readListEnd(); } @@ -7921,14 +7921,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->foreignKeys.clear(); - uint32_t _size1941; - ::apache::thrift::protocol::TType _etype1944; - xfer += iprot->readListBegin(_etype1944, _size1941); - this->foreignKeys.resize(_size1941); - uint32_t _i1945; - for (_i1945 = 0; _i1945 < _size1941; ++_i1945) + uint32_t _size1953; + ::apache::thrift::protocol::TType _etype1956; + xfer += iprot->readListBegin(_etype1956, _size1953); + this->foreignKeys.resize(_size1953); + uint32_t _i1957; + for (_i1957 = 0; _i1957 < _size1953; ++_i1957) { - xfer += this->foreignKeys[_i1945].read(iprot); + xfer += this->foreignKeys[_i1957].read(iprot); } xfer += iprot->readListEnd(); } @@ -7941,14 +7941,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->uniqueConstraints.clear(); - uint32_t _size1946; - ::apache::thrift::protocol::TType _etype1949; - xfer += iprot->readListBegin(_etype1949, _size1946); - this->uniqueConstraints.resize(_size1946); - uint32_t _i1950; - for (_i1950 = 0; _i1950 < _size1946; ++_i1950) + uint32_t _size1958; + ::apache::thrift::protocol::TType _etype1961; + xfer += iprot->readListBegin(_etype1961, _size1958); + this->uniqueConstraints.resize(_size1958); + uint32_t _i1962; + for (_i1962 = 0; _i1962 < _size1958; ++_i1962) { - xfer += this->uniqueConstraints[_i1950].read(iprot); + xfer += this->uniqueConstraints[_i1962].read(iprot); } xfer += iprot->readListEnd(); } @@ -7961,14 +7961,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->notNullConstraints.clear(); - uint32_t _size1951; - ::apache::thrift::protocol::TType _etype1954; - xfer += iprot->readListBegin(_etype1954, _size1951); - this->notNullConstraints.resize(_size1951); - uint32_t _i1955; - for (_i1955 = 0; _i1955 < _size1951; ++_i1955) + uint32_t _size1963; + ::apache::thrift::protocol::TType _etype1966; + xfer += iprot->readListBegin(_etype1966, _size1963); + this->notNullConstraints.resize(_size1963); + uint32_t _i1967; + for (_i1967 = 0; _i1967 < _size1963; ++_i1967) { - xfer += this->notNullConstraints[_i1955].read(iprot); + xfer += this->notNullConstraints[_i1967].read(iprot); } xfer += iprot->readListEnd(); } @@ -7981,14 +7981,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->defaultConstraints.clear(); - uint32_t _size1956; - ::apache::thrift::protocol::TType _etype1959; - xfer += iprot->readListBegin(_etype1959, _size1956); - this->defaultConstraints.resize(_size1956); - uint32_t _i1960; - for (_i1960 = 0; _i1960 < _size1956; ++_i1960) + uint32_t _size1968; + ::apache::thrift::protocol::TType _etype1971; + xfer += iprot->readListBegin(_etype1971, _size1968); + this->defaultConstraints.resize(_size1968); + uint32_t _i1972; + for (_i1972 = 0; _i1972 < _size1968; ++_i1972) { - xfer += this->defaultConstraints[_i1960].read(iprot); + xfer += this->defaultConstraints[_i1972].read(iprot); } xfer += iprot->readListEnd(); } @@ -8001,14 +8001,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->checkConstraints.clear(); - uint32_t _size1961; - ::apache::thrift::protocol::TType _etype1964; - xfer += iprot->readListBegin(_etype1964, _size1961); - this->checkConstraints.resize(_size1961); - uint32_t _i1965; - for (_i1965 = 0; _i1965 < _size1961; ++_i1965) + uint32_t _size1973; + ::apache::thrift::protocol::TType _etype1976; + xfer += iprot->readListBegin(_etype1976, _size1973); + this->checkConstraints.resize(_size1973); + uint32_t _i1977; + for (_i1977 = 0; _i1977 < _size1973; ++_i1977) { - xfer += this->checkConstraints[_i1965].read(iprot); + xfer += this->checkConstraints[_i1977].read(iprot); } xfer += iprot->readListEnd(); } @@ -8041,10 +8041,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); - std::vector ::const_iterator _iter1966; - for (_iter1966 = this->primaryKeys.begin(); _iter1966 != this->primaryKeys.end(); ++_iter1966) + std::vector ::const_iterator _iter1978; + for (_iter1978 = this->primaryKeys.begin(); _iter1978 != this->primaryKeys.end(); ++_iter1978) { - xfer += (*_iter1966).write(oprot); + xfer += (*_iter1978).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8053,10 +8053,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); - std::vector ::const_iterator _iter1967; - for (_iter1967 = this->foreignKeys.begin(); _iter1967 != this->foreignKeys.end(); ++_iter1967) + std::vector ::const_iterator _iter1979; + for (_iter1979 = this->foreignKeys.begin(); _iter1979 != this->foreignKeys.end(); ++_iter1979) { - xfer += (*_iter1967).write(oprot); + xfer += (*_iter1979).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8065,10 +8065,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); - std::vector ::const_iterator _iter1968; - for (_iter1968 = this->uniqueConstraints.begin(); _iter1968 != this->uniqueConstraints.end(); ++_iter1968) + std::vector ::const_iterator _iter1980; + for (_iter1980 = this->uniqueConstraints.begin(); _iter1980 != this->uniqueConstraints.end(); ++_iter1980) { - xfer += (*_iter1968).write(oprot); + xfer += (*_iter1980).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8077,10 +8077,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); - std::vector ::const_iterator _iter1969; - for (_iter1969 = this->notNullConstraints.begin(); _iter1969 != this->notNullConstraints.end(); ++_iter1969) + std::vector ::const_iterator _iter1981; + for (_iter1981 = this->notNullConstraints.begin(); _iter1981 != this->notNullConstraints.end(); ++_iter1981) { - xfer += (*_iter1969).write(oprot); + xfer += (*_iter1981).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8089,10 +8089,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->defaultConstraints.size())); - std::vector ::const_iterator _iter1970; - for (_iter1970 = this->defaultConstraints.begin(); _iter1970 != this->defaultConstraints.end(); ++_iter1970) + std::vector ::const_iterator _iter1982; + for (_iter1982 = this->defaultConstraints.begin(); _iter1982 != this->defaultConstraints.end(); ++_iter1982) { - xfer += (*_iter1970).write(oprot); + xfer += (*_iter1982).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8101,10 +8101,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->checkConstraints.size())); - std::vector ::const_iterator _iter1971; - for (_iter1971 = this->checkConstraints.begin(); _iter1971 != this->checkConstraints.end(); ++_iter1971) + std::vector ::const_iterator _iter1983; + for (_iter1983 = this->checkConstraints.begin(); _iter1983 != this->checkConstraints.end(); ++_iter1983) { - xfer += (*_iter1971).write(oprot); + xfer += (*_iter1983).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8132,10 +8132,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->primaryKeys)).size())); - std::vector ::const_iterator _iter1972; - for (_iter1972 = (*(this->primaryKeys)).begin(); _iter1972 != (*(this->primaryKeys)).end(); ++_iter1972) + std::vector ::const_iterator _iter1984; + for (_iter1984 = (*(this->primaryKeys)).begin(); _iter1984 != (*(this->primaryKeys)).end(); ++_iter1984) { - xfer += (*_iter1972).write(oprot); + xfer += (*_iter1984).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8144,10 +8144,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->foreignKeys)).size())); - std::vector ::const_iterator _iter1973; - for (_iter1973 = (*(this->foreignKeys)).begin(); _iter1973 != (*(this->foreignKeys)).end(); ++_iter1973) + std::vector ::const_iterator _iter1985; + for (_iter1985 = (*(this->foreignKeys)).begin(); _iter1985 != (*(this->foreignKeys)).end(); ++_iter1985) { - xfer += (*_iter1973).write(oprot); + xfer += (*_iter1985).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8156,10 +8156,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->uniqueConstraints)).size())); - std::vector ::const_iterator _iter1974; - for (_iter1974 = (*(this->uniqueConstraints)).begin(); _iter1974 != (*(this->uniqueConstraints)).end(); ++_iter1974) + std::vector ::const_iterator _iter1986; + for (_iter1986 = (*(this->uniqueConstraints)).begin(); _iter1986 != (*(this->uniqueConstraints)).end(); ++_iter1986) { - xfer += (*_iter1974).write(oprot); + xfer += (*_iter1986).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8168,10 +8168,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->notNullConstraints)).size())); - std::vector ::const_iterator _iter1975; - for (_iter1975 = (*(this->notNullConstraints)).begin(); _iter1975 != (*(this->notNullConstraints)).end(); ++_iter1975) + std::vector ::const_iterator _iter1987; + for (_iter1987 = (*(this->notNullConstraints)).begin(); _iter1987 != (*(this->notNullConstraints)).end(); ++_iter1987) { - xfer += (*_iter1975).write(oprot); + xfer += (*_iter1987).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8180,10 +8180,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->defaultConstraints)).size())); - std::vector ::const_iterator _iter1976; - for (_iter1976 = (*(this->defaultConstraints)).begin(); _iter1976 != (*(this->defaultConstraints)).end(); ++_iter1976) + std::vector ::const_iterator _iter1988; + for (_iter1988 = (*(this->defaultConstraints)).begin(); _iter1988 != (*(this->defaultConstraints)).end(); ++_iter1988) { - xfer += (*_iter1976).write(oprot); + xfer += (*_iter1988).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8192,10 +8192,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->checkConstraints)).size())); - std::vector ::const_iterator _iter1977; - for (_iter1977 = (*(this->checkConstraints)).begin(); _iter1977 != (*(this->checkConstraints)).end(); ++_iter1977) + std::vector ::const_iterator _iter1989; + for (_iter1989 = (*(this->checkConstraints)).begin(); _iter1989 != (*(this->checkConstraints)).end(); ++_iter1989) { - xfer += (*_iter1977).write(oprot); + xfer += (*_iter1989).write(oprot); } xfer += oprot->writeListEnd(); } @@ -10877,14 +10877,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partNames.clear(); - uint32_t _size1978; - ::apache::thrift::protocol::TType _etype1981; - xfer += iprot->readListBegin(_etype1981, _size1978); - this->partNames.resize(_size1978); - uint32_t _i1982; - for (_i1982 = 0; _i1982 < _size1978; ++_i1982) + uint32_t _size1990; + ::apache::thrift::protocol::TType _etype1993; + xfer += iprot->readListBegin(_etype1993, _size1990); + this->partNames.resize(_size1990); + uint32_t _i1994; + for (_i1994 = 0; _i1994 < _size1990; ++_i1994) { - xfer += iprot->readString(this->partNames[_i1982]); + xfer += iprot->readString(this->partNames[_i1994]); } xfer += iprot->readListEnd(); } @@ -10921,10 +10921,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); - std::vector ::const_iterator _iter1983; - for (_iter1983 = this->partNames.begin(); _iter1983 != this->partNames.end(); ++_iter1983) + std::vector ::const_iterator _iter1995; + for (_iter1995 = this->partNames.begin(); _iter1995 != this->partNames.end(); ++_iter1995) { - xfer += oprot->writeString((*_iter1983)); + xfer += oprot->writeString((*_iter1995)); } xfer += oprot->writeListEnd(); } @@ -10956,10 +10956,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->partNames)).size())); - std::vector ::const_iterator _iter1984; - for (_iter1984 = (*(this->partNames)).begin(); _iter1984 != (*(this->partNames)).end(); ++_iter1984) + std::vector ::const_iterator _iter1996; + for (_iter1996 = (*(this->partNames)).begin(); _iter1996 != (*(this->partNames)).end(); ++_iter1996) { - xfer += oprot->writeString((*_iter1984)); + xfer += oprot->writeString((*_iter1996)); } xfer += oprot->writeListEnd(); } @@ -11410,14 +11410,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1985; - ::apache::thrift::protocol::TType _etype1988; - xfer += iprot->readListBegin(_etype1988, _size1985); - this->success.resize(_size1985); - uint32_t _i1989; - for (_i1989 = 0; _i1989 < _size1985; ++_i1989) + uint32_t _size1997; + ::apache::thrift::protocol::TType _etype2000; + xfer += iprot->readListBegin(_etype2000, _size1997); + this->success.resize(_size1997); + uint32_t _i2001; + for (_i2001 = 0; _i2001 < _size1997; ++_i2001) { - xfer += iprot->readString(this->success[_i1989]); + xfer += iprot->readString(this->success[_i2001]); } xfer += iprot->readListEnd(); } @@ -11456,10 +11456,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1990; - for (_iter1990 = this->success.begin(); _iter1990 != this->success.end(); ++_iter1990) + std::vector ::const_iterator _iter2002; + for (_iter2002 = this->success.begin(); _iter2002 != this->success.end(); ++_iter2002) { - xfer += oprot->writeString((*_iter1990)); + xfer += oprot->writeString((*_iter2002)); } xfer += oprot->writeListEnd(); } @@ -11504,14 +11504,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1991; - ::apache::thrift::protocol::TType _etype1994; - xfer += iprot->readListBegin(_etype1994, _size1991); - (*(this->success)).resize(_size1991); - uint32_t _i1995; - for (_i1995 = 0; _i1995 < _size1991; ++_i1995) + uint32_t _size2003; + ::apache::thrift::protocol::TType _etype2006; + xfer += iprot->readListBegin(_etype2006, _size2003); + (*(this->success)).resize(_size2003); + uint32_t _i2007; + for (_i2007 = 0; _i2007 < _size2003; ++_i2007) { - xfer += iprot->readString((*(this->success))[_i1995]); + xfer += iprot->readString((*(this->success))[_i2007]); } xfer += iprot->readListEnd(); } @@ -11681,14 +11681,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1996; - ::apache::thrift::protocol::TType _etype1999; - xfer += iprot->readListBegin(_etype1999, _size1996); - this->success.resize(_size1996); - uint32_t _i2000; - for (_i2000 = 0; _i2000 < _size1996; ++_i2000) + uint32_t _size2008; + ::apache::thrift::protocol::TType _etype2011; + xfer += iprot->readListBegin(_etype2011, _size2008); + this->success.resize(_size2008); + uint32_t _i2012; + for (_i2012 = 0; _i2012 < _size2008; ++_i2012) { - xfer += iprot->readString(this->success[_i2000]); + xfer += iprot->readString(this->success[_i2012]); } xfer += iprot->readListEnd(); } @@ -11727,10 +11727,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift:: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2001; - for (_iter2001 = this->success.begin(); _iter2001 != this->success.end(); ++_iter2001) + std::vector ::const_iterator _iter2013; + for (_iter2013 = this->success.begin(); _iter2013 != this->success.end(); ++_iter2013) { - xfer += oprot->writeString((*_iter2001)); + xfer += oprot->writeString((*_iter2013)); } xfer += oprot->writeListEnd(); } @@ -11775,14 +11775,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2002; - ::apache::thrift::protocol::TType _etype2005; - xfer += iprot->readListBegin(_etype2005, _size2002); - (*(this->success)).resize(_size2002); - uint32_t _i2006; - for (_i2006 = 0; _i2006 < _size2002; ++_i2006) + uint32_t _size2014; + ::apache::thrift::protocol::TType _etype2017; + xfer += iprot->readListBegin(_etype2017, _size2014); + (*(this->success)).resize(_size2014); + uint32_t _i2018; + for (_i2018 = 0; _i2018 < _size2014; ++_i2018) { - xfer += iprot->readString((*(this->success))[_i2006]); + xfer += iprot->readString((*(this->success))[_i2018]); } xfer += iprot->readListEnd(); } @@ -11899,14 +11899,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2007; - ::apache::thrift::protocol::TType _etype2010; - xfer += iprot->readListBegin(_etype2010, _size2007); - this->success.resize(_size2007); - uint32_t _i2011; - for (_i2011 = 0; _i2011 < _size2007; ++_i2011) + uint32_t _size2019; + ::apache::thrift::protocol::TType _etype2022; + xfer += iprot->readListBegin(_etype2022, _size2019); + this->success.resize(_size2019); + uint32_t _i2023; + for (_i2023 = 0; _i2023 < _size2019; ++_i2023) { - xfer += this->success[_i2011].read(iprot); + xfer += this->success[_i2023].read(iprot); } xfer += iprot->readListEnd(); } @@ -11945,10 +11945,10 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector
::const_iterator _iter2012; - for (_iter2012 = this->success.begin(); _iter2012 != this->success.end(); ++_iter2012) + std::vector
::const_iterator _iter2024; + for (_iter2024 = this->success.begin(); _iter2024 != this->success.end(); ++_iter2024) { - xfer += (*_iter2012).write(oprot); + xfer += (*_iter2024).write(oprot); } xfer += oprot->writeListEnd(); } @@ -11993,14 +11993,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_pre if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2013; - ::apache::thrift::protocol::TType _etype2016; - xfer += iprot->readListBegin(_etype2016, _size2013); - (*(this->success)).resize(_size2013); - uint32_t _i2017; - for (_i2017 = 0; _i2017 < _size2013; ++_i2017) + uint32_t _size2025; + ::apache::thrift::protocol::TType _etype2028; + xfer += iprot->readListBegin(_etype2028, _size2025); + (*(this->success)).resize(_size2025); + uint32_t _i2029; + for (_i2029 = 0; _i2029 < _size2025; ++_i2029) { - xfer += (*(this->success))[_i2017].read(iprot); + xfer += (*(this->success))[_i2029].read(iprot); } xfer += iprot->readListEnd(); } @@ -12138,14 +12138,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2018; - ::apache::thrift::protocol::TType _etype2021; - xfer += iprot->readListBegin(_etype2021, _size2018); - this->success.resize(_size2018); - uint32_t _i2022; - for (_i2022 = 0; _i2022 < _size2018; ++_i2022) + uint32_t _size2030; + ::apache::thrift::protocol::TType _etype2033; + xfer += iprot->readListBegin(_etype2033, _size2030); + this->success.resize(_size2030); + uint32_t _i2034; + for (_i2034 = 0; _i2034 < _size2030; ++_i2034) { - xfer += iprot->readString(this->success[_i2022]); + xfer += iprot->readString(this->success[_i2034]); } xfer += iprot->readListEnd(); } @@ -12184,10 +12184,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write( xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2023; - for (_iter2023 = this->success.begin(); _iter2023 != this->success.end(); ++_iter2023) + std::vector ::const_iterator _iter2035; + for (_iter2035 = this->success.begin(); _iter2035 != this->success.end(); ++_iter2035) { - xfer += oprot->writeString((*_iter2023)); + xfer += oprot->writeString((*_iter2035)); } xfer += oprot->writeListEnd(); } @@ -12232,14 +12232,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read( if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2024; - ::apache::thrift::protocol::TType _etype2027; - xfer += iprot->readListBegin(_etype2027, _size2024); - (*(this->success)).resize(_size2024); - uint32_t _i2028; - for (_i2028 = 0; _i2028 < _size2024; ++_i2028) + uint32_t _size2036; + ::apache::thrift::protocol::TType _etype2039; + xfer += iprot->readListBegin(_etype2039, _size2036); + (*(this->success)).resize(_size2036); + uint32_t _i2040; + for (_i2040 = 0; _i2040 < _size2036; ++_i2040) { - xfer += iprot->readString((*(this->success))[_i2028]); + xfer += iprot->readString((*(this->success))[_i2040]); } xfer += iprot->readListEnd(); } @@ -12314,14 +12314,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tbl_types.clear(); - uint32_t _size2029; - ::apache::thrift::protocol::TType _etype2032; - xfer += iprot->readListBegin(_etype2032, _size2029); - this->tbl_types.resize(_size2029); - uint32_t _i2033; - for (_i2033 = 0; _i2033 < _size2029; ++_i2033) + uint32_t _size2041; + ::apache::thrift::protocol::TType _etype2044; + xfer += iprot->readListBegin(_etype2044, _size2041); + this->tbl_types.resize(_size2041); + uint32_t _i2045; + for (_i2045 = 0; _i2045 < _size2041; ++_i2045) { - xfer += iprot->readString(this->tbl_types[_i2033]); + xfer += iprot->readString(this->tbl_types[_i2045]); } xfer += iprot->readListEnd(); } @@ -12358,10 +12358,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_types.size())); - std::vector ::const_iterator _iter2034; - for (_iter2034 = this->tbl_types.begin(); _iter2034 != this->tbl_types.end(); ++_iter2034) + std::vector ::const_iterator _iter2046; + for (_iter2046 = this->tbl_types.begin(); _iter2046 != this->tbl_types.end(); ++_iter2046) { - xfer += oprot->writeString((*_iter2034)); + xfer += oprot->writeString((*_iter2046)); } xfer += oprot->writeListEnd(); } @@ -12393,10 +12393,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_types)).size())); - std::vector ::const_iterator _iter2035; - for (_iter2035 = (*(this->tbl_types)).begin(); _iter2035 != (*(this->tbl_types)).end(); ++_iter2035) + std::vector ::const_iterator _iter2047; + for (_iter2047 = (*(this->tbl_types)).begin(); _iter2047 != (*(this->tbl_types)).end(); ++_iter2047) { - xfer += oprot->writeString((*_iter2035)); + xfer += oprot->writeString((*_iter2047)); } xfer += oprot->writeListEnd(); } @@ -12437,14 +12437,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2036; - ::apache::thrift::protocol::TType _etype2039; - xfer += iprot->readListBegin(_etype2039, _size2036); - this->success.resize(_size2036); - uint32_t _i2040; - for (_i2040 = 0; _i2040 < _size2036; ++_i2040) + uint32_t _size2048; + ::apache::thrift::protocol::TType _etype2051; + xfer += iprot->readListBegin(_etype2051, _size2048); + this->success.resize(_size2048); + uint32_t _i2052; + for (_i2052 = 0; _i2052 < _size2048; ++_i2052) { - xfer += this->success[_i2040].read(iprot); + xfer += this->success[_i2052].read(iprot); } xfer += iprot->readListEnd(); } @@ -12483,10 +12483,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2041; - for (_iter2041 = this->success.begin(); _iter2041 != this->success.end(); ++_iter2041) + std::vector ::const_iterator _iter2053; + for (_iter2053 = this->success.begin(); _iter2053 != this->success.end(); ++_iter2053) { - xfer += (*_iter2041).write(oprot); + xfer += (*_iter2053).write(oprot); } xfer += oprot->writeListEnd(); } @@ -12531,14 +12531,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2042; - ::apache::thrift::protocol::TType _etype2045; - xfer += iprot->readListBegin(_etype2045, _size2042); - (*(this->success)).resize(_size2042); - uint32_t _i2046; - for (_i2046 = 0; _i2046 < _size2042; ++_i2046) + uint32_t _size2054; + ::apache::thrift::protocol::TType _etype2057; + xfer += iprot->readListBegin(_etype2057, _size2054); + (*(this->success)).resize(_size2054); + uint32_t _i2058; + for (_i2058 = 0; _i2058 < _size2054; ++_i2058) { - xfer += (*(this->success))[_i2046].read(iprot); + xfer += (*(this->success))[_i2058].read(iprot); } xfer += iprot->readListEnd(); } @@ -12676,14 +12676,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2047; - ::apache::thrift::protocol::TType _etype2050; - xfer += iprot->readListBegin(_etype2050, _size2047); - this->success.resize(_size2047); - uint32_t _i2051; - for (_i2051 = 0; _i2051 < _size2047; ++_i2051) + uint32_t _size2059; + ::apache::thrift::protocol::TType _etype2062; + xfer += iprot->readListBegin(_etype2062, _size2059); + this->success.resize(_size2059); + uint32_t _i2063; + for (_i2063 = 0; _i2063 < _size2059; ++_i2063) { - xfer += iprot->readString(this->success[_i2051]); + xfer += iprot->readString(this->success[_i2063]); } xfer += iprot->readListEnd(); } @@ -12722,10 +12722,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2052; - for (_iter2052 = this->success.begin(); _iter2052 != this->success.end(); ++_iter2052) + std::vector ::const_iterator _iter2064; + for (_iter2064 = this->success.begin(); _iter2064 != this->success.end(); ++_iter2064) { - xfer += oprot->writeString((*_iter2052)); + xfer += oprot->writeString((*_iter2064)); } xfer += oprot->writeListEnd(); } @@ -12770,14 +12770,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2053; - ::apache::thrift::protocol::TType _etype2056; - xfer += iprot->readListBegin(_etype2056, _size2053); - (*(this->success)).resize(_size2053); - uint32_t _i2057; - for (_i2057 = 0; _i2057 < _size2053; ++_i2057) + uint32_t _size2065; + ::apache::thrift::protocol::TType _etype2068; + xfer += iprot->readListBegin(_etype2068, _size2065); + (*(this->success)).resize(_size2065); + uint32_t _i2069; + for (_i2069 = 0; _i2069 < _size2065; ++_i2069) { - xfer += iprot->readString((*(this->success))[_i2057]); + xfer += iprot->readString((*(this->success))[_i2069]); } xfer += iprot->readListEnd(); } @@ -13087,14 +13087,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tbl_names.clear(); - uint32_t _size2058; - ::apache::thrift::protocol::TType _etype2061; - xfer += iprot->readListBegin(_etype2061, _size2058); - this->tbl_names.resize(_size2058); - uint32_t _i2062; - for (_i2062 = 0; _i2062 < _size2058; ++_i2062) + uint32_t _size2070; + ::apache::thrift::protocol::TType _etype2073; + xfer += iprot->readListBegin(_etype2073, _size2070); + this->tbl_names.resize(_size2070); + uint32_t _i2074; + for (_i2074 = 0; _i2074 < _size2070; ++_i2074) { - xfer += iprot->readString(this->tbl_names[_i2062]); + xfer += iprot->readString(this->tbl_names[_i2074]); } xfer += iprot->readListEnd(); } @@ -13127,10 +13127,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_names.size())); - std::vector ::const_iterator _iter2063; - for (_iter2063 = this->tbl_names.begin(); _iter2063 != this->tbl_names.end(); ++_iter2063) + std::vector ::const_iterator _iter2075; + for (_iter2075 = this->tbl_names.begin(); _iter2075 != this->tbl_names.end(); ++_iter2075) { - xfer += oprot->writeString((*_iter2063)); + xfer += oprot->writeString((*_iter2075)); } xfer += oprot->writeListEnd(); } @@ -13158,10 +13158,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_names)).size())); - std::vector ::const_iterator _iter2064; - for (_iter2064 = (*(this->tbl_names)).begin(); _iter2064 != (*(this->tbl_names)).end(); ++_iter2064) + std::vector ::const_iterator _iter2076; + for (_iter2076 = (*(this->tbl_names)).begin(); _iter2076 != (*(this->tbl_names)).end(); ++_iter2076) { - xfer += oprot->writeString((*_iter2064)); + xfer += oprot->writeString((*_iter2076)); } xfer += oprot->writeListEnd(); } @@ -13202,14 +13202,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2065; - ::apache::thrift::protocol::TType _etype2068; - xfer += iprot->readListBegin(_etype2068, _size2065); - this->success.resize(_size2065); - uint32_t _i2069; - for (_i2069 = 0; _i2069 < _size2065; ++_i2069) + uint32_t _size2077; + ::apache::thrift::protocol::TType _etype2080; + xfer += iprot->readListBegin(_etype2080, _size2077); + this->success.resize(_size2077); + uint32_t _i2081; + for (_i2081 = 0; _i2081 < _size2077; ++_i2081) { - xfer += this->success[_i2069].read(iprot); + xfer += this->success[_i2081].read(iprot); } xfer += iprot->readListEnd(); } @@ -13240,10 +13240,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector
::const_iterator _iter2070; - for (_iter2070 = this->success.begin(); _iter2070 != this->success.end(); ++_iter2070) + std::vector
::const_iterator _iter2082; + for (_iter2082 = this->success.begin(); _iter2082 != this->success.end(); ++_iter2082) { - xfer += (*_iter2070).write(oprot); + xfer += (*_iter2082).write(oprot); } xfer += oprot->writeListEnd(); } @@ -13284,14 +13284,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2071; - ::apache::thrift::protocol::TType _etype2074; - xfer += iprot->readListBegin(_etype2074, _size2071); - (*(this->success)).resize(_size2071); - uint32_t _i2075; - for (_i2075 = 0; _i2075 < _size2071; ++_i2075) + uint32_t _size2083; + ::apache::thrift::protocol::TType _etype2086; + xfer += iprot->readListBegin(_etype2086, _size2083); + (*(this->success)).resize(_size2083); + uint32_t _i2087; + for (_i2087 = 0; _i2087 < _size2083; ++_i2087) { - xfer += (*(this->success))[_i2075].read(iprot); + xfer += (*(this->success))[_i2087].read(iprot); } xfer += iprot->readListEnd(); } @@ -13421,14 +13421,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2076; - ::apache::thrift::protocol::TType _etype2079; - xfer += iprot->readListBegin(_etype2079, _size2076); - this->success.resize(_size2076); - uint32_t _i2080; - for (_i2080 = 0; _i2080 < _size2076; ++_i2080) + uint32_t _size2088; + ::apache::thrift::protocol::TType _etype2091; + xfer += iprot->readListBegin(_etype2091, _size2088); + this->success.resize(_size2088); + uint32_t _i2092; + for (_i2092 = 0; _i2092 < _size2088; ++_i2092) { - xfer += this->success[_i2080].read(iprot); + xfer += this->success[_i2092].read(iprot); } xfer += iprot->readListEnd(); } @@ -13467,10 +13467,10 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2081; - for (_iter2081 = this->success.begin(); _iter2081 != this->success.end(); ++_iter2081) + std::vector ::const_iterator _iter2093; + for (_iter2093 = this->success.begin(); _iter2093 != this->success.end(); ++_iter2093) { - xfer += (*_iter2081).write(oprot); + xfer += (*_iter2093).write(oprot); } xfer += oprot->writeListEnd(); } @@ -13515,14 +13515,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2082; - ::apache::thrift::protocol::TType _etype2085; - xfer += iprot->readListBegin(_etype2085, _size2082); - (*(this->success)).resize(_size2082); - uint32_t _i2086; - for (_i2086 = 0; _i2086 < _size2082; ++_i2086) + uint32_t _size2094; + ::apache::thrift::protocol::TType _etype2097; + xfer += iprot->readListBegin(_etype2097, _size2094); + (*(this->success)).resize(_size2094); + uint32_t _i2098; + for (_i2098 = 0; _i2098 < _size2094; ++_i2098) { - xfer += (*(this->success))[_i2086].read(iprot); + xfer += (*(this->success))[_i2098].read(iprot); } xfer += iprot->readListEnd(); } @@ -14704,14 +14704,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2087; - ::apache::thrift::protocol::TType _etype2090; - xfer += iprot->readListBegin(_etype2090, _size2087); - this->success.resize(_size2087); - uint32_t _i2091; - for (_i2091 = 0; _i2091 < _size2087; ++_i2091) + uint32_t _size2099; + ::apache::thrift::protocol::TType _etype2102; + xfer += iprot->readListBegin(_etype2102, _size2099); + this->success.resize(_size2099); + uint32_t _i2103; + for (_i2103 = 0; _i2103 < _size2099; ++_i2103) { - xfer += iprot->readString(this->success[_i2091]); + xfer += iprot->readString(this->success[_i2103]); } xfer += iprot->readListEnd(); } @@ -14766,10 +14766,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2092; - for (_iter2092 = this->success.begin(); _iter2092 != this->success.end(); ++_iter2092) + std::vector ::const_iterator _iter2104; + for (_iter2104 = this->success.begin(); _iter2104 != this->success.end(); ++_iter2104) { - xfer += oprot->writeString((*_iter2092)); + xfer += oprot->writeString((*_iter2104)); } xfer += oprot->writeListEnd(); } @@ -14822,14 +14822,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2093; - ::apache::thrift::protocol::TType _etype2096; - xfer += iprot->readListBegin(_etype2096, _size2093); - (*(this->success)).resize(_size2093); - uint32_t _i2097; - for (_i2097 = 0; _i2097 < _size2093; ++_i2097) + uint32_t _size2105; + ::apache::thrift::protocol::TType _etype2108; + xfer += iprot->readListBegin(_etype2108, _size2105); + (*(this->success)).resize(_size2105); + uint32_t _i2109; + for (_i2109 = 0; _i2109 < _size2105; ++_i2109) { - xfer += iprot->readString((*(this->success))[_i2097]); + xfer += iprot->readString((*(this->success))[_i2109]); } xfer += iprot->readListEnd(); } @@ -16390,14 +16390,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2098; - ::apache::thrift::protocol::TType _etype2101; - xfer += iprot->readListBegin(_etype2101, _size2098); - this->new_parts.resize(_size2098); - uint32_t _i2102; - for (_i2102 = 0; _i2102 < _size2098; ++_i2102) + uint32_t _size2110; + ::apache::thrift::protocol::TType _etype2113; + xfer += iprot->readListBegin(_etype2113, _size2110); + this->new_parts.resize(_size2110); + uint32_t _i2114; + for (_i2114 = 0; _i2114 < _size2110; ++_i2114) { - xfer += this->new_parts[_i2102].read(iprot); + xfer += this->new_parts[_i2114].read(iprot); } xfer += iprot->readListEnd(); } @@ -16426,10 +16426,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2103; - for (_iter2103 = this->new_parts.begin(); _iter2103 != this->new_parts.end(); ++_iter2103) + std::vector ::const_iterator _iter2115; + for (_iter2115 = this->new_parts.begin(); _iter2115 != this->new_parts.end(); ++_iter2115) { - xfer += (*_iter2103).write(oprot); + xfer += (*_iter2115).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16453,10 +16453,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2104; - for (_iter2104 = (*(this->new_parts)).begin(); _iter2104 != (*(this->new_parts)).end(); ++_iter2104) + std::vector ::const_iterator _iter2116; + for (_iter2116 = (*(this->new_parts)).begin(); _iter2116 != (*(this->new_parts)).end(); ++_iter2116) { - xfer += (*_iter2104).write(oprot); + xfer += (*_iter2116).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16665,14 +16665,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2105; - ::apache::thrift::protocol::TType _etype2108; - xfer += iprot->readListBegin(_etype2108, _size2105); - this->new_parts.resize(_size2105); - uint32_t _i2109; - for (_i2109 = 0; _i2109 < _size2105; ++_i2109) + uint32_t _size2117; + ::apache::thrift::protocol::TType _etype2120; + xfer += iprot->readListBegin(_etype2120, _size2117); + this->new_parts.resize(_size2117); + uint32_t _i2121; + for (_i2121 = 0; _i2121 < _size2117; ++_i2121) { - xfer += this->new_parts[_i2109].read(iprot); + xfer += this->new_parts[_i2121].read(iprot); } xfer += iprot->readListEnd(); } @@ -16701,10 +16701,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift:: xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2110; - for (_iter2110 = this->new_parts.begin(); _iter2110 != this->new_parts.end(); ++_iter2110) + std::vector ::const_iterator _iter2122; + for (_iter2122 = this->new_parts.begin(); _iter2122 != this->new_parts.end(); ++_iter2122) { - xfer += (*_iter2110).write(oprot); + xfer += (*_iter2122).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16728,10 +16728,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift: xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2111; - for (_iter2111 = (*(this->new_parts)).begin(); _iter2111 != (*(this->new_parts)).end(); ++_iter2111) + std::vector ::const_iterator _iter2123; + for (_iter2123 = (*(this->new_parts)).begin(); _iter2123 != (*(this->new_parts)).end(); ++_iter2123) { - xfer += (*_iter2111).write(oprot); + xfer += (*_iter2123).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16956,14 +16956,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2112; - ::apache::thrift::protocol::TType _etype2115; - xfer += iprot->readListBegin(_etype2115, _size2112); - this->part_vals.resize(_size2112); - uint32_t _i2116; - for (_i2116 = 0; _i2116 < _size2112; ++_i2116) + uint32_t _size2124; + ::apache::thrift::protocol::TType _etype2127; + xfer += iprot->readListBegin(_etype2127, _size2124); + this->part_vals.resize(_size2124); + uint32_t _i2128; + for (_i2128 = 0; _i2128 < _size2124; ++_i2128) { - xfer += iprot->readString(this->part_vals[_i2116]); + xfer += iprot->readString(this->part_vals[_i2128]); } xfer += iprot->readListEnd(); } @@ -17000,10 +17000,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2117; - for (_iter2117 = this->part_vals.begin(); _iter2117 != this->part_vals.end(); ++_iter2117) + std::vector ::const_iterator _iter2129; + for (_iter2129 = this->part_vals.begin(); _iter2129 != this->part_vals.end(); ++_iter2129) { - xfer += oprot->writeString((*_iter2117)); + xfer += oprot->writeString((*_iter2129)); } xfer += oprot->writeListEnd(); } @@ -17035,10 +17035,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2118; - for (_iter2118 = (*(this->part_vals)).begin(); _iter2118 != (*(this->part_vals)).end(); ++_iter2118) + std::vector ::const_iterator _iter2130; + for (_iter2130 = (*(this->part_vals)).begin(); _iter2130 != (*(this->part_vals)).end(); ++_iter2130) { - xfer += oprot->writeString((*_iter2118)); + xfer += oprot->writeString((*_iter2130)); } xfer += oprot->writeListEnd(); } @@ -17510,14 +17510,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2119; - ::apache::thrift::protocol::TType _etype2122; - xfer += iprot->readListBegin(_etype2122, _size2119); - this->part_vals.resize(_size2119); - uint32_t _i2123; - for (_i2123 = 0; _i2123 < _size2119; ++_i2123) + uint32_t _size2131; + ::apache::thrift::protocol::TType _etype2134; + xfer += iprot->readListBegin(_etype2134, _size2131); + this->part_vals.resize(_size2131); + uint32_t _i2135; + for (_i2135 = 0; _i2135 < _size2131; ++_i2135) { - xfer += iprot->readString(this->part_vals[_i2123]); + xfer += iprot->readString(this->part_vals[_i2135]); } xfer += iprot->readListEnd(); } @@ -17562,10 +17562,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2124; - for (_iter2124 = this->part_vals.begin(); _iter2124 != this->part_vals.end(); ++_iter2124) + std::vector ::const_iterator _iter2136; + for (_iter2136 = this->part_vals.begin(); _iter2136 != this->part_vals.end(); ++_iter2136) { - xfer += oprot->writeString((*_iter2124)); + xfer += oprot->writeString((*_iter2136)); } xfer += oprot->writeListEnd(); } @@ -17601,10 +17601,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2125; - for (_iter2125 = (*(this->part_vals)).begin(); _iter2125 != (*(this->part_vals)).end(); ++_iter2125) + std::vector ::const_iterator _iter2137; + for (_iter2137 = (*(this->part_vals)).begin(); _iter2137 != (*(this->part_vals)).end(); ++_iter2137) { - xfer += oprot->writeString((*_iter2125)); + xfer += oprot->writeString((*_iter2137)); } xfer += oprot->writeListEnd(); } @@ -18407,14 +18407,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2126; - ::apache::thrift::protocol::TType _etype2129; - xfer += iprot->readListBegin(_etype2129, _size2126); - this->part_vals.resize(_size2126); - uint32_t _i2130; - for (_i2130 = 0; _i2130 < _size2126; ++_i2130) + uint32_t _size2138; + ::apache::thrift::protocol::TType _etype2141; + xfer += iprot->readListBegin(_etype2141, _size2138); + this->part_vals.resize(_size2138); + uint32_t _i2142; + for (_i2142 = 0; _i2142 < _size2138; ++_i2142) { - xfer += iprot->readString(this->part_vals[_i2130]); + xfer += iprot->readString(this->part_vals[_i2142]); } xfer += iprot->readListEnd(); } @@ -18459,10 +18459,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2131; - for (_iter2131 = this->part_vals.begin(); _iter2131 != this->part_vals.end(); ++_iter2131) + std::vector ::const_iterator _iter2143; + for (_iter2143 = this->part_vals.begin(); _iter2143 != this->part_vals.end(); ++_iter2143) { - xfer += oprot->writeString((*_iter2131)); + xfer += oprot->writeString((*_iter2143)); } xfer += oprot->writeListEnd(); } @@ -18498,10 +18498,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2132; - for (_iter2132 = (*(this->part_vals)).begin(); _iter2132 != (*(this->part_vals)).end(); ++_iter2132) + std::vector ::const_iterator _iter2144; + for (_iter2144 = (*(this->part_vals)).begin(); _iter2144 != (*(this->part_vals)).end(); ++_iter2144) { - xfer += oprot->writeString((*_iter2132)); + xfer += oprot->writeString((*_iter2144)); } xfer += oprot->writeListEnd(); } @@ -18710,14 +18710,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read( if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2133; - ::apache::thrift::protocol::TType _etype2136; - xfer += iprot->readListBegin(_etype2136, _size2133); - this->part_vals.resize(_size2133); - uint32_t _i2137; - for (_i2137 = 0; _i2137 < _size2133; ++_i2137) + uint32_t _size2145; + ::apache::thrift::protocol::TType _etype2148; + xfer += iprot->readListBegin(_etype2148, _size2145); + this->part_vals.resize(_size2145); + uint32_t _i2149; + for (_i2149 = 0; _i2149 < _size2145; ++_i2149) { - xfer += iprot->readString(this->part_vals[_i2137]); + xfer += iprot->readString(this->part_vals[_i2149]); } xfer += iprot->readListEnd(); } @@ -18770,10 +18770,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2138; - for (_iter2138 = this->part_vals.begin(); _iter2138 != this->part_vals.end(); ++_iter2138) + std::vector ::const_iterator _iter2150; + for (_iter2150 = this->part_vals.begin(); _iter2150 != this->part_vals.end(); ++_iter2150) { - xfer += oprot->writeString((*_iter2138)); + xfer += oprot->writeString((*_iter2150)); } xfer += oprot->writeListEnd(); } @@ -18813,10 +18813,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2139; - for (_iter2139 = (*(this->part_vals)).begin(); _iter2139 != (*(this->part_vals)).end(); ++_iter2139) + std::vector ::const_iterator _iter2151; + for (_iter2151 = (*(this->part_vals)).begin(); _iter2151 != (*(this->part_vals)).end(); ++_iter2151) { - xfer += oprot->writeString((*_iter2139)); + xfer += oprot->writeString((*_iter2151)); } xfer += oprot->writeListEnd(); } @@ -19822,14 +19822,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2140; - ::apache::thrift::protocol::TType _etype2143; - xfer += iprot->readListBegin(_etype2143, _size2140); - this->part_vals.resize(_size2140); - uint32_t _i2144; - for (_i2144 = 0; _i2144 < _size2140; ++_i2144) + uint32_t _size2152; + ::apache::thrift::protocol::TType _etype2155; + xfer += iprot->readListBegin(_etype2155, _size2152); + this->part_vals.resize(_size2152); + uint32_t _i2156; + for (_i2156 = 0; _i2156 < _size2152; ++_i2156) { - xfer += iprot->readString(this->part_vals[_i2144]); + xfer += iprot->readString(this->part_vals[_i2156]); } xfer += iprot->readListEnd(); } @@ -19866,10 +19866,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2145; - for (_iter2145 = this->part_vals.begin(); _iter2145 != this->part_vals.end(); ++_iter2145) + std::vector ::const_iterator _iter2157; + for (_iter2157 = this->part_vals.begin(); _iter2157 != this->part_vals.end(); ++_iter2157) { - xfer += oprot->writeString((*_iter2145)); + xfer += oprot->writeString((*_iter2157)); } xfer += oprot->writeListEnd(); } @@ -19901,10 +19901,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2146; - for (_iter2146 = (*(this->part_vals)).begin(); _iter2146 != (*(this->part_vals)).end(); ++_iter2146) + std::vector ::const_iterator _iter2158; + for (_iter2158 = (*(this->part_vals)).begin(); _iter2158 != (*(this->part_vals)).end(); ++_iter2158) { - xfer += oprot->writeString((*_iter2146)); + xfer += oprot->writeString((*_iter2158)); } xfer += oprot->writeListEnd(); } @@ -20320,17 +20320,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_MAP) { { this->partitionSpecs.clear(); - uint32_t _size2147; - ::apache::thrift::protocol::TType _ktype2148; - ::apache::thrift::protocol::TType _vtype2149; - xfer += iprot->readMapBegin(_ktype2148, _vtype2149, _size2147); - uint32_t _i2151; - for (_i2151 = 0; _i2151 < _size2147; ++_i2151) + uint32_t _size2159; + ::apache::thrift::protocol::TType _ktype2160; + ::apache::thrift::protocol::TType _vtype2161; + xfer += iprot->readMapBegin(_ktype2160, _vtype2161, _size2159); + uint32_t _i2163; + for (_i2163 = 0; _i2163 < _size2159; ++_i2163) { - std::string _key2152; - xfer += iprot->readString(_key2152); - std::string& _val2153 = this->partitionSpecs[_key2152]; - xfer += iprot->readString(_val2153); + std::string _key2164; + xfer += iprot->readString(_key2164); + std::string& _val2165 = this->partitionSpecs[_key2164]; + xfer += iprot->readString(_val2165); } xfer += iprot->readMapEnd(); } @@ -20391,11 +20391,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); - std::map ::const_iterator _iter2154; - for (_iter2154 = this->partitionSpecs.begin(); _iter2154 != this->partitionSpecs.end(); ++_iter2154) + std::map ::const_iterator _iter2166; + for (_iter2166 = this->partitionSpecs.begin(); _iter2166 != this->partitionSpecs.end(); ++_iter2166) { - xfer += oprot->writeString(_iter2154->first); - xfer += oprot->writeString(_iter2154->second); + xfer += oprot->writeString(_iter2166->first); + xfer += oprot->writeString(_iter2166->second); } xfer += oprot->writeMapEnd(); } @@ -20435,11 +20435,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); - std::map ::const_iterator _iter2155; - for (_iter2155 = (*(this->partitionSpecs)).begin(); _iter2155 != (*(this->partitionSpecs)).end(); ++_iter2155) + std::map ::const_iterator _iter2167; + for (_iter2167 = (*(this->partitionSpecs)).begin(); _iter2167 != (*(this->partitionSpecs)).end(); ++_iter2167) { - xfer += oprot->writeString(_iter2155->first); - xfer += oprot->writeString(_iter2155->second); + xfer += oprot->writeString(_iter2167->first); + xfer += oprot->writeString(_iter2167->second); } xfer += oprot->writeMapEnd(); } @@ -20684,17 +20684,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_MAP) { { this->partitionSpecs.clear(); - uint32_t _size2156; - ::apache::thrift::protocol::TType _ktype2157; - ::apache::thrift::protocol::TType _vtype2158; - xfer += iprot->readMapBegin(_ktype2157, _vtype2158, _size2156); - uint32_t _i2160; - for (_i2160 = 0; _i2160 < _size2156; ++_i2160) + uint32_t _size2168; + ::apache::thrift::protocol::TType _ktype2169; + ::apache::thrift::protocol::TType _vtype2170; + xfer += iprot->readMapBegin(_ktype2169, _vtype2170, _size2168); + uint32_t _i2172; + for (_i2172 = 0; _i2172 < _size2168; ++_i2172) { - std::string _key2161; - xfer += iprot->readString(_key2161); - std::string& _val2162 = this->partitionSpecs[_key2161]; - xfer += iprot->readString(_val2162); + std::string _key2173; + xfer += iprot->readString(_key2173); + std::string& _val2174 = this->partitionSpecs[_key2173]; + xfer += iprot->readString(_val2174); } xfer += iprot->readMapEnd(); } @@ -20755,11 +20755,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); - std::map ::const_iterator _iter2163; - for (_iter2163 = this->partitionSpecs.begin(); _iter2163 != this->partitionSpecs.end(); ++_iter2163) + std::map ::const_iterator _iter2175; + for (_iter2175 = this->partitionSpecs.begin(); _iter2175 != this->partitionSpecs.end(); ++_iter2175) { - xfer += oprot->writeString(_iter2163->first); - xfer += oprot->writeString(_iter2163->second); + xfer += oprot->writeString(_iter2175->first); + xfer += oprot->writeString(_iter2175->second); } xfer += oprot->writeMapEnd(); } @@ -20799,11 +20799,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift:: xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); - std::map ::const_iterator _iter2164; - for (_iter2164 = (*(this->partitionSpecs)).begin(); _iter2164 != (*(this->partitionSpecs)).end(); ++_iter2164) + std::map ::const_iterator _iter2176; + for (_iter2176 = (*(this->partitionSpecs)).begin(); _iter2176 != (*(this->partitionSpecs)).end(); ++_iter2176) { - xfer += oprot->writeString(_iter2164->first); - xfer += oprot->writeString(_iter2164->second); + xfer += oprot->writeString(_iter2176->first); + xfer += oprot->writeString(_iter2176->second); } xfer += oprot->writeMapEnd(); } @@ -20860,14 +20860,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2165; - ::apache::thrift::protocol::TType _etype2168; - xfer += iprot->readListBegin(_etype2168, _size2165); - this->success.resize(_size2165); - uint32_t _i2169; - for (_i2169 = 0; _i2169 < _size2165; ++_i2169) + uint32_t _size2177; + ::apache::thrift::protocol::TType _etype2180; + xfer += iprot->readListBegin(_etype2180, _size2177); + this->success.resize(_size2177); + uint32_t _i2181; + for (_i2181 = 0; _i2181 < _size2177; ++_i2181) { - xfer += this->success[_i2169].read(iprot); + xfer += this->success[_i2181].read(iprot); } xfer += iprot->readListEnd(); } @@ -20930,10 +20930,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2170; - for (_iter2170 = this->success.begin(); _iter2170 != this->success.end(); ++_iter2170) + std::vector ::const_iterator _iter2182; + for (_iter2182 = this->success.begin(); _iter2182 != this->success.end(); ++_iter2182) { - xfer += (*_iter2170).write(oprot); + xfer += (*_iter2182).write(oprot); } xfer += oprot->writeListEnd(); } @@ -20990,14 +20990,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2171; - ::apache::thrift::protocol::TType _etype2174; - xfer += iprot->readListBegin(_etype2174, _size2171); - (*(this->success)).resize(_size2171); - uint32_t _i2175; - for (_i2175 = 0; _i2175 < _size2171; ++_i2175) + uint32_t _size2183; + ::apache::thrift::protocol::TType _etype2186; + xfer += iprot->readListBegin(_etype2186, _size2183); + (*(this->success)).resize(_size2183); + uint32_t _i2187; + for (_i2187 = 0; _i2187 < _size2183; ++_i2187) { - xfer += (*(this->success))[_i2175].read(iprot); + xfer += (*(this->success))[_i2187].read(iprot); } xfer += iprot->readListEnd(); } @@ -21096,14 +21096,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2176; - ::apache::thrift::protocol::TType _etype2179; - xfer += iprot->readListBegin(_etype2179, _size2176); - this->part_vals.resize(_size2176); - uint32_t _i2180; - for (_i2180 = 0; _i2180 < _size2176; ++_i2180) + uint32_t _size2188; + ::apache::thrift::protocol::TType _etype2191; + xfer += iprot->readListBegin(_etype2191, _size2188); + this->part_vals.resize(_size2188); + uint32_t _i2192; + for (_i2192 = 0; _i2192 < _size2188; ++_i2192) { - xfer += iprot->readString(this->part_vals[_i2180]); + xfer += iprot->readString(this->part_vals[_i2192]); } xfer += iprot->readListEnd(); } @@ -21124,14 +21124,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2181; - ::apache::thrift::protocol::TType _etype2184; - xfer += iprot->readListBegin(_etype2184, _size2181); - this->group_names.resize(_size2181); - uint32_t _i2185; - for (_i2185 = 0; _i2185 < _size2181; ++_i2185) + uint32_t _size2193; + ::apache::thrift::protocol::TType _etype2196; + xfer += iprot->readListBegin(_etype2196, _size2193); + this->group_names.resize(_size2193); + uint32_t _i2197; + for (_i2197 = 0; _i2197 < _size2193; ++_i2197) { - xfer += iprot->readString(this->group_names[_i2185]); + xfer += iprot->readString(this->group_names[_i2197]); } xfer += iprot->readListEnd(); } @@ -21168,10 +21168,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2186; - for (_iter2186 = this->part_vals.begin(); _iter2186 != this->part_vals.end(); ++_iter2186) + std::vector ::const_iterator _iter2198; + for (_iter2198 = this->part_vals.begin(); _iter2198 != this->part_vals.end(); ++_iter2198) { - xfer += oprot->writeString((*_iter2186)); + xfer += oprot->writeString((*_iter2198)); } xfer += oprot->writeListEnd(); } @@ -21184,10 +21184,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2187; - for (_iter2187 = this->group_names.begin(); _iter2187 != this->group_names.end(); ++_iter2187) + std::vector ::const_iterator _iter2199; + for (_iter2199 = this->group_names.begin(); _iter2199 != this->group_names.end(); ++_iter2199) { - xfer += oprot->writeString((*_iter2187)); + xfer += oprot->writeString((*_iter2199)); } xfer += oprot->writeListEnd(); } @@ -21219,10 +21219,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2188; - for (_iter2188 = (*(this->part_vals)).begin(); _iter2188 != (*(this->part_vals)).end(); ++_iter2188) + std::vector ::const_iterator _iter2200; + for (_iter2200 = (*(this->part_vals)).begin(); _iter2200 != (*(this->part_vals)).end(); ++_iter2200) { - xfer += oprot->writeString((*_iter2188)); + xfer += oprot->writeString((*_iter2200)); } xfer += oprot->writeListEnd(); } @@ -21235,10 +21235,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2189; - for (_iter2189 = (*(this->group_names)).begin(); _iter2189 != (*(this->group_names)).end(); ++_iter2189) + std::vector ::const_iterator _iter2201; + for (_iter2201 = (*(this->group_names)).begin(); _iter2201 != (*(this->group_names)).end(); ++_iter2201) { - xfer += oprot->writeString((*_iter2189)); + xfer += oprot->writeString((*_iter2201)); } xfer += oprot->writeListEnd(); } @@ -21797,14 +21797,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2190; - ::apache::thrift::protocol::TType _etype2193; - xfer += iprot->readListBegin(_etype2193, _size2190); - this->success.resize(_size2190); - uint32_t _i2194; - for (_i2194 = 0; _i2194 < _size2190; ++_i2194) + uint32_t _size2202; + ::apache::thrift::protocol::TType _etype2205; + xfer += iprot->readListBegin(_etype2205, _size2202); + this->success.resize(_size2202); + uint32_t _i2206; + for (_i2206 = 0; _i2206 < _size2202; ++_i2206) { - xfer += this->success[_i2194].read(iprot); + xfer += this->success[_i2206].read(iprot); } xfer += iprot->readListEnd(); } @@ -21851,10 +21851,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2195; - for (_iter2195 = this->success.begin(); _iter2195 != this->success.end(); ++_iter2195) + std::vector ::const_iterator _iter2207; + for (_iter2207 = this->success.begin(); _iter2207 != this->success.end(); ++_iter2207) { - xfer += (*_iter2195).write(oprot); + xfer += (*_iter2207).write(oprot); } xfer += oprot->writeListEnd(); } @@ -21903,14 +21903,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2196; - ::apache::thrift::protocol::TType _etype2199; - xfer += iprot->readListBegin(_etype2199, _size2196); - (*(this->success)).resize(_size2196); - uint32_t _i2200; - for (_i2200 = 0; _i2200 < _size2196; ++_i2200) + uint32_t _size2208; + ::apache::thrift::protocol::TType _etype2211; + xfer += iprot->readListBegin(_etype2211, _size2208); + (*(this->success)).resize(_size2208); + uint32_t _i2212; + for (_i2212 = 0; _i2212 < _size2208; ++_i2212) { - xfer += (*(this->success))[_i2200].read(iprot); + xfer += (*(this->success))[_i2212].read(iprot); } xfer += iprot->readListEnd(); } @@ -22236,14 +22236,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2201; - ::apache::thrift::protocol::TType _etype2204; - xfer += iprot->readListBegin(_etype2204, _size2201); - this->group_names.resize(_size2201); - uint32_t _i2205; - for (_i2205 = 0; _i2205 < _size2201; ++_i2205) + uint32_t _size2213; + ::apache::thrift::protocol::TType _etype2216; + xfer += iprot->readListBegin(_etype2216, _size2213); + this->group_names.resize(_size2213); + uint32_t _i2217; + for (_i2217 = 0; _i2217 < _size2213; ++_i2217) { - xfer += iprot->readString(this->group_names[_i2205]); + xfer += iprot->readString(this->group_names[_i2217]); } xfer += iprot->readListEnd(); } @@ -22288,10 +22288,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2206; - for (_iter2206 = this->group_names.begin(); _iter2206 != this->group_names.end(); ++_iter2206) + std::vector ::const_iterator _iter2218; + for (_iter2218 = this->group_names.begin(); _iter2218 != this->group_names.end(); ++_iter2218) { - xfer += oprot->writeString((*_iter2206)); + xfer += oprot->writeString((*_iter2218)); } xfer += oprot->writeListEnd(); } @@ -22331,10 +22331,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2207; - for (_iter2207 = (*(this->group_names)).begin(); _iter2207 != (*(this->group_names)).end(); ++_iter2207) + std::vector ::const_iterator _iter2219; + for (_iter2219 = (*(this->group_names)).begin(); _iter2219 != (*(this->group_names)).end(); ++_iter2219) { - xfer += oprot->writeString((*_iter2207)); + xfer += oprot->writeString((*_iter2219)); } xfer += oprot->writeListEnd(); } @@ -22375,14 +22375,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2208; - ::apache::thrift::protocol::TType _etype2211; - xfer += iprot->readListBegin(_etype2211, _size2208); - this->success.resize(_size2208); - uint32_t _i2212; - for (_i2212 = 0; _i2212 < _size2208; ++_i2212) + uint32_t _size2220; + ::apache::thrift::protocol::TType _etype2223; + xfer += iprot->readListBegin(_etype2223, _size2220); + this->success.resize(_size2220); + uint32_t _i2224; + for (_i2224 = 0; _i2224 < _size2220; ++_i2224) { - xfer += this->success[_i2212].read(iprot); + xfer += this->success[_i2224].read(iprot); } xfer += iprot->readListEnd(); } @@ -22429,10 +22429,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2213; - for (_iter2213 = this->success.begin(); _iter2213 != this->success.end(); ++_iter2213) + std::vector ::const_iterator _iter2225; + for (_iter2225 = this->success.begin(); _iter2225 != this->success.end(); ++_iter2225) { - xfer += (*_iter2213).write(oprot); + xfer += (*_iter2225).write(oprot); } xfer += oprot->writeListEnd(); } @@ -22481,14 +22481,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2214; - ::apache::thrift::protocol::TType _etype2217; - xfer += iprot->readListBegin(_etype2217, _size2214); - (*(this->success)).resize(_size2214); - uint32_t _i2218; - for (_i2218 = 0; _i2218 < _size2214; ++_i2218) + uint32_t _size2226; + ::apache::thrift::protocol::TType _etype2229; + xfer += iprot->readListBegin(_etype2229, _size2226); + (*(this->success)).resize(_size2226); + uint32_t _i2230; + for (_i2230 = 0; _i2230 < _size2226; ++_i2230) { - xfer += (*(this->success))[_i2218].read(iprot); + xfer += (*(this->success))[_i2230].read(iprot); } xfer += iprot->readListEnd(); } @@ -22666,14 +22666,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2219; - ::apache::thrift::protocol::TType _etype2222; - xfer += iprot->readListBegin(_etype2222, _size2219); - this->success.resize(_size2219); - uint32_t _i2223; - for (_i2223 = 0; _i2223 < _size2219; ++_i2223) + uint32_t _size2231; + ::apache::thrift::protocol::TType _etype2234; + xfer += iprot->readListBegin(_etype2234, _size2231); + this->success.resize(_size2231); + uint32_t _i2235; + for (_i2235 = 0; _i2235 < _size2231; ++_i2235) { - xfer += this->success[_i2223].read(iprot); + xfer += this->success[_i2235].read(iprot); } xfer += iprot->readListEnd(); } @@ -22720,10 +22720,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2224; - for (_iter2224 = this->success.begin(); _iter2224 != this->success.end(); ++_iter2224) + std::vector ::const_iterator _iter2236; + for (_iter2236 = this->success.begin(); _iter2236 != this->success.end(); ++_iter2236) { - xfer += (*_iter2224).write(oprot); + xfer += (*_iter2236).write(oprot); } xfer += oprot->writeListEnd(); } @@ -22772,14 +22772,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2225; - ::apache::thrift::protocol::TType _etype2228; - xfer += iprot->readListBegin(_etype2228, _size2225); - (*(this->success)).resize(_size2225); - uint32_t _i2229; - for (_i2229 = 0; _i2229 < _size2225; ++_i2229) + uint32_t _size2237; + ::apache::thrift::protocol::TType _etype2240; + xfer += iprot->readListBegin(_etype2240, _size2237); + (*(this->success)).resize(_size2237); + uint32_t _i2241; + for (_i2241 = 0; _i2241 < _size2237; ++_i2241) { - xfer += (*(this->success))[_i2229].read(iprot); + xfer += (*(this->success))[_i2241].read(iprot); } xfer += iprot->readListEnd(); } @@ -22957,14 +22957,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2230; - ::apache::thrift::protocol::TType _etype2233; - xfer += iprot->readListBegin(_etype2233, _size2230); - this->success.resize(_size2230); - uint32_t _i2234; - for (_i2234 = 0; _i2234 < _size2230; ++_i2234) + uint32_t _size2242; + ::apache::thrift::protocol::TType _etype2245; + xfer += iprot->readListBegin(_etype2245, _size2242); + this->success.resize(_size2242); + uint32_t _i2246; + for (_i2246 = 0; _i2246 < _size2242; ++_i2246) { - xfer += iprot->readString(this->success[_i2234]); + xfer += iprot->readString(this->success[_i2246]); } xfer += iprot->readListEnd(); } @@ -23011,10 +23011,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2235; - for (_iter2235 = this->success.begin(); _iter2235 != this->success.end(); ++_iter2235) + std::vector ::const_iterator _iter2247; + for (_iter2247 = this->success.begin(); _iter2247 != this->success.end(); ++_iter2247) { - xfer += oprot->writeString((*_iter2235)); + xfer += oprot->writeString((*_iter2247)); } xfer += oprot->writeListEnd(); } @@ -23063,14 +23063,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2236; - ::apache::thrift::protocol::TType _etype2239; - xfer += iprot->readListBegin(_etype2239, _size2236); - (*(this->success)).resize(_size2236); - uint32_t _i2240; - for (_i2240 = 0; _i2240 < _size2236; ++_i2240) + uint32_t _size2248; + ::apache::thrift::protocol::TType _etype2251; + xfer += iprot->readListBegin(_etype2251, _size2248); + (*(this->success)).resize(_size2248); + uint32_t _i2252; + for (_i2252 = 0; _i2252 < _size2248; ++_i2252) { - xfer += iprot->readString((*(this->success))[_i2240]); + xfer += iprot->readString((*(this->success))[_i2252]); } xfer += iprot->readListEnd(); } @@ -23380,14 +23380,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2241; - ::apache::thrift::protocol::TType _etype2244; - xfer += iprot->readListBegin(_etype2244, _size2241); - this->part_vals.resize(_size2241); - uint32_t _i2245; - for (_i2245 = 0; _i2245 < _size2241; ++_i2245) + uint32_t _size2253; + ::apache::thrift::protocol::TType _etype2256; + xfer += iprot->readListBegin(_etype2256, _size2253); + this->part_vals.resize(_size2253); + uint32_t _i2257; + for (_i2257 = 0; _i2257 < _size2253; ++_i2257) { - xfer += iprot->readString(this->part_vals[_i2245]); + xfer += iprot->readString(this->part_vals[_i2257]); } xfer += iprot->readListEnd(); } @@ -23432,10 +23432,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2246; - for (_iter2246 = this->part_vals.begin(); _iter2246 != this->part_vals.end(); ++_iter2246) + std::vector ::const_iterator _iter2258; + for (_iter2258 = this->part_vals.begin(); _iter2258 != this->part_vals.end(); ++_iter2258) { - xfer += oprot->writeString((*_iter2246)); + xfer += oprot->writeString((*_iter2258)); } xfer += oprot->writeListEnd(); } @@ -23471,10 +23471,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2247; - for (_iter2247 = (*(this->part_vals)).begin(); _iter2247 != (*(this->part_vals)).end(); ++_iter2247) + std::vector ::const_iterator _iter2259; + for (_iter2259 = (*(this->part_vals)).begin(); _iter2259 != (*(this->part_vals)).end(); ++_iter2259) { - xfer += oprot->writeString((*_iter2247)); + xfer += oprot->writeString((*_iter2259)); } xfer += oprot->writeListEnd(); } @@ -23519,14 +23519,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2248; - ::apache::thrift::protocol::TType _etype2251; - xfer += iprot->readListBegin(_etype2251, _size2248); - this->success.resize(_size2248); - uint32_t _i2252; - for (_i2252 = 0; _i2252 < _size2248; ++_i2252) + uint32_t _size2260; + ::apache::thrift::protocol::TType _etype2263; + xfer += iprot->readListBegin(_etype2263, _size2260); + this->success.resize(_size2260); + uint32_t _i2264; + for (_i2264 = 0; _i2264 < _size2260; ++_i2264) { - xfer += this->success[_i2252].read(iprot); + xfer += this->success[_i2264].read(iprot); } xfer += iprot->readListEnd(); } @@ -23573,10 +23573,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2253; - for (_iter2253 = this->success.begin(); _iter2253 != this->success.end(); ++_iter2253) + std::vector ::const_iterator _iter2265; + for (_iter2265 = this->success.begin(); _iter2265 != this->success.end(); ++_iter2265) { - xfer += (*_iter2253).write(oprot); + xfer += (*_iter2265).write(oprot); } xfer += oprot->writeListEnd(); } @@ -23625,14 +23625,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2254; - ::apache::thrift::protocol::TType _etype2257; - xfer += iprot->readListBegin(_etype2257, _size2254); - (*(this->success)).resize(_size2254); - uint32_t _i2258; - for (_i2258 = 0; _i2258 < _size2254; ++_i2258) + uint32_t _size2266; + ::apache::thrift::protocol::TType _etype2269; + xfer += iprot->readListBegin(_etype2269, _size2266); + (*(this->success)).resize(_size2266); + uint32_t _i2270; + for (_i2270 = 0; _i2270 < _size2266; ++_i2270) { - xfer += (*(this->success))[_i2258].read(iprot); + xfer += (*(this->success))[_i2270].read(iprot); } xfer += iprot->readListEnd(); } @@ -23715,14 +23715,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2259; - ::apache::thrift::protocol::TType _etype2262; - xfer += iprot->readListBegin(_etype2262, _size2259); - this->part_vals.resize(_size2259); - uint32_t _i2263; - for (_i2263 = 0; _i2263 < _size2259; ++_i2263) + uint32_t _size2271; + ::apache::thrift::protocol::TType _etype2274; + xfer += iprot->readListBegin(_etype2274, _size2271); + this->part_vals.resize(_size2271); + uint32_t _i2275; + for (_i2275 = 0; _i2275 < _size2271; ++_i2275) { - xfer += iprot->readString(this->part_vals[_i2263]); + xfer += iprot->readString(this->part_vals[_i2275]); } xfer += iprot->readListEnd(); } @@ -23751,14 +23751,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2264; - ::apache::thrift::protocol::TType _etype2267; - xfer += iprot->readListBegin(_etype2267, _size2264); - this->group_names.resize(_size2264); - uint32_t _i2268; - for (_i2268 = 0; _i2268 < _size2264; ++_i2268) + uint32_t _size2276; + ::apache::thrift::protocol::TType _etype2279; + xfer += iprot->readListBegin(_etype2279, _size2276); + this->group_names.resize(_size2276); + uint32_t _i2280; + for (_i2280 = 0; _i2280 < _size2276; ++_i2280) { - xfer += iprot->readString(this->group_names[_i2268]); + xfer += iprot->readString(this->group_names[_i2280]); } xfer += iprot->readListEnd(); } @@ -23795,10 +23795,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2269; - for (_iter2269 = this->part_vals.begin(); _iter2269 != this->part_vals.end(); ++_iter2269) + std::vector ::const_iterator _iter2281; + for (_iter2281 = this->part_vals.begin(); _iter2281 != this->part_vals.end(); ++_iter2281) { - xfer += oprot->writeString((*_iter2269)); + xfer += oprot->writeString((*_iter2281)); } xfer += oprot->writeListEnd(); } @@ -23815,10 +23815,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2270; - for (_iter2270 = this->group_names.begin(); _iter2270 != this->group_names.end(); ++_iter2270) + std::vector ::const_iterator _iter2282; + for (_iter2282 = this->group_names.begin(); _iter2282 != this->group_names.end(); ++_iter2282) { - xfer += oprot->writeString((*_iter2270)); + xfer += oprot->writeString((*_iter2282)); } xfer += oprot->writeListEnd(); } @@ -23850,10 +23850,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache:: xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2271; - for (_iter2271 = (*(this->part_vals)).begin(); _iter2271 != (*(this->part_vals)).end(); ++_iter2271) + std::vector ::const_iterator _iter2283; + for (_iter2283 = (*(this->part_vals)).begin(); _iter2283 != (*(this->part_vals)).end(); ++_iter2283) { - xfer += oprot->writeString((*_iter2271)); + xfer += oprot->writeString((*_iter2283)); } xfer += oprot->writeListEnd(); } @@ -23870,10 +23870,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache:: xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2272; - for (_iter2272 = (*(this->group_names)).begin(); _iter2272 != (*(this->group_names)).end(); ++_iter2272) + std::vector ::const_iterator _iter2284; + for (_iter2284 = (*(this->group_names)).begin(); _iter2284 != (*(this->group_names)).end(); ++_iter2284) { - xfer += oprot->writeString((*_iter2272)); + xfer += oprot->writeString((*_iter2284)); } xfer += oprot->writeListEnd(); } @@ -23914,14 +23914,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2273; - ::apache::thrift::protocol::TType _etype2276; - xfer += iprot->readListBegin(_etype2276, _size2273); - this->success.resize(_size2273); - uint32_t _i2277; - for (_i2277 = 0; _i2277 < _size2273; ++_i2277) + uint32_t _size2285; + ::apache::thrift::protocol::TType _etype2288; + xfer += iprot->readListBegin(_etype2288, _size2285); + this->success.resize(_size2285); + uint32_t _i2289; + for (_i2289 = 0; _i2289 < _size2285; ++_i2289) { - xfer += this->success[_i2277].read(iprot); + xfer += this->success[_i2289].read(iprot); } xfer += iprot->readListEnd(); } @@ -23968,10 +23968,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2278; - for (_iter2278 = this->success.begin(); _iter2278 != this->success.end(); ++_iter2278) + std::vector ::const_iterator _iter2290; + for (_iter2290 = this->success.begin(); _iter2290 != this->success.end(); ++_iter2290) { - xfer += (*_iter2278).write(oprot); + xfer += (*_iter2290).write(oprot); } xfer += oprot->writeListEnd(); } @@ -24020,14 +24020,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2279; - ::apache::thrift::protocol::TType _etype2282; - xfer += iprot->readListBegin(_etype2282, _size2279); - (*(this->success)).resize(_size2279); - uint32_t _i2283; - for (_i2283 = 0; _i2283 < _size2279; ++_i2283) + uint32_t _size2291; + ::apache::thrift::protocol::TType _etype2294; + xfer += iprot->readListBegin(_etype2294, _size2291); + (*(this->success)).resize(_size2291); + uint32_t _i2295; + for (_i2295 = 0; _i2295 < _size2291; ++_i2295) { - xfer += (*(this->success))[_i2283].read(iprot); + xfer += (*(this->success))[_i2295].read(iprot); } xfer += iprot->readListEnd(); } @@ -24337,14 +24337,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2284; - ::apache::thrift::protocol::TType _etype2287; - xfer += iprot->readListBegin(_etype2287, _size2284); - this->part_vals.resize(_size2284); - uint32_t _i2288; - for (_i2288 = 0; _i2288 < _size2284; ++_i2288) + uint32_t _size2296; + ::apache::thrift::protocol::TType _etype2299; + xfer += iprot->readListBegin(_etype2299, _size2296); + this->part_vals.resize(_size2296); + uint32_t _i2300; + for (_i2300 = 0; _i2300 < _size2296; ++_i2300) { - xfer += iprot->readString(this->part_vals[_i2288]); + xfer += iprot->readString(this->part_vals[_i2300]); } xfer += iprot->readListEnd(); } @@ -24389,10 +24389,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2289; - for (_iter2289 = this->part_vals.begin(); _iter2289 != this->part_vals.end(); ++_iter2289) + std::vector ::const_iterator _iter2301; + for (_iter2301 = this->part_vals.begin(); _iter2301 != this->part_vals.end(); ++_iter2301) { - xfer += oprot->writeString((*_iter2289)); + xfer += oprot->writeString((*_iter2301)); } xfer += oprot->writeListEnd(); } @@ -24428,10 +24428,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrif xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2290; - for (_iter2290 = (*(this->part_vals)).begin(); _iter2290 != (*(this->part_vals)).end(); ++_iter2290) + std::vector ::const_iterator _iter2302; + for (_iter2302 = (*(this->part_vals)).begin(); _iter2302 != (*(this->part_vals)).end(); ++_iter2302) { - xfer += oprot->writeString((*_iter2290)); + xfer += oprot->writeString((*_iter2302)); } xfer += oprot->writeListEnd(); } @@ -24476,14 +24476,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2291; - ::apache::thrift::protocol::TType _etype2294; - xfer += iprot->readListBegin(_etype2294, _size2291); - this->success.resize(_size2291); - uint32_t _i2295; - for (_i2295 = 0; _i2295 < _size2291; ++_i2295) + uint32_t _size2303; + ::apache::thrift::protocol::TType _etype2306; + xfer += iprot->readListBegin(_etype2306, _size2303); + this->success.resize(_size2303); + uint32_t _i2307; + for (_i2307 = 0; _i2307 < _size2303; ++_i2307) { - xfer += iprot->readString(this->success[_i2295]); + xfer += iprot->readString(this->success[_i2307]); } xfer += iprot->readListEnd(); } @@ -24530,10 +24530,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2296; - for (_iter2296 = this->success.begin(); _iter2296 != this->success.end(); ++_iter2296) + std::vector ::const_iterator _iter2308; + for (_iter2308 = this->success.begin(); _iter2308 != this->success.end(); ++_iter2308) { - xfer += oprot->writeString((*_iter2296)); + xfer += oprot->writeString((*_iter2308)); } xfer += oprot->writeListEnd(); } @@ -24582,14 +24582,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2297; - ::apache::thrift::protocol::TType _etype2300; - xfer += iprot->readListBegin(_etype2300, _size2297); - (*(this->success)).resize(_size2297); - uint32_t _i2301; - for (_i2301 = 0; _i2301 < _size2297; ++_i2301) + uint32_t _size2309; + ::apache::thrift::protocol::TType _etype2312; + xfer += iprot->readListBegin(_etype2312, _size2309); + (*(this->success)).resize(_size2309); + uint32_t _i2313; + for (_i2313 = 0; _i2313 < _size2309; ++_i2313) { - xfer += iprot->readString((*(this->success))[_i2301]); + xfer += iprot->readString((*(this->success))[_i2313]); } xfer += iprot->readListEnd(); } @@ -24962,14 +24962,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2302; - ::apache::thrift::protocol::TType _etype2305; - xfer += iprot->readListBegin(_etype2305, _size2302); - this->success.resize(_size2302); - uint32_t _i2306; - for (_i2306 = 0; _i2306 < _size2302; ++_i2306) + uint32_t _size2314; + ::apache::thrift::protocol::TType _etype2317; + xfer += iprot->readListBegin(_etype2317, _size2314); + this->success.resize(_size2314); + uint32_t _i2318; + for (_i2318 = 0; _i2318 < _size2314; ++_i2318) { - xfer += iprot->readString(this->success[_i2306]); + xfer += iprot->readString(this->success[_i2318]); } xfer += iprot->readListEnd(); } @@ -25016,10 +25016,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2307; - for (_iter2307 = this->success.begin(); _iter2307 != this->success.end(); ++_iter2307) + std::vector ::const_iterator _iter2319; + for (_iter2319 = this->success.begin(); _iter2319 != this->success.end(); ++_iter2319) { - xfer += oprot->writeString((*_iter2307)); + xfer += oprot->writeString((*_iter2319)); } xfer += oprot->writeListEnd(); } @@ -25068,14 +25068,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2308; - ::apache::thrift::protocol::TType _etype2311; - xfer += iprot->readListBegin(_etype2311, _size2308); - (*(this->success)).resize(_size2308); - uint32_t _i2312; - for (_i2312 = 0; _i2312 < _size2308; ++_i2312) + uint32_t _size2320; + ::apache::thrift::protocol::TType _etype2323; + xfer += iprot->readListBegin(_etype2323, _size2320); + (*(this->success)).resize(_size2320); + uint32_t _i2324; + for (_i2324 = 0; _i2324 < _size2320; ++_i2324) { - xfer += iprot->readString((*(this->success))[_i2312]); + xfer += iprot->readString((*(this->success))[_i2324]); } xfer += iprot->readListEnd(); } @@ -25269,14 +25269,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2313; - ::apache::thrift::protocol::TType _etype2316; - xfer += iprot->readListBegin(_etype2316, _size2313); - this->success.resize(_size2313); - uint32_t _i2317; - for (_i2317 = 0; _i2317 < _size2313; ++_i2317) + uint32_t _size2325; + ::apache::thrift::protocol::TType _etype2328; + xfer += iprot->readListBegin(_etype2328, _size2325); + this->success.resize(_size2325); + uint32_t _i2329; + for (_i2329 = 0; _i2329 < _size2325; ++_i2329) { - xfer += this->success[_i2317].read(iprot); + xfer += this->success[_i2329].read(iprot); } xfer += iprot->readListEnd(); } @@ -25323,10 +25323,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2318; - for (_iter2318 = this->success.begin(); _iter2318 != this->success.end(); ++_iter2318) + std::vector ::const_iterator _iter2330; + for (_iter2330 = this->success.begin(); _iter2330 != this->success.end(); ++_iter2330) { - xfer += (*_iter2318).write(oprot); + xfer += (*_iter2330).write(oprot); } xfer += oprot->writeListEnd(); } @@ -25375,14 +25375,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2319; - ::apache::thrift::protocol::TType _etype2322; - xfer += iprot->readListBegin(_etype2322, _size2319); - (*(this->success)).resize(_size2319); - uint32_t _i2323; - for (_i2323 = 0; _i2323 < _size2319; ++_i2323) + uint32_t _size2331; + ::apache::thrift::protocol::TType _etype2334; + xfer += iprot->readListBegin(_etype2334, _size2331); + (*(this->success)).resize(_size2331); + uint32_t _i2335; + for (_i2335 = 0; _i2335 < _size2331; ++_i2335) { - xfer += (*(this->success))[_i2323].read(iprot); + xfer += (*(this->success))[_i2335].read(iprot); } xfer += iprot->readListEnd(); } @@ -25528,14 +25528,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_result::read(::apache: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2324; - ::apache::thrift::protocol::TType _etype2327; - xfer += iprot->readListBegin(_etype2327, _size2324); - this->success.resize(_size2324); - uint32_t _i2328; - for (_i2328 = 0; _i2328 < _size2324; ++_i2328) + uint32_t _size2336; + ::apache::thrift::protocol::TType _etype2339; + xfer += iprot->readListBegin(_etype2339, _size2336); + this->success.resize(_size2336); + uint32_t _i2340; + for (_i2340 = 0; _i2340 < _size2336; ++_i2340) { - xfer += this->success[_i2328].read(iprot); + xfer += this->success[_i2340].read(iprot); } xfer += iprot->readListEnd(); } @@ -25582,10 +25582,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_result::write(::apache xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2329; - for (_iter2329 = this->success.begin(); _iter2329 != this->success.end(); ++_iter2329) + std::vector ::const_iterator _iter2341; + for (_iter2341 = this->success.begin(); _iter2341 != this->success.end(); ++_iter2341) { - xfer += (*_iter2329).write(oprot); + xfer += (*_iter2341).write(oprot); } xfer += oprot->writeListEnd(); } @@ -25634,14 +25634,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_presult::read(::apache if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2330; - ::apache::thrift::protocol::TType _etype2333; - xfer += iprot->readListBegin(_etype2333, _size2330); - (*(this->success)).resize(_size2330); - uint32_t _i2334; - for (_i2334 = 0; _i2334 < _size2330; ++_i2334) + uint32_t _size2342; + ::apache::thrift::protocol::TType _etype2345; + xfer += iprot->readListBegin(_etype2345, _size2342); + (*(this->success)).resize(_size2342); + uint32_t _i2346; + for (_i2346 = 0; _i2346 < _size2342; ++_i2346) { - xfer += (*(this->success))[_i2334].read(iprot); + xfer += (*(this->success))[_i2346].read(iprot); } xfer += iprot->readListEnd(); } @@ -25835,14 +25835,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2335; - ::apache::thrift::protocol::TType _etype2338; - xfer += iprot->readListBegin(_etype2338, _size2335); - this->success.resize(_size2335); - uint32_t _i2339; - for (_i2339 = 0; _i2339 < _size2335; ++_i2339) + uint32_t _size2347; + ::apache::thrift::protocol::TType _etype2350; + xfer += iprot->readListBegin(_etype2350, _size2347); + this->success.resize(_size2347); + uint32_t _i2351; + for (_i2351 = 0; _i2351 < _size2347; ++_i2351) { - xfer += this->success[_i2339].read(iprot); + xfer += this->success[_i2351].read(iprot); } xfer += iprot->readListEnd(); } @@ -25889,10 +25889,10 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2340; - for (_iter2340 = this->success.begin(); _iter2340 != this->success.end(); ++_iter2340) + std::vector ::const_iterator _iter2352; + for (_iter2352 = this->success.begin(); _iter2352 != this->success.end(); ++_iter2352) { - xfer += (*_iter2340).write(oprot); + xfer += (*_iter2352).write(oprot); } xfer += oprot->writeListEnd(); } @@ -25941,14 +25941,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2341; - ::apache::thrift::protocol::TType _etype2344; - xfer += iprot->readListBegin(_etype2344, _size2341); - (*(this->success)).resize(_size2341); - uint32_t _i2345; - for (_i2345 = 0; _i2345 < _size2341; ++_i2345) + uint32_t _size2353; + ::apache::thrift::protocol::TType _etype2356; + xfer += iprot->readListBegin(_etype2356, _size2353); + (*(this->success)).resize(_size2353); + uint32_t _i2357; + for (_i2357 = 0; _i2357 < _size2353; ++_i2357) { - xfer += (*(this->success))[_i2345].read(iprot); + xfer += (*(this->success))[_i2357].read(iprot); } xfer += iprot->readListEnd(); } @@ -26744,14 +26744,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->names.clear(); - uint32_t _size2346; - ::apache::thrift::protocol::TType _etype2349; - xfer += iprot->readListBegin(_etype2349, _size2346); - this->names.resize(_size2346); - uint32_t _i2350; - for (_i2350 = 0; _i2350 < _size2346; ++_i2350) + uint32_t _size2358; + ::apache::thrift::protocol::TType _etype2361; + xfer += iprot->readListBegin(_etype2361, _size2358); + this->names.resize(_size2358); + uint32_t _i2362; + for (_i2362 = 0; _i2362 < _size2358; ++_i2362) { - xfer += iprot->readString(this->names[_i2350]); + xfer += iprot->readString(this->names[_i2362]); } xfer += iprot->readListEnd(); } @@ -26788,10 +26788,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrif xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); - std::vector ::const_iterator _iter2351; - for (_iter2351 = this->names.begin(); _iter2351 != this->names.end(); ++_iter2351) + std::vector ::const_iterator _iter2363; + for (_iter2363 = this->names.begin(); _iter2363 != this->names.end(); ++_iter2363) { - xfer += oprot->writeString((*_iter2351)); + xfer += oprot->writeString((*_iter2363)); } xfer += oprot->writeListEnd(); } @@ -26823,10 +26823,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thri xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->names)).size())); - std::vector ::const_iterator _iter2352; - for (_iter2352 = (*(this->names)).begin(); _iter2352 != (*(this->names)).end(); ++_iter2352) + std::vector ::const_iterator _iter2364; + for (_iter2364 = (*(this->names)).begin(); _iter2364 != (*(this->names)).end(); ++_iter2364) { - xfer += oprot->writeString((*_iter2352)); + xfer += oprot->writeString((*_iter2364)); } xfer += oprot->writeListEnd(); } @@ -26867,14 +26867,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2353; - ::apache::thrift::protocol::TType _etype2356; - xfer += iprot->readListBegin(_etype2356, _size2353); - this->success.resize(_size2353); - uint32_t _i2357; - for (_i2357 = 0; _i2357 < _size2353; ++_i2357) + uint32_t _size2365; + ::apache::thrift::protocol::TType _etype2368; + xfer += iprot->readListBegin(_etype2368, _size2365); + this->success.resize(_size2365); + uint32_t _i2369; + for (_i2369 = 0; _i2369 < _size2365; ++_i2369) { - xfer += this->success[_i2357].read(iprot); + xfer += this->success[_i2369].read(iprot); } xfer += iprot->readListEnd(); } @@ -26929,10 +26929,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2358; - for (_iter2358 = this->success.begin(); _iter2358 != this->success.end(); ++_iter2358) + std::vector ::const_iterator _iter2370; + for (_iter2370 = this->success.begin(); _iter2370 != this->success.end(); ++_iter2370) { - xfer += (*_iter2358).write(oprot); + xfer += (*_iter2370).write(oprot); } xfer += oprot->writeListEnd(); } @@ -26985,14 +26985,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2359; - ::apache::thrift::protocol::TType _etype2362; - xfer += iprot->readListBegin(_etype2362, _size2359); - (*(this->success)).resize(_size2359); - uint32_t _i2363; - for (_i2363 = 0; _i2363 < _size2359; ++_i2363) + uint32_t _size2371; + ::apache::thrift::protocol::TType _etype2374; + xfer += iprot->readListBegin(_etype2374, _size2371); + (*(this->success)).resize(_size2371); + uint32_t _i2375; + for (_i2375 = 0; _i2375 < _size2371; ++_i2375) { - xfer += (*(this->success))[_i2363].read(iprot); + xfer += (*(this->success))[_i2375].read(iprot); } xfer += iprot->readListEnd(); } @@ -28023,14 +28023,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2364; - ::apache::thrift::protocol::TType _etype2367; - xfer += iprot->readListBegin(_etype2367, _size2364); - this->new_parts.resize(_size2364); - uint32_t _i2368; - for (_i2368 = 0; _i2368 < _size2364; ++_i2368) + uint32_t _size2376; + ::apache::thrift::protocol::TType _etype2379; + xfer += iprot->readListBegin(_etype2379, _size2376); + this->new_parts.resize(_size2376); + uint32_t _i2380; + for (_i2380 = 0; _i2380 < _size2376; ++_i2380) { - xfer += this->new_parts[_i2368].read(iprot); + xfer += this->new_parts[_i2380].read(iprot); } xfer += iprot->readListEnd(); } @@ -28067,10 +28067,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2369; - for (_iter2369 = this->new_parts.begin(); _iter2369 != this->new_parts.end(); ++_iter2369) + std::vector ::const_iterator _iter2381; + for (_iter2381 = this->new_parts.begin(); _iter2381 != this->new_parts.end(); ++_iter2381) { - xfer += (*_iter2369).write(oprot); + xfer += (*_iter2381).write(oprot); } xfer += oprot->writeListEnd(); } @@ -28102,10 +28102,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2370; - for (_iter2370 = (*(this->new_parts)).begin(); _iter2370 != (*(this->new_parts)).end(); ++_iter2370) + std::vector ::const_iterator _iter2382; + for (_iter2382 = (*(this->new_parts)).begin(); _iter2382 != (*(this->new_parts)).end(); ++_iter2382) { - xfer += (*_iter2370).write(oprot); + xfer += (*_iter2382).write(oprot); } xfer += oprot->writeListEnd(); } @@ -28290,14 +28290,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::rea if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2371; - ::apache::thrift::protocol::TType _etype2374; - xfer += iprot->readListBegin(_etype2374, _size2371); - this->new_parts.resize(_size2371); - uint32_t _i2375; - for (_i2375 = 0; _i2375 < _size2371; ++_i2375) + uint32_t _size2383; + ::apache::thrift::protocol::TType _etype2386; + xfer += iprot->readListBegin(_etype2386, _size2383); + this->new_parts.resize(_size2383); + uint32_t _i2387; + for (_i2387 = 0; _i2387 < _size2383; ++_i2387) { - xfer += this->new_parts[_i2375].read(iprot); + xfer += this->new_parts[_i2387].read(iprot); } xfer += iprot->readListEnd(); } @@ -28342,10 +28342,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::wri xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2376; - for (_iter2376 = this->new_parts.begin(); _iter2376 != this->new_parts.end(); ++_iter2376) + std::vector ::const_iterator _iter2388; + for (_iter2388 = this->new_parts.begin(); _iter2388 != this->new_parts.end(); ++_iter2388) { - xfer += (*_iter2376).write(oprot); + xfer += (*_iter2388).write(oprot); } xfer += oprot->writeListEnd(); } @@ -28381,10 +28381,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::wr xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2377; - for (_iter2377 = (*(this->new_parts)).begin(); _iter2377 != (*(this->new_parts)).end(); ++_iter2377) + std::vector ::const_iterator _iter2389; + for (_iter2389 = (*(this->new_parts)).begin(); _iter2389 != (*(this->new_parts)).end(); ++_iter2389) { - xfer += (*_iter2377).write(oprot); + xfer += (*_iter2389).write(oprot); } xfer += oprot->writeListEnd(); } @@ -29055,14 +29055,14 @@ uint32_t ThriftHiveMetastore_rename_partition_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2378; - ::apache::thrift::protocol::TType _etype2381; - xfer += iprot->readListBegin(_etype2381, _size2378); - this->part_vals.resize(_size2378); - uint32_t _i2382; - for (_i2382 = 0; _i2382 < _size2378; ++_i2382) + uint32_t _size2390; + ::apache::thrift::protocol::TType _etype2393; + xfer += iprot->readListBegin(_etype2393, _size2390); + this->part_vals.resize(_size2390); + uint32_t _i2394; + for (_i2394 = 0; _i2394 < _size2390; ++_i2394) { - xfer += iprot->readString(this->part_vals[_i2382]); + xfer += iprot->readString(this->part_vals[_i2394]); } xfer += iprot->readListEnd(); } @@ -29107,10 +29107,10 @@ uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2383; - for (_iter2383 = this->part_vals.begin(); _iter2383 != this->part_vals.end(); ++_iter2383) + std::vector ::const_iterator _iter2395; + for (_iter2395 = this->part_vals.begin(); _iter2395 != this->part_vals.end(); ++_iter2395) { - xfer += oprot->writeString((*_iter2383)); + xfer += oprot->writeString((*_iter2395)); } xfer += oprot->writeListEnd(); } @@ -29146,10 +29146,10 @@ uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2384; - for (_iter2384 = (*(this->part_vals)).begin(); _iter2384 != (*(this->part_vals)).end(); ++_iter2384) + std::vector ::const_iterator _iter2396; + for (_iter2396 = (*(this->part_vals)).begin(); _iter2396 != (*(this->part_vals)).end(); ++_iter2396) { - xfer += oprot->writeString((*_iter2384)); + xfer += oprot->writeString((*_iter2396)); } xfer += oprot->writeListEnd(); } @@ -29549,14 +29549,14 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::read(::ap if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2385; - ::apache::thrift::protocol::TType _etype2388; - xfer += iprot->readListBegin(_etype2388, _size2385); - this->part_vals.resize(_size2385); - uint32_t _i2389; - for (_i2389 = 0; _i2389 < _size2385; ++_i2389) + uint32_t _size2397; + ::apache::thrift::protocol::TType _etype2400; + xfer += iprot->readListBegin(_etype2400, _size2397); + this->part_vals.resize(_size2397); + uint32_t _i2401; + for (_i2401 = 0; _i2401 < _size2397; ++_i2401) { - xfer += iprot->readString(this->part_vals[_i2389]); + xfer += iprot->readString(this->part_vals[_i2401]); } xfer += iprot->readListEnd(); } @@ -29593,10 +29593,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::a xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2390; - for (_iter2390 = this->part_vals.begin(); _iter2390 != this->part_vals.end(); ++_iter2390) + std::vector ::const_iterator _iter2402; + for (_iter2402 = this->part_vals.begin(); _iter2402 != this->part_vals.end(); ++_iter2402) { - xfer += oprot->writeString((*_iter2390)); + xfer += oprot->writeString((*_iter2402)); } xfer += oprot->writeListEnd(); } @@ -29624,10 +29624,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(:: xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2391; - for (_iter2391 = (*(this->part_vals)).begin(); _iter2391 != (*(this->part_vals)).end(); ++_iter2391) + std::vector ::const_iterator _iter2403; + for (_iter2403 = (*(this->part_vals)).begin(); _iter2403 != (*(this->part_vals)).end(); ++_iter2403) { - xfer += oprot->writeString((*_iter2391)); + xfer += oprot->writeString((*_iter2403)); } xfer += oprot->writeListEnd(); } @@ -30102,14 +30102,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2392; - ::apache::thrift::protocol::TType _etype2395; - xfer += iprot->readListBegin(_etype2395, _size2392); - this->success.resize(_size2392); - uint32_t _i2396; - for (_i2396 = 0; _i2396 < _size2392; ++_i2396) + uint32_t _size2404; + ::apache::thrift::protocol::TType _etype2407; + xfer += iprot->readListBegin(_etype2407, _size2404); + this->success.resize(_size2404); + uint32_t _i2408; + for (_i2408 = 0; _i2408 < _size2404; ++_i2408) { - xfer += iprot->readString(this->success[_i2396]); + xfer += iprot->readString(this->success[_i2408]); } xfer += iprot->readListEnd(); } @@ -30148,10 +30148,10 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thri xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2397; - for (_iter2397 = this->success.begin(); _iter2397 != this->success.end(); ++_iter2397) + std::vector ::const_iterator _iter2409; + for (_iter2409 = this->success.begin(); _iter2409 != this->success.end(); ++_iter2409) { - xfer += oprot->writeString((*_iter2397)); + xfer += oprot->writeString((*_iter2409)); } xfer += oprot->writeListEnd(); } @@ -30196,14 +30196,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2398; - ::apache::thrift::protocol::TType _etype2401; - xfer += iprot->readListBegin(_etype2401, _size2398); - (*(this->success)).resize(_size2398); - uint32_t _i2402; - for (_i2402 = 0; _i2402 < _size2398; ++_i2402) + uint32_t _size2410; + ::apache::thrift::protocol::TType _etype2413; + xfer += iprot->readListBegin(_etype2413, _size2410); + (*(this->success)).resize(_size2410); + uint32_t _i2414; + for (_i2414 = 0; _i2414 < _size2410; ++_i2414) { - xfer += iprot->readString((*(this->success))[_i2402]); + xfer += iprot->readString((*(this->success))[_i2414]); } xfer += iprot->readListEnd(); } @@ -30341,17 +30341,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_MAP) { { this->success.clear(); - uint32_t _size2403; - ::apache::thrift::protocol::TType _ktype2404; - ::apache::thrift::protocol::TType _vtype2405; - xfer += iprot->readMapBegin(_ktype2404, _vtype2405, _size2403); - uint32_t _i2407; - for (_i2407 = 0; _i2407 < _size2403; ++_i2407) + uint32_t _size2415; + ::apache::thrift::protocol::TType _ktype2416; + ::apache::thrift::protocol::TType _vtype2417; + xfer += iprot->readMapBegin(_ktype2416, _vtype2417, _size2415); + uint32_t _i2419; + for (_i2419 = 0; _i2419 < _size2415; ++_i2419) { - std::string _key2408; - xfer += iprot->readString(_key2408); - std::string& _val2409 = this->success[_key2408]; - xfer += iprot->readString(_val2409); + std::string _key2420; + xfer += iprot->readString(_key2420); + std::string& _val2421 = this->success[_key2420]; + xfer += iprot->readString(_val2421); } xfer += iprot->readMapEnd(); } @@ -30390,11 +30390,11 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thri xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::map ::const_iterator _iter2410; - for (_iter2410 = this->success.begin(); _iter2410 != this->success.end(); ++_iter2410) + std::map ::const_iterator _iter2422; + for (_iter2422 = this->success.begin(); _iter2422 != this->success.end(); ++_iter2422) { - xfer += oprot->writeString(_iter2410->first); - xfer += oprot->writeString(_iter2410->second); + xfer += oprot->writeString(_iter2422->first); + xfer += oprot->writeString(_iter2422->second); } xfer += oprot->writeMapEnd(); } @@ -30439,17 +30439,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_MAP) { { (*(this->success)).clear(); - uint32_t _size2411; - ::apache::thrift::protocol::TType _ktype2412; - ::apache::thrift::protocol::TType _vtype2413; - xfer += iprot->readMapBegin(_ktype2412, _vtype2413, _size2411); - uint32_t _i2415; - for (_i2415 = 0; _i2415 < _size2411; ++_i2415) + uint32_t _size2423; + ::apache::thrift::protocol::TType _ktype2424; + ::apache::thrift::protocol::TType _vtype2425; + xfer += iprot->readMapBegin(_ktype2424, _vtype2425, _size2423); + uint32_t _i2427; + for (_i2427 = 0; _i2427 < _size2423; ++_i2427) { - std::string _key2416; - xfer += iprot->readString(_key2416); - std::string& _val2417 = (*(this->success))[_key2416]; - xfer += iprot->readString(_val2417); + std::string _key2428; + xfer += iprot->readString(_key2428); + std::string& _val2429 = (*(this->success))[_key2428]; + xfer += iprot->readString(_val2429); } xfer += iprot->readMapEnd(); } @@ -30524,17 +30524,17 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_MAP) { { this->part_vals.clear(); - uint32_t _size2418; - ::apache::thrift::protocol::TType _ktype2419; - ::apache::thrift::protocol::TType _vtype2420; - xfer += iprot->readMapBegin(_ktype2419, _vtype2420, _size2418); - uint32_t _i2422; - for (_i2422 = 0; _i2422 < _size2418; ++_i2422) + uint32_t _size2430; + ::apache::thrift::protocol::TType _ktype2431; + ::apache::thrift::protocol::TType _vtype2432; + xfer += iprot->readMapBegin(_ktype2431, _vtype2432, _size2430); + uint32_t _i2434; + for (_i2434 = 0; _i2434 < _size2430; ++_i2434) { - std::string _key2423; - xfer += iprot->readString(_key2423); - std::string& _val2424 = this->part_vals[_key2423]; - xfer += iprot->readString(_val2424); + std::string _key2435; + xfer += iprot->readString(_key2435); + std::string& _val2436 = this->part_vals[_key2435]; + xfer += iprot->readString(_val2436); } xfer += iprot->readMapEnd(); } @@ -30545,9 +30545,9 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift:: break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2425; - xfer += iprot->readI32(ecast2425); - this->eventType = static_cast(ecast2425); + int32_t ecast2437; + xfer += iprot->readI32(ecast2437); + this->eventType = static_cast(ecast2437); this->__isset.eventType = true; } else { xfer += iprot->skip(ftype); @@ -30581,11 +30581,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift: xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::map ::const_iterator _iter2426; - for (_iter2426 = this->part_vals.begin(); _iter2426 != this->part_vals.end(); ++_iter2426) + std::map ::const_iterator _iter2438; + for (_iter2438 = this->part_vals.begin(); _iter2438 != this->part_vals.end(); ++_iter2438) { - xfer += oprot->writeString(_iter2426->first); - xfer += oprot->writeString(_iter2426->second); + xfer += oprot->writeString(_iter2438->first); + xfer += oprot->writeString(_iter2438->second); } xfer += oprot->writeMapEnd(); } @@ -30621,11 +30621,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::map ::const_iterator _iter2427; - for (_iter2427 = (*(this->part_vals)).begin(); _iter2427 != (*(this->part_vals)).end(); ++_iter2427) + std::map ::const_iterator _iter2439; + for (_iter2439 = (*(this->part_vals)).begin(); _iter2439 != (*(this->part_vals)).end(); ++_iter2439) { - xfer += oprot->writeString(_iter2427->first); - xfer += oprot->writeString(_iter2427->second); + xfer += oprot->writeString(_iter2439->first); + xfer += oprot->writeString(_iter2439->second); } xfer += oprot->writeMapEnd(); } @@ -30894,17 +30894,17 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_MAP) { { this->part_vals.clear(); - uint32_t _size2428; - ::apache::thrift::protocol::TType _ktype2429; - ::apache::thrift::protocol::TType _vtype2430; - xfer += iprot->readMapBegin(_ktype2429, _vtype2430, _size2428); - uint32_t _i2432; - for (_i2432 = 0; _i2432 < _size2428; ++_i2432) + uint32_t _size2440; + ::apache::thrift::protocol::TType _ktype2441; + ::apache::thrift::protocol::TType _vtype2442; + xfer += iprot->readMapBegin(_ktype2441, _vtype2442, _size2440); + uint32_t _i2444; + for (_i2444 = 0; _i2444 < _size2440; ++_i2444) { - std::string _key2433; - xfer += iprot->readString(_key2433); - std::string& _val2434 = this->part_vals[_key2433]; - xfer += iprot->readString(_val2434); + std::string _key2445; + xfer += iprot->readString(_key2445); + std::string& _val2446 = this->part_vals[_key2445]; + xfer += iprot->readString(_val2446); } xfer += iprot->readMapEnd(); } @@ -30915,9 +30915,9 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2435; - xfer += iprot->readI32(ecast2435); - this->eventType = static_cast(ecast2435); + int32_t ecast2447; + xfer += iprot->readI32(ecast2447); + this->eventType = static_cast(ecast2447); this->__isset.eventType = true; } else { xfer += iprot->skip(ftype); @@ -30951,11 +30951,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thr xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::map ::const_iterator _iter2436; - for (_iter2436 = this->part_vals.begin(); _iter2436 != this->part_vals.end(); ++_iter2436) + std::map ::const_iterator _iter2448; + for (_iter2448 = this->part_vals.begin(); _iter2448 != this->part_vals.end(); ++_iter2448) { - xfer += oprot->writeString(_iter2436->first); - xfer += oprot->writeString(_iter2436->second); + xfer += oprot->writeString(_iter2448->first); + xfer += oprot->writeString(_iter2448->second); } xfer += oprot->writeMapEnd(); } @@ -30991,11 +30991,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::th xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::map ::const_iterator _iter2437; - for (_iter2437 = (*(this->part_vals)).begin(); _iter2437 != (*(this->part_vals)).end(); ++_iter2437) + std::map ::const_iterator _iter2449; + for (_iter2449 = (*(this->part_vals)).begin(); _iter2449 != (*(this->part_vals)).end(); ++_iter2449) { - xfer += oprot->writeString(_iter2437->first); - xfer += oprot->writeString(_iter2437->second); + xfer += oprot->writeString(_iter2449->first); + xfer += oprot->writeString(_iter2449->second); } xfer += oprot->writeMapEnd(); } @@ -37124,14 +37124,14 @@ uint32_t ThriftHiveMetastore_get_functions_result::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2438; - ::apache::thrift::protocol::TType _etype2441; - xfer += iprot->readListBegin(_etype2441, _size2438); - this->success.resize(_size2438); - uint32_t _i2442; - for (_i2442 = 0; _i2442 < _size2438; ++_i2442) + uint32_t _size2450; + ::apache::thrift::protocol::TType _etype2453; + xfer += iprot->readListBegin(_etype2453, _size2450); + this->success.resize(_size2450); + uint32_t _i2454; + for (_i2454 = 0; _i2454 < _size2450; ++_i2454) { - xfer += iprot->readString(this->success[_i2442]); + xfer += iprot->readString(this->success[_i2454]); } xfer += iprot->readListEnd(); } @@ -37170,10 +37170,10 @@ uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2443; - for (_iter2443 = this->success.begin(); _iter2443 != this->success.end(); ++_iter2443) + std::vector ::const_iterator _iter2455; + for (_iter2455 = this->success.begin(); _iter2455 != this->success.end(); ++_iter2455) { - xfer += oprot->writeString((*_iter2443)); + xfer += oprot->writeString((*_iter2455)); } xfer += oprot->writeListEnd(); } @@ -37218,14 +37218,14 @@ uint32_t ThriftHiveMetastore_get_functions_presult::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2444; - ::apache::thrift::protocol::TType _etype2447; - xfer += iprot->readListBegin(_etype2447, _size2444); - (*(this->success)).resize(_size2444); - uint32_t _i2448; - for (_i2448 = 0; _i2448 < _size2444; ++_i2448) + uint32_t _size2456; + ::apache::thrift::protocol::TType _etype2459; + xfer += iprot->readListBegin(_etype2459, _size2456); + (*(this->success)).resize(_size2456); + uint32_t _i2460; + for (_i2460 = 0; _i2460 < _size2456; ++_i2460) { - xfer += iprot->readString((*(this->success))[_i2448]); + xfer += iprot->readString((*(this->success))[_i2460]); } xfer += iprot->readListEnd(); } @@ -38185,14 +38185,14 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2449; - ::apache::thrift::protocol::TType _etype2452; - xfer += iprot->readListBegin(_etype2452, _size2449); - this->success.resize(_size2449); - uint32_t _i2453; - for (_i2453 = 0; _i2453 < _size2449; ++_i2453) + uint32_t _size2461; + ::apache::thrift::protocol::TType _etype2464; + xfer += iprot->readListBegin(_etype2464, _size2461); + this->success.resize(_size2461); + uint32_t _i2465; + for (_i2465 = 0; _i2465 < _size2461; ++_i2465) { - xfer += iprot->readString(this->success[_i2453]); + xfer += iprot->readString(this->success[_i2465]); } xfer += iprot->readListEnd(); } @@ -38231,10 +38231,10 @@ uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2454; - for (_iter2454 = this->success.begin(); _iter2454 != this->success.end(); ++_iter2454) + std::vector ::const_iterator _iter2466; + for (_iter2466 = this->success.begin(); _iter2466 != this->success.end(); ++_iter2466) { - xfer += oprot->writeString((*_iter2454)); + xfer += oprot->writeString((*_iter2466)); } xfer += oprot->writeListEnd(); } @@ -38279,14 +38279,14 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2455; - ::apache::thrift::protocol::TType _etype2458; - xfer += iprot->readListBegin(_etype2458, _size2455); - (*(this->success)).resize(_size2455); - uint32_t _i2459; - for (_i2459 = 0; _i2459 < _size2455; ++_i2459) + uint32_t _size2467; + ::apache::thrift::protocol::TType _etype2470; + xfer += iprot->readListBegin(_etype2470, _size2467); + (*(this->success)).resize(_size2467); + uint32_t _i2471; + for (_i2471 = 0; _i2471 < _size2467; ++_i2471) { - xfer += iprot->readString((*(this->success))[_i2459]); + xfer += iprot->readString((*(this->success))[_i2471]); } xfer += iprot->readListEnd(); } @@ -38359,9 +38359,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2460; - xfer += iprot->readI32(ecast2460); - this->principal_type = static_cast(ecast2460); + int32_t ecast2472; + xfer += iprot->readI32(ecast2472); + this->principal_type = static_cast(ecast2472); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -38377,9 +38377,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T break; case 5: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2461; - xfer += iprot->readI32(ecast2461); - this->grantorType = static_cast(ecast2461); + int32_t ecast2473; + xfer += iprot->readI32(ecast2473); + this->grantorType = static_cast(ecast2473); this->__isset.grantorType = true; } else { xfer += iprot->skip(ftype); @@ -38650,9 +38650,9 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol:: break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2462; - xfer += iprot->readI32(ecast2462); - this->principal_type = static_cast(ecast2462); + int32_t ecast2474; + xfer += iprot->readI32(ecast2474); + this->principal_type = static_cast(ecast2474); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -38883,9 +38883,9 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2463; - xfer += iprot->readI32(ecast2463); - this->principal_type = static_cast(ecast2463); + int32_t ecast2475; + xfer += iprot->readI32(ecast2475); + this->principal_type = static_cast(ecast2475); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -38974,14 +38974,14 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2464; - ::apache::thrift::protocol::TType _etype2467; - xfer += iprot->readListBegin(_etype2467, _size2464); - this->success.resize(_size2464); - uint32_t _i2468; - for (_i2468 = 0; _i2468 < _size2464; ++_i2468) + uint32_t _size2476; + ::apache::thrift::protocol::TType _etype2479; + xfer += iprot->readListBegin(_etype2479, _size2476); + this->success.resize(_size2476); + uint32_t _i2480; + for (_i2480 = 0; _i2480 < _size2476; ++_i2480) { - xfer += this->success[_i2468].read(iprot); + xfer += this->success[_i2480].read(iprot); } xfer += iprot->readListEnd(); } @@ -39020,10 +39020,10 @@ uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2469; - for (_iter2469 = this->success.begin(); _iter2469 != this->success.end(); ++_iter2469) + std::vector ::const_iterator _iter2481; + for (_iter2481 = this->success.begin(); _iter2481 != this->success.end(); ++_iter2481) { - xfer += (*_iter2469).write(oprot); + xfer += (*_iter2481).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39068,14 +39068,14 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2470; - ::apache::thrift::protocol::TType _etype2473; - xfer += iprot->readListBegin(_etype2473, _size2470); - (*(this->success)).resize(_size2470); - uint32_t _i2474; - for (_i2474 = 0; _i2474 < _size2470; ++_i2474) + uint32_t _size2482; + ::apache::thrift::protocol::TType _etype2485; + xfer += iprot->readListBegin(_etype2485, _size2482); + (*(this->success)).resize(_size2482); + uint32_t _i2486; + for (_i2486 = 0; _i2486 < _size2482; ++_i2486) { - xfer += (*(this->success))[_i2474].read(iprot); + xfer += (*(this->success))[_i2486].read(iprot); } xfer += iprot->readListEnd(); } @@ -39771,14 +39771,14 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2475; - ::apache::thrift::protocol::TType _etype2478; - xfer += iprot->readListBegin(_etype2478, _size2475); - this->group_names.resize(_size2475); - uint32_t _i2479; - for (_i2479 = 0; _i2479 < _size2475; ++_i2479) + uint32_t _size2487; + ::apache::thrift::protocol::TType _etype2490; + xfer += iprot->readListBegin(_etype2490, _size2487); + this->group_names.resize(_size2487); + uint32_t _i2491; + for (_i2491 = 0; _i2491 < _size2487; ++_i2491) { - xfer += iprot->readString(this->group_names[_i2479]); + xfer += iprot->readString(this->group_names[_i2491]); } xfer += iprot->readListEnd(); } @@ -39815,10 +39815,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2480; - for (_iter2480 = this->group_names.begin(); _iter2480 != this->group_names.end(); ++_iter2480) + std::vector ::const_iterator _iter2492; + for (_iter2492 = this->group_names.begin(); _iter2492 != this->group_names.end(); ++_iter2492) { - xfer += oprot->writeString((*_iter2480)); + xfer += oprot->writeString((*_iter2492)); } xfer += oprot->writeListEnd(); } @@ -39850,10 +39850,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2481; - for (_iter2481 = (*(this->group_names)).begin(); _iter2481 != (*(this->group_names)).end(); ++_iter2481) + std::vector ::const_iterator _iter2493; + for (_iter2493 = (*(this->group_names)).begin(); _iter2493 != (*(this->group_names)).end(); ++_iter2493) { - xfer += oprot->writeString((*_iter2481)); + xfer += oprot->writeString((*_iter2493)); } xfer += oprot->writeListEnd(); } @@ -40028,9 +40028,9 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2482; - xfer += iprot->readI32(ecast2482); - this->principal_type = static_cast(ecast2482); + int32_t ecast2494; + xfer += iprot->readI32(ecast2494); + this->principal_type = static_cast(ecast2494); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -40135,14 +40135,14 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2483; - ::apache::thrift::protocol::TType _etype2486; - xfer += iprot->readListBegin(_etype2486, _size2483); - this->success.resize(_size2483); - uint32_t _i2487; - for (_i2487 = 0; _i2487 < _size2483; ++_i2487) + uint32_t _size2495; + ::apache::thrift::protocol::TType _etype2498; + xfer += iprot->readListBegin(_etype2498, _size2495); + this->success.resize(_size2495); + uint32_t _i2499; + for (_i2499 = 0; _i2499 < _size2495; ++_i2499) { - xfer += this->success[_i2487].read(iprot); + xfer += this->success[_i2499].read(iprot); } xfer += iprot->readListEnd(); } @@ -40181,10 +40181,10 @@ uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2488; - for (_iter2488 = this->success.begin(); _iter2488 != this->success.end(); ++_iter2488) + std::vector ::const_iterator _iter2500; + for (_iter2500 = this->success.begin(); _iter2500 != this->success.end(); ++_iter2500) { - xfer += (*_iter2488).write(oprot); + xfer += (*_iter2500).write(oprot); } xfer += oprot->writeListEnd(); } @@ -40229,14 +40229,14 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2489; - ::apache::thrift::protocol::TType _etype2492; - xfer += iprot->readListBegin(_etype2492, _size2489); - (*(this->success)).resize(_size2489); - uint32_t _i2493; - for (_i2493 = 0; _i2493 < _size2489; ++_i2493) + uint32_t _size2501; + ::apache::thrift::protocol::TType _etype2504; + xfer += iprot->readListBegin(_etype2504, _size2501); + (*(this->success)).resize(_size2501); + uint32_t _i2505; + for (_i2505 = 0; _i2505 < _size2501; ++_i2505) { - xfer += (*(this->success))[_i2493].read(iprot); + xfer += (*(this->success))[_i2505].read(iprot); } xfer += iprot->readListEnd(); } @@ -41163,14 +41163,14 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2494; - ::apache::thrift::protocol::TType _etype2497; - xfer += iprot->readListBegin(_etype2497, _size2494); - this->group_names.resize(_size2494); - uint32_t _i2498; - for (_i2498 = 0; _i2498 < _size2494; ++_i2498) + uint32_t _size2506; + ::apache::thrift::protocol::TType _etype2509; + xfer += iprot->readListBegin(_etype2509, _size2506); + this->group_names.resize(_size2506); + uint32_t _i2510; + for (_i2510 = 0; _i2510 < _size2506; ++_i2510) { - xfer += iprot->readString(this->group_names[_i2498]); + xfer += iprot->readString(this->group_names[_i2510]); } xfer += iprot->readListEnd(); } @@ -41203,10 +41203,10 @@ uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TPr xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2499; - for (_iter2499 = this->group_names.begin(); _iter2499 != this->group_names.end(); ++_iter2499) + std::vector ::const_iterator _iter2511; + for (_iter2511 = this->group_names.begin(); _iter2511 != this->group_names.end(); ++_iter2511) { - xfer += oprot->writeString((*_iter2499)); + xfer += oprot->writeString((*_iter2511)); } xfer += oprot->writeListEnd(); } @@ -41234,10 +41234,10 @@ uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TP xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2500; - for (_iter2500 = (*(this->group_names)).begin(); _iter2500 != (*(this->group_names)).end(); ++_iter2500) + std::vector ::const_iterator _iter2512; + for (_iter2512 = (*(this->group_names)).begin(); _iter2512 != (*(this->group_names)).end(); ++_iter2512) { - xfer += oprot->writeString((*_iter2500)); + xfer += oprot->writeString((*_iter2512)); } xfer += oprot->writeListEnd(); } @@ -41278,14 +41278,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2501; - ::apache::thrift::protocol::TType _etype2504; - xfer += iprot->readListBegin(_etype2504, _size2501); - this->success.resize(_size2501); - uint32_t _i2505; - for (_i2505 = 0; _i2505 < _size2501; ++_i2505) + uint32_t _size2513; + ::apache::thrift::protocol::TType _etype2516; + xfer += iprot->readListBegin(_etype2516, _size2513); + this->success.resize(_size2513); + uint32_t _i2517; + for (_i2517 = 0; _i2517 < _size2513; ++_i2517) { - xfer += iprot->readString(this->success[_i2505]); + xfer += iprot->readString(this->success[_i2517]); } xfer += iprot->readListEnd(); } @@ -41324,10 +41324,10 @@ uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::T xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2506; - for (_iter2506 = this->success.begin(); _iter2506 != this->success.end(); ++_iter2506) + std::vector ::const_iterator _iter2518; + for (_iter2518 = this->success.begin(); _iter2518 != this->success.end(); ++_iter2518) { - xfer += oprot->writeString((*_iter2506)); + xfer += oprot->writeString((*_iter2518)); } xfer += oprot->writeListEnd(); } @@ -41372,14 +41372,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2507; - ::apache::thrift::protocol::TType _etype2510; - xfer += iprot->readListBegin(_etype2510, _size2507); - (*(this->success)).resize(_size2507); - uint32_t _i2511; - for (_i2511 = 0; _i2511 < _size2507; ++_i2511) + uint32_t _size2519; + ::apache::thrift::protocol::TType _etype2522; + xfer += iprot->readListBegin(_etype2522, _size2519); + (*(this->success)).resize(_size2519); + uint32_t _i2523; + for (_i2523 = 0; _i2523 < _size2519; ++_i2523) { - xfer += iprot->readString((*(this->success))[_i2511]); + xfer += iprot->readString((*(this->success))[_i2523]); } xfer += iprot->readListEnd(); } @@ -42690,14 +42690,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2512; - ::apache::thrift::protocol::TType _etype2515; - xfer += iprot->readListBegin(_etype2515, _size2512); - this->success.resize(_size2512); - uint32_t _i2516; - for (_i2516 = 0; _i2516 < _size2512; ++_i2516) + uint32_t _size2524; + ::apache::thrift::protocol::TType _etype2527; + xfer += iprot->readListBegin(_etype2527, _size2524); + this->success.resize(_size2524); + uint32_t _i2528; + for (_i2528 = 0; _i2528 < _size2524; ++_i2528) { - xfer += iprot->readString(this->success[_i2516]); + xfer += iprot->readString(this->success[_i2528]); } xfer += iprot->readListEnd(); } @@ -42728,10 +42728,10 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2517; - for (_iter2517 = this->success.begin(); _iter2517 != this->success.end(); ++_iter2517) + std::vector ::const_iterator _iter2529; + for (_iter2529 = this->success.begin(); _iter2529 != this->success.end(); ++_iter2529) { - xfer += oprot->writeString((*_iter2517)); + xfer += oprot->writeString((*_iter2529)); } xfer += oprot->writeListEnd(); } @@ -42772,14 +42772,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2518; - ::apache::thrift::protocol::TType _etype2521; - xfer += iprot->readListBegin(_etype2521, _size2518); - (*(this->success)).resize(_size2518); - uint32_t _i2522; - for (_i2522 = 0; _i2522 < _size2518; ++_i2522) + uint32_t _size2530; + ::apache::thrift::protocol::TType _etype2533; + xfer += iprot->readListBegin(_etype2533, _size2530); + (*(this->success)).resize(_size2530); + uint32_t _i2534; + for (_i2534 = 0; _i2534 < _size2530; ++_i2534) { - xfer += iprot->readString((*(this->success))[_i2522]); + xfer += iprot->readString((*(this->success))[_i2534]); } xfer += iprot->readListEnd(); } @@ -43505,14 +43505,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2523; - ::apache::thrift::protocol::TType _etype2526; - xfer += iprot->readListBegin(_etype2526, _size2523); - this->success.resize(_size2523); - uint32_t _i2527; - for (_i2527 = 0; _i2527 < _size2523; ++_i2527) + uint32_t _size2535; + ::apache::thrift::protocol::TType _etype2538; + xfer += iprot->readListBegin(_etype2538, _size2535); + this->success.resize(_size2535); + uint32_t _i2539; + for (_i2539 = 0; _i2539 < _size2535; ++_i2539) { - xfer += iprot->readString(this->success[_i2527]); + xfer += iprot->readString(this->success[_i2539]); } xfer += iprot->readListEnd(); } @@ -43543,10 +43543,10 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2528; - for (_iter2528 = this->success.begin(); _iter2528 != this->success.end(); ++_iter2528) + std::vector ::const_iterator _iter2540; + for (_iter2540 = this->success.begin(); _iter2540 != this->success.end(); ++_iter2540) { - xfer += oprot->writeString((*_iter2528)); + xfer += oprot->writeString((*_iter2540)); } xfer += oprot->writeListEnd(); } @@ -43587,14 +43587,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_presult::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2529; - ::apache::thrift::protocol::TType _etype2532; - xfer += iprot->readListBegin(_etype2532, _size2529); - (*(this->success)).resize(_size2529); - uint32_t _i2533; - for (_i2533 = 0; _i2533 < _size2529; ++_i2533) + uint32_t _size2541; + ::apache::thrift::protocol::TType _etype2544; + xfer += iprot->readListBegin(_etype2544, _size2541); + (*(this->success)).resize(_size2541); + uint32_t _i2545; + for (_i2545 = 0; _i2545 < _size2541; ++_i2545) { - xfer += iprot->readString((*(this->success))[_i2533]); + xfer += iprot->readString((*(this->success))[_i2545]); } xfer += iprot->readListEnd(); } @@ -45343,17 +45343,17 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::read(::apache::t if (ftype == ::apache::thrift::protocol::T_MAP) { { this->writeIds.clear(); - uint32_t _size2534; - ::apache::thrift::protocol::TType _ktype2535; - ::apache::thrift::protocol::TType _vtype2536; - xfer += iprot->readMapBegin(_ktype2535, _vtype2536, _size2534); - uint32_t _i2538; - for (_i2538 = 0; _i2538 < _size2534; ++_i2538) + uint32_t _size2546; + ::apache::thrift::protocol::TType _ktype2547; + ::apache::thrift::protocol::TType _vtype2548; + xfer += iprot->readMapBegin(_ktype2547, _vtype2548, _size2546); + uint32_t _i2550; + for (_i2550 = 0; _i2550 < _size2546; ++_i2550) { - std::string _key2539; - xfer += iprot->readString(_key2539); - int64_t& _val2540 = this->writeIds[_key2539]; - xfer += iprot->readI64(_val2540); + std::string _key2551; + xfer += iprot->readString(_key2551); + int64_t& _val2552 = this->writeIds[_key2551]; + xfer += iprot->readI64(_val2552); } xfer += iprot->readMapEnd(); } @@ -45386,11 +45386,11 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::write(::apache:: xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast(this->writeIds.size())); - std::map ::const_iterator _iter2541; - for (_iter2541 = this->writeIds.begin(); _iter2541 != this->writeIds.end(); ++_iter2541) + std::map ::const_iterator _iter2553; + for (_iter2553 = this->writeIds.begin(); _iter2553 != this->writeIds.end(); ++_iter2553) { - xfer += oprot->writeString(_iter2541->first); - xfer += oprot->writeI64(_iter2541->second); + xfer += oprot->writeString(_iter2553->first); + xfer += oprot->writeI64(_iter2553->second); } xfer += oprot->writeMapEnd(); } @@ -45418,11 +45418,11 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_pargs::write(::apache: xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast((*(this->writeIds)).size())); - std::map ::const_iterator _iter2542; - for (_iter2542 = (*(this->writeIds)).begin(); _iter2542 != (*(this->writeIds)).end(); ++_iter2542) + std::map ::const_iterator _iter2554; + for (_iter2554 = (*(this->writeIds)).begin(); _iter2554 != (*(this->writeIds)).end(); ++_iter2554) { - xfer += oprot->writeString(_iter2542->first); - xfer += oprot->writeI64(_iter2542->second); + xfer += oprot->writeString(_iter2554->first); + xfer += oprot->writeI64(_iter2554->second); } xfer += oprot->writeMapEnd(); } @@ -49322,14 +49322,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2543; - ::apache::thrift::protocol::TType _etype2546; - xfer += iprot->readListBegin(_etype2546, _size2543); - this->success.resize(_size2543); - uint32_t _i2547; - for (_i2547 = 0; _i2547 < _size2543; ++_i2547) + uint32_t _size2555; + ::apache::thrift::protocol::TType _etype2558; + xfer += iprot->readListBegin(_etype2558, _size2555); + this->success.resize(_size2555); + uint32_t _i2559; + for (_i2559 = 0; _i2559 < _size2555; ++_i2559) { - xfer += iprot->readString(this->success[_i2547]); + xfer += iprot->readString(this->success[_i2559]); } xfer += iprot->readListEnd(); } @@ -49360,10 +49360,10 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2548; - for (_iter2548 = this->success.begin(); _iter2548 != this->success.end(); ++_iter2548) + std::vector ::const_iterator _iter2560; + for (_iter2560 = this->success.begin(); _iter2560 != this->success.end(); ++_iter2560) { - xfer += oprot->writeString((*_iter2548)); + xfer += oprot->writeString((*_iter2560)); } xfer += oprot->writeListEnd(); } @@ -49404,14 +49404,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2549; - ::apache::thrift::protocol::TType _etype2552; - xfer += iprot->readListBegin(_etype2552, _size2549); - (*(this->success)).resize(_size2549); - uint32_t _i2553; - for (_i2553 = 0; _i2553 < _size2549; ++_i2553) + uint32_t _size2561; + ::apache::thrift::protocol::TType _etype2564; + xfer += iprot->readListBegin(_etype2564, _size2561); + (*(this->success)).resize(_size2561); + uint32_t _i2565; + for (_i2565 = 0; _i2565 < _size2561; ++_i2565) { - xfer += iprot->readString((*(this->success))[_i2553]); + xfer += iprot->readString((*(this->success))[_i2565]); } xfer += iprot->readListEnd(); } @@ -59334,14 +59334,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2554; - ::apache::thrift::protocol::TType _etype2557; - xfer += iprot->readListBegin(_etype2557, _size2554); - this->success.resize(_size2554); - uint32_t _i2558; - for (_i2558 = 0; _i2558 < _size2554; ++_i2558) + uint32_t _size2566; + ::apache::thrift::protocol::TType _etype2569; + xfer += iprot->readListBegin(_etype2569, _size2566); + this->success.resize(_size2566); + uint32_t _i2570; + for (_i2570 = 0; _i2570 < _size2566; ++_i2570) { - xfer += this->success[_i2558].read(iprot); + xfer += this->success[_i2570].read(iprot); } xfer += iprot->readListEnd(); } @@ -59388,10 +59388,10 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2559; - for (_iter2559 = this->success.begin(); _iter2559 != this->success.end(); ++_iter2559) + std::vector ::const_iterator _iter2571; + for (_iter2571 = this->success.begin(); _iter2571 != this->success.end(); ++_iter2571) { - xfer += (*_iter2559).write(oprot); + xfer += (*_iter2571).write(oprot); } xfer += oprot->writeListEnd(); } @@ -59440,14 +59440,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2560; - ::apache::thrift::protocol::TType _etype2563; - xfer += iprot->readListBegin(_etype2563, _size2560); - (*(this->success)).resize(_size2560); - uint32_t _i2564; - for (_i2564 = 0; _i2564 < _size2560; ++_i2564) + uint32_t _size2572; + ::apache::thrift::protocol::TType _etype2575; + xfer += iprot->readListBegin(_etype2575, _size2572); + (*(this->success)).resize(_size2572); + uint32_t _i2576; + for (_i2576 = 0; _i2576 < _size2572; ++_i2576) { - xfer += (*(this->success))[_i2564].read(iprot); + xfer += (*(this->success))[_i2576].read(iprot); } xfer += iprot->readListEnd(); } @@ -61500,14 +61500,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2565; - ::apache::thrift::protocol::TType _etype2568; - xfer += iprot->readListBegin(_etype2568, _size2565); - this->success.resize(_size2565); - uint32_t _i2569; - for (_i2569 = 0; _i2569 < _size2565; ++_i2569) + uint32_t _size2577; + ::apache::thrift::protocol::TType _etype2580; + xfer += iprot->readListBegin(_etype2580, _size2577); + this->success.resize(_size2577); + uint32_t _i2581; + for (_i2581 = 0; _i2581 < _size2577; ++_i2581) { - xfer += this->success[_i2569].read(iprot); + xfer += this->success[_i2581].read(iprot); } xfer += iprot->readListEnd(); } @@ -61546,10 +61546,10 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::write(::apache::thrift::p xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2570; - for (_iter2570 = this->success.begin(); _iter2570 != this->success.end(); ++_iter2570) + std::vector ::const_iterator _iter2582; + for (_iter2582 = this->success.begin(); _iter2582 != this->success.end(); ++_iter2582) { - xfer += (*_iter2570).write(oprot); + xfer += (*_iter2582).write(oprot); } xfer += oprot->writeListEnd(); } @@ -61594,14 +61594,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2571; - ::apache::thrift::protocol::TType _etype2574; - xfer += iprot->readListBegin(_etype2574, _size2571); - (*(this->success)).resize(_size2571); - uint32_t _i2575; - for (_i2575 = 0; _i2575 < _size2571; ++_i2575) + uint32_t _size2583; + ::apache::thrift::protocol::TType _etype2586; + xfer += iprot->readListBegin(_etype2586, _size2583); + (*(this->success)).resize(_size2583); + uint32_t _i2587; + for (_i2587 = 0; _i2587 < _size2583; ++_i2587) { - xfer += (*(this->success))[_i2575].read(iprot); + xfer += (*(this->success))[_i2587].read(iprot); } xfer += iprot->readListEnd(); } @@ -64036,14 +64036,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2576; - ::apache::thrift::protocol::TType _etype2579; - xfer += iprot->readListBegin(_etype2579, _size2576); - this->success.resize(_size2576); - uint32_t _i2580; - for (_i2580 = 0; _i2580 < _size2576; ++_i2580) + uint32_t _size2588; + ::apache::thrift::protocol::TType _etype2591; + xfer += iprot->readListBegin(_etype2591, _size2588); + this->success.resize(_size2588); + uint32_t _i2592; + for (_i2592 = 0; _i2592 < _size2588; ++_i2592) { - xfer += iprot->readString(this->success[_i2580]); + xfer += iprot->readString(this->success[_i2592]); } xfer += iprot->readListEnd(); } @@ -64082,10 +64082,10 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2581; - for (_iter2581 = this->success.begin(); _iter2581 != this->success.end(); ++_iter2581) + std::vector ::const_iterator _iter2593; + for (_iter2593 = this->success.begin(); _iter2593 != this->success.end(); ++_iter2593) { - xfer += oprot->writeString((*_iter2581)); + xfer += oprot->writeString((*_iter2593)); } xfer += oprot->writeListEnd(); } @@ -64130,14 +64130,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2582; - ::apache::thrift::protocol::TType _etype2585; - xfer += iprot->readListBegin(_etype2585, _size2582); - (*(this->success)).resize(_size2582); - uint32_t _i2586; - for (_i2586 = 0; _i2586 < _size2582; ++_i2586) + uint32_t _size2594; + ::apache::thrift::protocol::TType _etype2597; + xfer += iprot->readListBegin(_etype2597, _size2594); + (*(this->success)).resize(_size2594); + uint32_t _i2598; + for (_i2598 = 0; _i2598 < _size2594; ++_i2598) { - xfer += iprot->readString((*(this->success))[_i2586]); + xfer += iprot->readString((*(this->success))[_i2598]); } xfer += iprot->readListEnd(); } @@ -64689,14 +64689,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2587; - ::apache::thrift::protocol::TType _etype2590; - xfer += iprot->readListBegin(_etype2590, _size2587); - this->success.resize(_size2587); - uint32_t _i2591; - for (_i2591 = 0; _i2591 < _size2587; ++_i2591) + uint32_t _size2599; + ::apache::thrift::protocol::TType _etype2602; + xfer += iprot->readListBegin(_etype2602, _size2599); + this->success.resize(_size2599); + uint32_t _i2603; + for (_i2603 = 0; _i2603 < _size2599; ++_i2603) { - xfer += iprot->readString(this->success[_i2591]); + xfer += iprot->readString(this->success[_i2603]); } xfer += iprot->readListEnd(); } @@ -64735,10 +64735,10 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2592; - for (_iter2592 = this->success.begin(); _iter2592 != this->success.end(); ++_iter2592) + std::vector ::const_iterator _iter2604; + for (_iter2604 = this->success.begin(); _iter2604 != this->success.end(); ++_iter2604) { - xfer += oprot->writeString((*_iter2592)); + xfer += oprot->writeString((*_iter2604)); } xfer += oprot->writeListEnd(); } @@ -64783,14 +64783,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_presult::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2593; - ::apache::thrift::protocol::TType _etype2596; - xfer += iprot->readListBegin(_etype2596, _size2593); - (*(this->success)).resize(_size2593); - uint32_t _i2597; - for (_i2597 = 0; _i2597 < _size2593; ++_i2597) + uint32_t _size2605; + ::apache::thrift::protocol::TType _etype2608; + xfer += iprot->readListBegin(_etype2608, _size2605); + (*(this->success)).resize(_size2605); + uint32_t _i2609; + for (_i2609 = 0; _i2609 < _size2605; ++_i2609) { - xfer += iprot->readString((*(this->success))[_i2597]); + xfer += iprot->readString((*(this->success))[_i2609]); } xfer += iprot->readListEnd(); } @@ -65115,14 +65115,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2598; - ::apache::thrift::protocol::TType _etype2601; - xfer += iprot->readListBegin(_etype2601, _size2598); - this->success.resize(_size2598); - uint32_t _i2602; - for (_i2602 = 0; _i2602 < _size2598; ++_i2602) + uint32_t _size2610; + ::apache::thrift::protocol::TType _etype2613; + xfer += iprot->readListBegin(_etype2613, _size2610); + this->success.resize(_size2610); + uint32_t _i2614; + for (_i2614 = 0; _i2614 < _size2610; ++_i2614) { - xfer += this->success[_i2602].read(iprot); + xfer += this->success[_i2614].read(iprot); } xfer += iprot->readListEnd(); } @@ -65161,10 +65161,10 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2603; - for (_iter2603 = this->success.begin(); _iter2603 != this->success.end(); ++_iter2603) + std::vector ::const_iterator _iter2615; + for (_iter2615 = this->success.begin(); _iter2615 != this->success.end(); ++_iter2615) { - xfer += (*_iter2603).write(oprot); + xfer += (*_iter2615).write(oprot); } xfer += oprot->writeListEnd(); } @@ -65209,14 +65209,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2604; - ::apache::thrift::protocol::TType _etype2607; - xfer += iprot->readListBegin(_etype2607, _size2604); - (*(this->success)).resize(_size2604); - uint32_t _i2608; - for (_i2608 = 0; _i2608 < _size2604; ++_i2608) + uint32_t _size2616; + ::apache::thrift::protocol::TType _etype2619; + xfer += iprot->readListBegin(_etype2619, _size2616); + (*(this->success)).resize(_size2616); + uint32_t _i2620; + for (_i2620 = 0; _i2620 < _size2616; ++_i2620) { - xfer += (*(this->success))[_i2608].read(iprot); + xfer += (*(this->success))[_i2620].read(iprot); } xfer += iprot->readListEnd(); } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index 80a7959c5143..a0b3bc5ad88c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -31307,6 +31307,21 @@ void NotificationEventRequest::__set_eventTypeSkipList(const std::vectoreventTypeSkipList = val; __isset.eventTypeSkipList = true; } + +void NotificationEventRequest::__set_catName(const std::string& val) { + this->catName = val; +__isset.catName = true; +} + +void NotificationEventRequest::__set_dbName(const std::string& val) { + this->dbName = val; +__isset.dbName = true; +} + +void NotificationEventRequest::__set_tableNames(const std::vector & val) { + this->tableNames = val; +__isset.tableNames = true; +} std::ostream& operator<<(std::ostream& out, const NotificationEventRequest& obj) { obj.printTo(out); @@ -31372,6 +31387,42 @@ uint32_t NotificationEventRequest::read(::apache::thrift::protocol::TProtocol* i xfer += iprot->skip(ftype); } break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->catName); + this->__isset.catName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->tableNames.clear(); + uint32_t _size1111; + ::apache::thrift::protocol::TType _etype1114; + xfer += iprot->readListBegin(_etype1114, _size1111); + this->tableNames.resize(_size1111); + uint32_t _i1115; + for (_i1115 = 0; _i1115 < _size1111; ++_i1115) + { + xfer += iprot->readString(this->tableNames[_i1115]); + } + xfer += iprot->readListEnd(); + } + this->__isset.tableNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -31404,10 +31455,33 @@ uint32_t NotificationEventRequest::write(::apache::thrift::protocol::TProtocol* xfer += oprot->writeFieldBegin("eventTypeSkipList", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->eventTypeSkipList.size())); - std::vector ::const_iterator _iter1111; - for (_iter1111 = this->eventTypeSkipList.begin(); _iter1111 != this->eventTypeSkipList.end(); ++_iter1111) + std::vector ::const_iterator _iter1116; + for (_iter1116 = this->eventTypeSkipList.begin(); _iter1116 != this->eventTypeSkipList.end(); ++_iter1116) + { + xfer += oprot->writeString((*_iter1116)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.catName) { + xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->catName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.dbName) { + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.tableNames) { + xfer += oprot->writeFieldBegin("tableNames", ::apache::thrift::protocol::T_LIST, 6); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tableNames.size())); + std::vector ::const_iterator _iter1117; + for (_iter1117 = this->tableNames.begin(); _iter1117 != this->tableNames.end(); ++_iter1117) { - xfer += oprot->writeString((*_iter1111)); + xfer += oprot->writeString((*_iter1117)); } xfer += oprot->writeListEnd(); } @@ -31423,20 +31497,29 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) { swap(a.lastEvent, b.lastEvent); swap(a.maxEvents, b.maxEvents); swap(a.eventTypeSkipList, b.eventTypeSkipList); + swap(a.catName, b.catName); + swap(a.dbName, b.dbName); + swap(a.tableNames, b.tableNames); swap(a.__isset, b.__isset); } -NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other1112) { - lastEvent = other1112.lastEvent; - maxEvents = other1112.maxEvents; - eventTypeSkipList = other1112.eventTypeSkipList; - __isset = other1112.__isset; +NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other1118) { + lastEvent = other1118.lastEvent; + maxEvents = other1118.maxEvents; + eventTypeSkipList = other1118.eventTypeSkipList; + catName = other1118.catName; + dbName = other1118.dbName; + tableNames = other1118.tableNames; + __isset = other1118.__isset; } -NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other1113) { - lastEvent = other1113.lastEvent; - maxEvents = other1113.maxEvents; - eventTypeSkipList = other1113.eventTypeSkipList; - __isset = other1113.__isset; +NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other1119) { + lastEvent = other1119.lastEvent; + maxEvents = other1119.maxEvents; + eventTypeSkipList = other1119.eventTypeSkipList; + catName = other1119.catName; + dbName = other1119.dbName; + tableNames = other1119.tableNames; + __isset = other1119.__isset; return *this; } void NotificationEventRequest::printTo(std::ostream& out) const { @@ -31445,6 +31528,9 @@ void NotificationEventRequest::printTo(std::ostream& out) const { out << "lastEvent=" << to_string(lastEvent); out << ", " << "maxEvents="; (__isset.maxEvents ? (out << to_string(maxEvents)) : (out << "")); out << ", " << "eventTypeSkipList="; (__isset.eventTypeSkipList ? (out << to_string(eventTypeSkipList)) : (out << "")); + out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "")); + out << ", " << "dbName="; (__isset.dbName ? (out << to_string(dbName)) : (out << "")); + out << ", " << "tableNames="; (__isset.tableNames ? (out << to_string(tableNames)) : (out << "")); out << ")"; } @@ -31663,27 +31749,27 @@ void swap(NotificationEvent &a, NotificationEvent &b) { swap(a.__isset, b.__isset); } -NotificationEvent::NotificationEvent(const NotificationEvent& other1114) { - eventId = other1114.eventId; - eventTime = other1114.eventTime; - eventType = other1114.eventType; - dbName = other1114.dbName; - tableName = other1114.tableName; - message = other1114.message; - messageFormat = other1114.messageFormat; - catName = other1114.catName; - __isset = other1114.__isset; -} -NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other1115) { - eventId = other1115.eventId; - eventTime = other1115.eventTime; - eventType = other1115.eventType; - dbName = other1115.dbName; - tableName = other1115.tableName; - message = other1115.message; - messageFormat = other1115.messageFormat; - catName = other1115.catName; - __isset = other1115.__isset; +NotificationEvent::NotificationEvent(const NotificationEvent& other1120) { + eventId = other1120.eventId; + eventTime = other1120.eventTime; + eventType = other1120.eventType; + dbName = other1120.dbName; + tableName = other1120.tableName; + message = other1120.message; + messageFormat = other1120.messageFormat; + catName = other1120.catName; + __isset = other1120.__isset; +} +NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other1121) { + eventId = other1121.eventId; + eventTime = other1121.eventTime; + eventType = other1121.eventType; + dbName = other1121.dbName; + tableName = other1121.tableName; + message = other1121.message; + messageFormat = other1121.messageFormat; + catName = other1121.catName; + __isset = other1121.__isset; return *this; } void NotificationEvent::printTo(std::ostream& out) const { @@ -31741,14 +31827,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol* if (ftype == ::apache::thrift::protocol::T_LIST) { { this->events.clear(); - uint32_t _size1116; - ::apache::thrift::protocol::TType _etype1119; - xfer += iprot->readListBegin(_etype1119, _size1116); - this->events.resize(_size1116); - uint32_t _i1120; - for (_i1120 = 0; _i1120 < _size1116; ++_i1120) + uint32_t _size1122; + ::apache::thrift::protocol::TType _etype1125; + xfer += iprot->readListBegin(_etype1125, _size1122); + this->events.resize(_size1122); + uint32_t _i1126; + for (_i1126 = 0; _i1126 < _size1122; ++_i1126) { - xfer += this->events[_i1120].read(iprot); + xfer += this->events[_i1126].read(iprot); } xfer += iprot->readListEnd(); } @@ -31779,10 +31865,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol* xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->events.size())); - std::vector ::const_iterator _iter1121; - for (_iter1121 = this->events.begin(); _iter1121 != this->events.end(); ++_iter1121) + std::vector ::const_iterator _iter1127; + for (_iter1127 = this->events.begin(); _iter1127 != this->events.end(); ++_iter1127) { - xfer += (*_iter1121).write(oprot); + xfer += (*_iter1127).write(oprot); } xfer += oprot->writeListEnd(); } @@ -31798,11 +31884,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) { swap(a.events, b.events); } -NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other1122) { - events = other1122.events; +NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other1128) { + events = other1128.events; } -NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other1123) { - events = other1123.events; +NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other1129) { + events = other1129.events; return *this; } void NotificationEventResponse::printTo(std::ostream& out) const { @@ -31890,11 +31976,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) { swap(a.eventId, b.eventId); } -CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other1124) noexcept { - eventId = other1124.eventId; +CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other1130) noexcept { + eventId = other1130.eventId; } -CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other1125) noexcept { - eventId = other1125.eventId; +CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other1131) noexcept { + eventId = other1131.eventId; return *this; } void CurrentNotificationEventId::printTo(std::ostream& out) const { @@ -31931,6 +32017,11 @@ void NotificationEventsCountRequest::__set_limit(const int64_t val) { this->limit = val; __isset.limit = true; } + +void NotificationEventsCountRequest::__set_tableNames(const std::vector & val) { + this->tableNames = val; +__isset.tableNames = true; +} std::ostream& operator<<(std::ostream& out, const NotificationEventsCountRequest& obj) { obj.printTo(out); @@ -32001,6 +32092,26 @@ uint32_t NotificationEventsCountRequest::read(::apache::thrift::protocol::TProto xfer += iprot->skip(ftype); } break; + case 6: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->tableNames.clear(); + uint32_t _size1132; + ::apache::thrift::protocol::TType _etype1135; + xfer += iprot->readListBegin(_etype1135, _size1132); + this->tableNames.resize(_size1132); + uint32_t _i1136; + for (_i1136 = 0; _i1136 < _size1132; ++_i1136) + { + xfer += iprot->readString(this->tableNames[_i1136]); + } + xfer += iprot->readListEnd(); + } + this->__isset.tableNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -32045,6 +32156,19 @@ uint32_t NotificationEventsCountRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeI64(this->limit); xfer += oprot->writeFieldEnd(); } + if (this->__isset.tableNames) { + xfer += oprot->writeFieldBegin("tableNames", ::apache::thrift::protocol::T_LIST, 6); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tableNames.size())); + std::vector ::const_iterator _iter1137; + for (_iter1137 = this->tableNames.begin(); _iter1137 != this->tableNames.end(); ++_iter1137) + { + xfer += oprot->writeString((*_iter1137)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -32057,24 +32181,27 @@ void swap(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b) swap(a.catName, b.catName); swap(a.toEventId, b.toEventId); swap(a.limit, b.limit); + swap(a.tableNames, b.tableNames); swap(a.__isset, b.__isset); } -NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other1126) { - fromEventId = other1126.fromEventId; - dbName = other1126.dbName; - catName = other1126.catName; - toEventId = other1126.toEventId; - limit = other1126.limit; - __isset = other1126.__isset; +NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other1138) { + fromEventId = other1138.fromEventId; + dbName = other1138.dbName; + catName = other1138.catName; + toEventId = other1138.toEventId; + limit = other1138.limit; + tableNames = other1138.tableNames; + __isset = other1138.__isset; } -NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other1127) { - fromEventId = other1127.fromEventId; - dbName = other1127.dbName; - catName = other1127.catName; - toEventId = other1127.toEventId; - limit = other1127.limit; - __isset = other1127.__isset; +NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other1139) { + fromEventId = other1139.fromEventId; + dbName = other1139.dbName; + catName = other1139.catName; + toEventId = other1139.toEventId; + limit = other1139.limit; + tableNames = other1139.tableNames; + __isset = other1139.__isset; return *this; } void NotificationEventsCountRequest::printTo(std::ostream& out) const { @@ -32085,6 +32212,7 @@ void NotificationEventsCountRequest::printTo(std::ostream& out) const { out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "")); out << ", " << "toEventId="; (__isset.toEventId ? (out << to_string(toEventId)) : (out << "")); out << ", " << "limit="; (__isset.limit ? (out << to_string(limit)) : (out << "")); + out << ", " << "tableNames="; (__isset.tableNames ? (out << to_string(tableNames)) : (out << "")); out << ")"; } @@ -32166,11 +32294,11 @@ void swap(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b swap(a.eventsCount, b.eventsCount); } -NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other1128) noexcept { - eventsCount = other1128.eventsCount; +NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other1140) noexcept { + eventsCount = other1140.eventsCount; } -NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other1129) noexcept { - eventsCount = other1129.eventsCount; +NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other1141) noexcept { + eventsCount = other1141.eventsCount; return *this; } void NotificationEventsCountResponse::printTo(std::ostream& out) const { @@ -32249,14 +32377,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->filesAdded.clear(); - uint32_t _size1130; - ::apache::thrift::protocol::TType _etype1133; - xfer += iprot->readListBegin(_etype1133, _size1130); - this->filesAdded.resize(_size1130); - uint32_t _i1134; - for (_i1134 = 0; _i1134 < _size1130; ++_i1134) + uint32_t _size1142; + ::apache::thrift::protocol::TType _etype1145; + xfer += iprot->readListBegin(_etype1145, _size1142); + this->filesAdded.resize(_size1142); + uint32_t _i1146; + for (_i1146 = 0; _i1146 < _size1142; ++_i1146) { - xfer += iprot->readString(this->filesAdded[_i1134]); + xfer += iprot->readString(this->filesAdded[_i1146]); } xfer += iprot->readListEnd(); } @@ -32269,14 +32397,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->filesAddedChecksum.clear(); - uint32_t _size1135; - ::apache::thrift::protocol::TType _etype1138; - xfer += iprot->readListBegin(_etype1138, _size1135); - this->filesAddedChecksum.resize(_size1135); - uint32_t _i1139; - for (_i1139 = 0; _i1139 < _size1135; ++_i1139) + uint32_t _size1147; + ::apache::thrift::protocol::TType _etype1150; + xfer += iprot->readListBegin(_etype1150, _size1147); + this->filesAddedChecksum.resize(_size1147); + uint32_t _i1151; + for (_i1151 = 0; _i1151 < _size1147; ++_i1151) { - xfer += iprot->readString(this->filesAddedChecksum[_i1139]); + xfer += iprot->readString(this->filesAddedChecksum[_i1151]); } xfer += iprot->readListEnd(); } @@ -32289,14 +32417,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->subDirectoryList.clear(); - uint32_t _size1140; - ::apache::thrift::protocol::TType _etype1143; - xfer += iprot->readListBegin(_etype1143, _size1140); - this->subDirectoryList.resize(_size1140); - uint32_t _i1144; - for (_i1144 = 0; _i1144 < _size1140; ++_i1144) + uint32_t _size1152; + ::apache::thrift::protocol::TType _etype1155; + xfer += iprot->readListBegin(_etype1155, _size1152); + this->subDirectoryList.resize(_size1152); + uint32_t _i1156; + for (_i1156 = 0; _i1156 < _size1152; ++_i1156) { - xfer += iprot->readString(this->subDirectoryList[_i1144]); + xfer += iprot->readString(this->subDirectoryList[_i1156]); } xfer += iprot->readListEnd(); } @@ -32309,14 +32437,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitionVal.clear(); - uint32_t _size1145; - ::apache::thrift::protocol::TType _etype1148; - xfer += iprot->readListBegin(_etype1148, _size1145); - this->partitionVal.resize(_size1145); - uint32_t _i1149; - for (_i1149 = 0; _i1149 < _size1145; ++_i1149) + uint32_t _size1157; + ::apache::thrift::protocol::TType _etype1160; + xfer += iprot->readListBegin(_etype1160, _size1157); + this->partitionVal.resize(_size1157); + uint32_t _i1161; + for (_i1161 = 0; _i1161 < _size1157; ++_i1161) { - xfer += iprot->readString(this->partitionVal[_i1149]); + xfer += iprot->readString(this->partitionVal[_i1161]); } xfer += iprot->readListEnd(); } @@ -32352,10 +32480,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filesAdded.size())); - std::vector ::const_iterator _iter1150; - for (_iter1150 = this->filesAdded.begin(); _iter1150 != this->filesAdded.end(); ++_iter1150) + std::vector ::const_iterator _iter1162; + for (_iter1162 = this->filesAdded.begin(); _iter1162 != this->filesAdded.end(); ++_iter1162) { - xfer += oprot->writeString((*_iter1150)); + xfer += oprot->writeString((*_iter1162)); } xfer += oprot->writeListEnd(); } @@ -32365,10 +32493,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filesAddedChecksum.size())); - std::vector ::const_iterator _iter1151; - for (_iter1151 = this->filesAddedChecksum.begin(); _iter1151 != this->filesAddedChecksum.end(); ++_iter1151) + std::vector ::const_iterator _iter1163; + for (_iter1163 = this->filesAddedChecksum.begin(); _iter1163 != this->filesAddedChecksum.end(); ++_iter1163) { - xfer += oprot->writeString((*_iter1151)); + xfer += oprot->writeString((*_iter1163)); } xfer += oprot->writeListEnd(); } @@ -32378,10 +32506,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("subDirectoryList", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->subDirectoryList.size())); - std::vector ::const_iterator _iter1152; - for (_iter1152 = this->subDirectoryList.begin(); _iter1152 != this->subDirectoryList.end(); ++_iter1152) + std::vector ::const_iterator _iter1164; + for (_iter1164 = this->subDirectoryList.begin(); _iter1164 != this->subDirectoryList.end(); ++_iter1164) { - xfer += oprot->writeString((*_iter1152)); + xfer += oprot->writeString((*_iter1164)); } xfer += oprot->writeListEnd(); } @@ -32391,10 +32519,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("partitionVal", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionVal.size())); - std::vector ::const_iterator _iter1153; - for (_iter1153 = this->partitionVal.begin(); _iter1153 != this->partitionVal.end(); ++_iter1153) + std::vector ::const_iterator _iter1165; + for (_iter1165 = this->partitionVal.begin(); _iter1165 != this->partitionVal.end(); ++_iter1165) { - xfer += oprot->writeString((*_iter1153)); + xfer += oprot->writeString((*_iter1165)); } xfer += oprot->writeListEnd(); } @@ -32415,21 +32543,21 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) { swap(a.__isset, b.__isset); } -InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other1154) { - replace = other1154.replace; - filesAdded = other1154.filesAdded; - filesAddedChecksum = other1154.filesAddedChecksum; - subDirectoryList = other1154.subDirectoryList; - partitionVal = other1154.partitionVal; - __isset = other1154.__isset; +InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other1166) { + replace = other1166.replace; + filesAdded = other1166.filesAdded; + filesAddedChecksum = other1166.filesAddedChecksum; + subDirectoryList = other1166.subDirectoryList; + partitionVal = other1166.partitionVal; + __isset = other1166.__isset; } -InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other1155) { - replace = other1155.replace; - filesAdded = other1155.filesAdded; - filesAddedChecksum = other1155.filesAddedChecksum; - subDirectoryList = other1155.subDirectoryList; - partitionVal = other1155.partitionVal; - __isset = other1155.__isset; +InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other1167) { + replace = other1167.replace; + filesAdded = other1167.filesAdded; + filesAddedChecksum = other1167.filesAddedChecksum; + subDirectoryList = other1167.subDirectoryList; + partitionVal = other1167.partitionVal; + __isset = other1167.__isset; return *this; } void InsertEventRequestData::printTo(std::ostream& out) const { @@ -32502,14 +32630,14 @@ uint32_t FireEventRequestData::read(::apache::thrift::protocol::TProtocol* iprot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->insertDatas.clear(); - uint32_t _size1156; - ::apache::thrift::protocol::TType _etype1159; - xfer += iprot->readListBegin(_etype1159, _size1156); - this->insertDatas.resize(_size1156); - uint32_t _i1160; - for (_i1160 = 0; _i1160 < _size1156; ++_i1160) + uint32_t _size1168; + ::apache::thrift::protocol::TType _etype1171; + xfer += iprot->readListBegin(_etype1171, _size1168); + this->insertDatas.resize(_size1168); + uint32_t _i1172; + for (_i1172 = 0; _i1172 < _size1168; ++_i1172) { - xfer += this->insertDatas[_i1160].read(iprot); + xfer += this->insertDatas[_i1172].read(iprot); } xfer += iprot->readListEnd(); } @@ -32552,10 +32680,10 @@ uint32_t FireEventRequestData::write(::apache::thrift::protocol::TProtocol* opro xfer += oprot->writeFieldBegin("insertDatas", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->insertDatas.size())); - std::vector ::const_iterator _iter1161; - for (_iter1161 = this->insertDatas.begin(); _iter1161 != this->insertDatas.end(); ++_iter1161) + std::vector ::const_iterator _iter1173; + for (_iter1173 = this->insertDatas.begin(); _iter1173 != this->insertDatas.end(); ++_iter1173) { - xfer += (*_iter1161).write(oprot); + xfer += (*_iter1173).write(oprot); } xfer += oprot->writeListEnd(); } @@ -32579,17 +32707,17 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) { swap(a.__isset, b.__isset); } -FireEventRequestData::FireEventRequestData(const FireEventRequestData& other1162) { - insertData = other1162.insertData; - insertDatas = other1162.insertDatas; - refreshEvent = other1162.refreshEvent; - __isset = other1162.__isset; +FireEventRequestData::FireEventRequestData(const FireEventRequestData& other1174) { + insertData = other1174.insertData; + insertDatas = other1174.insertDatas; + refreshEvent = other1174.refreshEvent; + __isset = other1174.__isset; } -FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other1163) { - insertData = other1163.insertData; - insertDatas = other1163.insertDatas; - refreshEvent = other1163.refreshEvent; - __isset = other1163.__isset; +FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other1175) { + insertData = other1175.insertData; + insertDatas = other1175.insertDatas; + refreshEvent = other1175.refreshEvent; + __isset = other1175.__isset; return *this; } void FireEventRequestData::printTo(std::ostream& out) const { @@ -32704,14 +32832,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitionVals.clear(); - uint32_t _size1164; - ::apache::thrift::protocol::TType _etype1167; - xfer += iprot->readListBegin(_etype1167, _size1164); - this->partitionVals.resize(_size1164); - uint32_t _i1168; - for (_i1168 = 0; _i1168 < _size1164; ++_i1168) + uint32_t _size1176; + ::apache::thrift::protocol::TType _etype1179; + xfer += iprot->readListBegin(_etype1179, _size1176); + this->partitionVals.resize(_size1176); + uint32_t _i1180; + for (_i1180 = 0; _i1180 < _size1176; ++_i1180) { - xfer += iprot->readString(this->partitionVals[_i1168]); + xfer += iprot->readString(this->partitionVals[_i1180]); } xfer += iprot->readListEnd(); } @@ -32732,17 +32860,17 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_MAP) { { this->tblParams.clear(); - uint32_t _size1169; - ::apache::thrift::protocol::TType _ktype1170; - ::apache::thrift::protocol::TType _vtype1171; - xfer += iprot->readMapBegin(_ktype1170, _vtype1171, _size1169); - uint32_t _i1173; - for (_i1173 = 0; _i1173 < _size1169; ++_i1173) + uint32_t _size1181; + ::apache::thrift::protocol::TType _ktype1182; + ::apache::thrift::protocol::TType _vtype1183; + xfer += iprot->readMapBegin(_ktype1182, _vtype1183, _size1181); + uint32_t _i1185; + for (_i1185 = 0; _i1185 < _size1181; ++_i1185) { - std::string _key1174; - xfer += iprot->readString(_key1174); - std::string& _val1175 = this->tblParams[_key1174]; - xfer += iprot->readString(_val1175); + std::string _key1186; + xfer += iprot->readString(_key1186); + std::string& _val1187 = this->tblParams[_key1186]; + xfer += iprot->readString(_val1187); } xfer += iprot->readMapEnd(); } @@ -32794,10 +32922,10 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionVals.size())); - std::vector ::const_iterator _iter1176; - for (_iter1176 = this->partitionVals.begin(); _iter1176 != this->partitionVals.end(); ++_iter1176) + std::vector ::const_iterator _iter1188; + for (_iter1188 = this->partitionVals.begin(); _iter1188 != this->partitionVals.end(); ++_iter1188) { - xfer += oprot->writeString((*_iter1176)); + xfer += oprot->writeString((*_iter1188)); } xfer += oprot->writeListEnd(); } @@ -32812,11 +32940,11 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c xfer += oprot->writeFieldBegin("tblParams", ::apache::thrift::protocol::T_MAP, 7); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->tblParams.size())); - std::map ::const_iterator _iter1177; - for (_iter1177 = this->tblParams.begin(); _iter1177 != this->tblParams.end(); ++_iter1177) + std::map ::const_iterator _iter1189; + for (_iter1189 = this->tblParams.begin(); _iter1189 != this->tblParams.end(); ++_iter1189) { - xfer += oprot->writeString(_iter1177->first); - xfer += oprot->writeString(_iter1177->second); + xfer += oprot->writeString(_iter1189->first); + xfer += oprot->writeString(_iter1189->second); } xfer += oprot->writeMapEnd(); } @@ -32839,25 +32967,25 @@ void swap(FireEventRequest &a, FireEventRequest &b) { swap(a.__isset, b.__isset); } -FireEventRequest::FireEventRequest(const FireEventRequest& other1178) { - successful = other1178.successful; - data = other1178.data; - dbName = other1178.dbName; - tableName = other1178.tableName; - partitionVals = other1178.partitionVals; - catName = other1178.catName; - tblParams = other1178.tblParams; - __isset = other1178.__isset; +FireEventRequest::FireEventRequest(const FireEventRequest& other1190) { + successful = other1190.successful; + data = other1190.data; + dbName = other1190.dbName; + tableName = other1190.tableName; + partitionVals = other1190.partitionVals; + catName = other1190.catName; + tblParams = other1190.tblParams; + __isset = other1190.__isset; } -FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other1179) { - successful = other1179.successful; - data = other1179.data; - dbName = other1179.dbName; - tableName = other1179.tableName; - partitionVals = other1179.partitionVals; - catName = other1179.catName; - tblParams = other1179.tblParams; - __isset = other1179.__isset; +FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other1191) { + successful = other1191.successful; + data = other1191.data; + dbName = other1191.dbName; + tableName = other1191.tableName; + partitionVals = other1191.partitionVals; + catName = other1191.catName; + tblParams = other1191.tblParams; + __isset = other1191.__isset; return *this; } void FireEventRequest::printTo(std::ostream& out) const { @@ -32913,14 +33041,14 @@ uint32_t FireEventResponse::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->eventIds.clear(); - uint32_t _size1180; - ::apache::thrift::protocol::TType _etype1183; - xfer += iprot->readListBegin(_etype1183, _size1180); - this->eventIds.resize(_size1180); - uint32_t _i1184; - for (_i1184 = 0; _i1184 < _size1180; ++_i1184) + uint32_t _size1192; + ::apache::thrift::protocol::TType _etype1195; + xfer += iprot->readListBegin(_etype1195, _size1192); + this->eventIds.resize(_size1192); + uint32_t _i1196; + for (_i1196 = 0; _i1196 < _size1192; ++_i1196) { - xfer += iprot->readI64(this->eventIds[_i1184]); + xfer += iprot->readI64(this->eventIds[_i1196]); } xfer += iprot->readListEnd(); } @@ -32949,10 +33077,10 @@ uint32_t FireEventResponse::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("eventIds", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->eventIds.size())); - std::vector ::const_iterator _iter1185; - for (_iter1185 = this->eventIds.begin(); _iter1185 != this->eventIds.end(); ++_iter1185) + std::vector ::const_iterator _iter1197; + for (_iter1197 = this->eventIds.begin(); _iter1197 != this->eventIds.end(); ++_iter1197) { - xfer += oprot->writeI64((*_iter1185)); + xfer += oprot->writeI64((*_iter1197)); } xfer += oprot->writeListEnd(); } @@ -32969,13 +33097,13 @@ void swap(FireEventResponse &a, FireEventResponse &b) { swap(a.__isset, b.__isset); } -FireEventResponse::FireEventResponse(const FireEventResponse& other1186) { - eventIds = other1186.eventIds; - __isset = other1186.__isset; +FireEventResponse::FireEventResponse(const FireEventResponse& other1198) { + eventIds = other1198.eventIds; + __isset = other1198.__isset; } -FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other1187) { - eventIds = other1187.eventIds; - __isset = other1187.__isset; +FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other1199) { + eventIds = other1199.eventIds; + __isset = other1199.__isset; return *this; } void FireEventResponse::printTo(std::ostream& out) const { @@ -33091,14 +33219,14 @@ uint32_t WriteNotificationLogRequest::read(::apache::thrift::protocol::TProtocol if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitionVals.clear(); - uint32_t _size1188; - ::apache::thrift::protocol::TType _etype1191; - xfer += iprot->readListBegin(_etype1191, _size1188); - this->partitionVals.resize(_size1188); - uint32_t _i1192; - for (_i1192 = 0; _i1192 < _size1188; ++_i1192) + uint32_t _size1200; + ::apache::thrift::protocol::TType _etype1203; + xfer += iprot->readListBegin(_etype1203, _size1200); + this->partitionVals.resize(_size1200); + uint32_t _i1204; + for (_i1204 = 0; _i1204 < _size1200; ++_i1204) { - xfer += iprot->readString(this->partitionVals[_i1192]); + xfer += iprot->readString(this->partitionVals[_i1204]); } xfer += iprot->readListEnd(); } @@ -33158,10 +33286,10 @@ uint32_t WriteNotificationLogRequest::write(::apache::thrift::protocol::TProtoco xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionVals.size())); - std::vector ::const_iterator _iter1193; - for (_iter1193 = this->partitionVals.begin(); _iter1193 != this->partitionVals.end(); ++_iter1193) + std::vector ::const_iterator _iter1205; + for (_iter1205 = this->partitionVals.begin(); _iter1205 != this->partitionVals.end(); ++_iter1205) { - xfer += oprot->writeString((*_iter1193)); + xfer += oprot->writeString((*_iter1205)); } xfer += oprot->writeListEnd(); } @@ -33183,23 +33311,23 @@ void swap(WriteNotificationLogRequest &a, WriteNotificationLogRequest &b) { swap(a.__isset, b.__isset); } -WriteNotificationLogRequest::WriteNotificationLogRequest(const WriteNotificationLogRequest& other1194) { - txnId = other1194.txnId; - writeId = other1194.writeId; - db = other1194.db; - table = other1194.table; - fileInfo = other1194.fileInfo; - partitionVals = other1194.partitionVals; - __isset = other1194.__isset; +WriteNotificationLogRequest::WriteNotificationLogRequest(const WriteNotificationLogRequest& other1206) { + txnId = other1206.txnId; + writeId = other1206.writeId; + db = other1206.db; + table = other1206.table; + fileInfo = other1206.fileInfo; + partitionVals = other1206.partitionVals; + __isset = other1206.__isset; } -WriteNotificationLogRequest& WriteNotificationLogRequest::operator=(const WriteNotificationLogRequest& other1195) { - txnId = other1195.txnId; - writeId = other1195.writeId; - db = other1195.db; - table = other1195.table; - fileInfo = other1195.fileInfo; - partitionVals = other1195.partitionVals; - __isset = other1195.__isset; +WriteNotificationLogRequest& WriteNotificationLogRequest::operator=(const WriteNotificationLogRequest& other1207) { + txnId = other1207.txnId; + writeId = other1207.writeId; + db = other1207.db; + table = other1207.table; + fileInfo = other1207.fileInfo; + partitionVals = other1207.partitionVals; + __isset = other1207.__isset; return *this; } void WriteNotificationLogRequest::printTo(std::ostream& out) const { @@ -33269,11 +33397,11 @@ void swap(WriteNotificationLogResponse &a, WriteNotificationLogResponse &b) { (void) b; } -WriteNotificationLogResponse::WriteNotificationLogResponse(const WriteNotificationLogResponse& other1196) noexcept { - (void) other1196; +WriteNotificationLogResponse::WriteNotificationLogResponse(const WriteNotificationLogResponse& other1208) noexcept { + (void) other1208; } -WriteNotificationLogResponse& WriteNotificationLogResponse::operator=(const WriteNotificationLogResponse& other1197) noexcept { - (void) other1197; +WriteNotificationLogResponse& WriteNotificationLogResponse::operator=(const WriteNotificationLogResponse& other1209) noexcept { + (void) other1209; return *this; } void WriteNotificationLogResponse::printTo(std::ostream& out) const { @@ -33362,14 +33490,14 @@ uint32_t WriteNotificationLogBatchRequest::read(::apache::thrift::protocol::TPro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->requestList.clear(); - uint32_t _size1198; - ::apache::thrift::protocol::TType _etype1201; - xfer += iprot->readListBegin(_etype1201, _size1198); - this->requestList.resize(_size1198); - uint32_t _i1202; - for (_i1202 = 0; _i1202 < _size1198; ++_i1202) + uint32_t _size1210; + ::apache::thrift::protocol::TType _etype1213; + xfer += iprot->readListBegin(_etype1213, _size1210); + this->requestList.resize(_size1210); + uint32_t _i1214; + for (_i1214 = 0; _i1214 < _size1210; ++_i1214) { - xfer += this->requestList[_i1202].read(iprot); + xfer += this->requestList[_i1214].read(iprot); } xfer += iprot->readListEnd(); } @@ -33418,10 +33546,10 @@ uint32_t WriteNotificationLogBatchRequest::write(::apache::thrift::protocol::TPr xfer += oprot->writeFieldBegin("requestList", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->requestList.size())); - std::vector ::const_iterator _iter1203; - for (_iter1203 = this->requestList.begin(); _iter1203 != this->requestList.end(); ++_iter1203) + std::vector ::const_iterator _iter1215; + for (_iter1215 = this->requestList.begin(); _iter1215 != this->requestList.end(); ++_iter1215) { - xfer += (*_iter1203).write(oprot); + xfer += (*_iter1215).write(oprot); } xfer += oprot->writeListEnd(); } @@ -33440,17 +33568,17 @@ void swap(WriteNotificationLogBatchRequest &a, WriteNotificationLogBatchRequest swap(a.requestList, b.requestList); } -WriteNotificationLogBatchRequest::WriteNotificationLogBatchRequest(const WriteNotificationLogBatchRequest& other1204) { - catalog = other1204.catalog; - db = other1204.db; - table = other1204.table; - requestList = other1204.requestList; +WriteNotificationLogBatchRequest::WriteNotificationLogBatchRequest(const WriteNotificationLogBatchRequest& other1216) { + catalog = other1216.catalog; + db = other1216.db; + table = other1216.table; + requestList = other1216.requestList; } -WriteNotificationLogBatchRequest& WriteNotificationLogBatchRequest::operator=(const WriteNotificationLogBatchRequest& other1205) { - catalog = other1205.catalog; - db = other1205.db; - table = other1205.table; - requestList = other1205.requestList; +WriteNotificationLogBatchRequest& WriteNotificationLogBatchRequest::operator=(const WriteNotificationLogBatchRequest& other1217) { + catalog = other1217.catalog; + db = other1217.db; + table = other1217.table; + requestList = other1217.requestList; return *this; } void WriteNotificationLogBatchRequest::printTo(std::ostream& out) const { @@ -33518,11 +33646,11 @@ void swap(WriteNotificationLogBatchResponse &a, WriteNotificationLogBatchRespons (void) b; } -WriteNotificationLogBatchResponse::WriteNotificationLogBatchResponse(const WriteNotificationLogBatchResponse& other1206) noexcept { - (void) other1206; +WriteNotificationLogBatchResponse::WriteNotificationLogBatchResponse(const WriteNotificationLogBatchResponse& other1218) noexcept { + (void) other1218; } -WriteNotificationLogBatchResponse& WriteNotificationLogBatchResponse::operator=(const WriteNotificationLogBatchResponse& other1207) noexcept { - (void) other1207; +WriteNotificationLogBatchResponse& WriteNotificationLogBatchResponse::operator=(const WriteNotificationLogBatchResponse& other1219) noexcept { + (void) other1219; return *this; } void WriteNotificationLogBatchResponse::printTo(std::ostream& out) const { @@ -33628,15 +33756,15 @@ void swap(MetadataPpdResult &a, MetadataPpdResult &b) { swap(a.__isset, b.__isset); } -MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other1208) { - metadata = other1208.metadata; - includeBitset = other1208.includeBitset; - __isset = other1208.__isset; +MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other1220) { + metadata = other1220.metadata; + includeBitset = other1220.includeBitset; + __isset = other1220.__isset; } -MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other1209) { - metadata = other1209.metadata; - includeBitset = other1209.includeBitset; - __isset = other1209.__isset; +MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other1221) { + metadata = other1221.metadata; + includeBitset = other1221.includeBitset; + __isset = other1221.__isset; return *this; } void MetadataPpdResult::printTo(std::ostream& out) const { @@ -33693,17 +33821,17 @@ uint32_t GetFileMetadataByExprResult::read(::apache::thrift::protocol::TProtocol if (ftype == ::apache::thrift::protocol::T_MAP) { { this->metadata.clear(); - uint32_t _size1210; - ::apache::thrift::protocol::TType _ktype1211; - ::apache::thrift::protocol::TType _vtype1212; - xfer += iprot->readMapBegin(_ktype1211, _vtype1212, _size1210); - uint32_t _i1214; - for (_i1214 = 0; _i1214 < _size1210; ++_i1214) + uint32_t _size1222; + ::apache::thrift::protocol::TType _ktype1223; + ::apache::thrift::protocol::TType _vtype1224; + xfer += iprot->readMapBegin(_ktype1223, _vtype1224, _size1222); + uint32_t _i1226; + for (_i1226 = 0; _i1226 < _size1222; ++_i1226) { - int64_t _key1215; - xfer += iprot->readI64(_key1215); - MetadataPpdResult& _val1216 = this->metadata[_key1215]; - xfer += _val1216.read(iprot); + int64_t _key1227; + xfer += iprot->readI64(_key1227); + MetadataPpdResult& _val1228 = this->metadata[_key1227]; + xfer += _val1228.read(iprot); } xfer += iprot->readMapEnd(); } @@ -33744,11 +33872,11 @@ uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtoco xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast(this->metadata.size())); - std::map ::const_iterator _iter1217; - for (_iter1217 = this->metadata.begin(); _iter1217 != this->metadata.end(); ++_iter1217) + std::map ::const_iterator _iter1229; + for (_iter1229 = this->metadata.begin(); _iter1229 != this->metadata.end(); ++_iter1229) { - xfer += oprot->writeI64(_iter1217->first); - xfer += _iter1217->second.write(oprot); + xfer += oprot->writeI64(_iter1229->first); + xfer += _iter1229->second.write(oprot); } xfer += oprot->writeMapEnd(); } @@ -33769,13 +33897,13 @@ void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) { swap(a.isSupported, b.isSupported); } -GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other1218) { - metadata = other1218.metadata; - isSupported = other1218.isSupported; +GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other1230) { + metadata = other1230.metadata; + isSupported = other1230.isSupported; } -GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other1219) { - metadata = other1219.metadata; - isSupported = other1219.isSupported; +GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other1231) { + metadata = other1231.metadata; + isSupported = other1231.isSupported; return *this; } void GetFileMetadataByExprResult::printTo(std::ostream& out) const { @@ -33842,14 +33970,14 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fileIds.clear(); - uint32_t _size1220; - ::apache::thrift::protocol::TType _etype1223; - xfer += iprot->readListBegin(_etype1223, _size1220); - this->fileIds.resize(_size1220); - uint32_t _i1224; - for (_i1224 = 0; _i1224 < _size1220; ++_i1224) + uint32_t _size1232; + ::apache::thrift::protocol::TType _etype1235; + xfer += iprot->readListBegin(_etype1235, _size1232); + this->fileIds.resize(_size1232); + uint32_t _i1236; + for (_i1236 = 0; _i1236 < _size1232; ++_i1236) { - xfer += iprot->readI64(this->fileIds[_i1224]); + xfer += iprot->readI64(this->fileIds[_i1236]); } xfer += iprot->readListEnd(); } @@ -33876,9 +34004,9 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1225; - xfer += iprot->readI32(ecast1225); - this->type = static_cast(ecast1225); + int32_t ecast1237; + xfer += iprot->readI32(ecast1237); + this->type = static_cast(ecast1237); this->__isset.type = true; } else { xfer += iprot->skip(ftype); @@ -33908,10 +34036,10 @@ uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtoc xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter1226; - for (_iter1226 = this->fileIds.begin(); _iter1226 != this->fileIds.end(); ++_iter1226) + std::vector ::const_iterator _iter1238; + for (_iter1238 = this->fileIds.begin(); _iter1238 != this->fileIds.end(); ++_iter1238) { - xfer += oprot->writeI64((*_iter1226)); + xfer += oprot->writeI64((*_iter1238)); } xfer += oprot->writeListEnd(); } @@ -33945,19 +34073,19 @@ void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) { swap(a.__isset, b.__isset); } -GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other1227) { - fileIds = other1227.fileIds; - expr = other1227.expr; - doGetFooters = other1227.doGetFooters; - type = other1227.type; - __isset = other1227.__isset; +GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other1239) { + fileIds = other1239.fileIds; + expr = other1239.expr; + doGetFooters = other1239.doGetFooters; + type = other1239.type; + __isset = other1239.__isset; } -GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other1228) { - fileIds = other1228.fileIds; - expr = other1228.expr; - doGetFooters = other1228.doGetFooters; - type = other1228.type; - __isset = other1228.__isset; +GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other1240) { + fileIds = other1240.fileIds; + expr = other1240.expr; + doGetFooters = other1240.doGetFooters; + type = other1240.type; + __isset = other1240.__isset; return *this; } void GetFileMetadataByExprRequest::printTo(std::ostream& out) const { @@ -34016,17 +34144,17 @@ uint32_t GetFileMetadataResult::read(::apache::thrift::protocol::TProtocol* ipro if (ftype == ::apache::thrift::protocol::T_MAP) { { this->metadata.clear(); - uint32_t _size1229; - ::apache::thrift::protocol::TType _ktype1230; - ::apache::thrift::protocol::TType _vtype1231; - xfer += iprot->readMapBegin(_ktype1230, _vtype1231, _size1229); - uint32_t _i1233; - for (_i1233 = 0; _i1233 < _size1229; ++_i1233) + uint32_t _size1241; + ::apache::thrift::protocol::TType _ktype1242; + ::apache::thrift::protocol::TType _vtype1243; + xfer += iprot->readMapBegin(_ktype1242, _vtype1243, _size1241); + uint32_t _i1245; + for (_i1245 = 0; _i1245 < _size1241; ++_i1245) { - int64_t _key1234; - xfer += iprot->readI64(_key1234); - std::string& _val1235 = this->metadata[_key1234]; - xfer += iprot->readBinary(_val1235); + int64_t _key1246; + xfer += iprot->readI64(_key1246); + std::string& _val1247 = this->metadata[_key1246]; + xfer += iprot->readBinary(_val1247); } xfer += iprot->readMapEnd(); } @@ -34067,11 +34195,11 @@ uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* opr xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast(this->metadata.size())); - std::map ::const_iterator _iter1236; - for (_iter1236 = this->metadata.begin(); _iter1236 != this->metadata.end(); ++_iter1236) + std::map ::const_iterator _iter1248; + for (_iter1248 = this->metadata.begin(); _iter1248 != this->metadata.end(); ++_iter1248) { - xfer += oprot->writeI64(_iter1236->first); - xfer += oprot->writeBinary(_iter1236->second); + xfer += oprot->writeI64(_iter1248->first); + xfer += oprot->writeBinary(_iter1248->second); } xfer += oprot->writeMapEnd(); } @@ -34092,13 +34220,13 @@ void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) { swap(a.isSupported, b.isSupported); } -GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other1237) { - metadata = other1237.metadata; - isSupported = other1237.isSupported; +GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other1249) { + metadata = other1249.metadata; + isSupported = other1249.isSupported; } -GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other1238) { - metadata = other1238.metadata; - isSupported = other1238.isSupported; +GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other1250) { + metadata = other1250.metadata; + isSupported = other1250.isSupported; return *this; } void GetFileMetadataResult::printTo(std::ostream& out) const { @@ -34150,14 +34278,14 @@ uint32_t GetFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fileIds.clear(); - uint32_t _size1239; - ::apache::thrift::protocol::TType _etype1242; - xfer += iprot->readListBegin(_etype1242, _size1239); - this->fileIds.resize(_size1239); - uint32_t _i1243; - for (_i1243 = 0; _i1243 < _size1239; ++_i1243) + uint32_t _size1251; + ::apache::thrift::protocol::TType _etype1254; + xfer += iprot->readListBegin(_etype1254, _size1251); + this->fileIds.resize(_size1251); + uint32_t _i1255; + for (_i1255 = 0; _i1255 < _size1251; ++_i1255) { - xfer += iprot->readI64(this->fileIds[_i1243]); + xfer += iprot->readI64(this->fileIds[_i1255]); } xfer += iprot->readListEnd(); } @@ -34188,10 +34316,10 @@ uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter1244; - for (_iter1244 = this->fileIds.begin(); _iter1244 != this->fileIds.end(); ++_iter1244) + std::vector ::const_iterator _iter1256; + for (_iter1256 = this->fileIds.begin(); _iter1256 != this->fileIds.end(); ++_iter1256) { - xfer += oprot->writeI64((*_iter1244)); + xfer += oprot->writeI64((*_iter1256)); } xfer += oprot->writeListEnd(); } @@ -34207,11 +34335,11 @@ void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) { swap(a.fileIds, b.fileIds); } -GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other1245) { - fileIds = other1245.fileIds; +GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other1257) { + fileIds = other1257.fileIds; } -GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other1246) { - fileIds = other1246.fileIds; +GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other1258) { + fileIds = other1258.fileIds; return *this; } void GetFileMetadataRequest::printTo(std::ostream& out) const { @@ -34276,11 +34404,11 @@ void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) { (void) b; } -PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other1247) noexcept { - (void) other1247; +PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other1259) noexcept { + (void) other1259; } -PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other1248) noexcept { - (void) other1248; +PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other1260) noexcept { + (void) other1260; return *this; } void PutFileMetadataResult::printTo(std::ostream& out) const { @@ -34340,14 +34468,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fileIds.clear(); - uint32_t _size1249; - ::apache::thrift::protocol::TType _etype1252; - xfer += iprot->readListBegin(_etype1252, _size1249); - this->fileIds.resize(_size1249); - uint32_t _i1253; - for (_i1253 = 0; _i1253 < _size1249; ++_i1253) + uint32_t _size1261; + ::apache::thrift::protocol::TType _etype1264; + xfer += iprot->readListBegin(_etype1264, _size1261); + this->fileIds.resize(_size1261); + uint32_t _i1265; + for (_i1265 = 0; _i1265 < _size1261; ++_i1265) { - xfer += iprot->readI64(this->fileIds[_i1253]); + xfer += iprot->readI64(this->fileIds[_i1265]); } xfer += iprot->readListEnd(); } @@ -34360,14 +34488,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->metadata.clear(); - uint32_t _size1254; - ::apache::thrift::protocol::TType _etype1257; - xfer += iprot->readListBegin(_etype1257, _size1254); - this->metadata.resize(_size1254); - uint32_t _i1258; - for (_i1258 = 0; _i1258 < _size1254; ++_i1258) + uint32_t _size1266; + ::apache::thrift::protocol::TType _etype1269; + xfer += iprot->readListBegin(_etype1269, _size1266); + this->metadata.resize(_size1266); + uint32_t _i1270; + for (_i1270 = 0; _i1270 < _size1266; ++_i1270) { - xfer += iprot->readBinary(this->metadata[_i1258]); + xfer += iprot->readBinary(this->metadata[_i1270]); } xfer += iprot->readListEnd(); } @@ -34378,9 +34506,9 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1259; - xfer += iprot->readI32(ecast1259); - this->type = static_cast(ecast1259); + int32_t ecast1271; + xfer += iprot->readI32(ecast1271); + this->type = static_cast(ecast1271); this->__isset.type = true; } else { xfer += iprot->skip(ftype); @@ -34410,10 +34538,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter1260; - for (_iter1260 = this->fileIds.begin(); _iter1260 != this->fileIds.end(); ++_iter1260) + std::vector ::const_iterator _iter1272; + for (_iter1272 = this->fileIds.begin(); _iter1272 != this->fileIds.end(); ++_iter1272) { - xfer += oprot->writeI64((*_iter1260)); + xfer += oprot->writeI64((*_iter1272)); } xfer += oprot->writeListEnd(); } @@ -34422,10 +34550,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->metadata.size())); - std::vector ::const_iterator _iter1261; - for (_iter1261 = this->metadata.begin(); _iter1261 != this->metadata.end(); ++_iter1261) + std::vector ::const_iterator _iter1273; + for (_iter1273 = this->metadata.begin(); _iter1273 != this->metadata.end(); ++_iter1273) { - xfer += oprot->writeBinary((*_iter1261)); + xfer += oprot->writeBinary((*_iter1273)); } xfer += oprot->writeListEnd(); } @@ -34449,17 +34577,17 @@ void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) { swap(a.__isset, b.__isset); } -PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other1262) { - fileIds = other1262.fileIds; - metadata = other1262.metadata; - type = other1262.type; - __isset = other1262.__isset; +PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other1274) { + fileIds = other1274.fileIds; + metadata = other1274.metadata; + type = other1274.type; + __isset = other1274.__isset; } -PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other1263) { - fileIds = other1263.fileIds; - metadata = other1263.metadata; - type = other1263.type; - __isset = other1263.__isset; +PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other1275) { + fileIds = other1275.fileIds; + metadata = other1275.metadata; + type = other1275.type; + __isset = other1275.__isset; return *this; } void PutFileMetadataRequest::printTo(std::ostream& out) const { @@ -34526,11 +34654,11 @@ void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) { (void) b; } -ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other1264) noexcept { - (void) other1264; +ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other1276) noexcept { + (void) other1276; } -ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other1265) noexcept { - (void) other1265; +ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other1277) noexcept { + (void) other1277; return *this; } void ClearFileMetadataResult::printTo(std::ostream& out) const { @@ -34580,14 +34708,14 @@ uint32_t ClearFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* i if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fileIds.clear(); - uint32_t _size1266; - ::apache::thrift::protocol::TType _etype1269; - xfer += iprot->readListBegin(_etype1269, _size1266); - this->fileIds.resize(_size1266); - uint32_t _i1270; - for (_i1270 = 0; _i1270 < _size1266; ++_i1270) + uint32_t _size1278; + ::apache::thrift::protocol::TType _etype1281; + xfer += iprot->readListBegin(_etype1281, _size1278); + this->fileIds.resize(_size1278); + uint32_t _i1282; + for (_i1282 = 0; _i1282 < _size1278; ++_i1282) { - xfer += iprot->readI64(this->fileIds[_i1270]); + xfer += iprot->readI64(this->fileIds[_i1282]); } xfer += iprot->readListEnd(); } @@ -34618,10 +34746,10 @@ uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter1271; - for (_iter1271 = this->fileIds.begin(); _iter1271 != this->fileIds.end(); ++_iter1271) + std::vector ::const_iterator _iter1283; + for (_iter1283 = this->fileIds.begin(); _iter1283 != this->fileIds.end(); ++_iter1283) { - xfer += oprot->writeI64((*_iter1271)); + xfer += oprot->writeI64((*_iter1283)); } xfer += oprot->writeListEnd(); } @@ -34637,11 +34765,11 @@ void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) { swap(a.fileIds, b.fileIds); } -ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other1272) { - fileIds = other1272.fileIds; +ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other1284) { + fileIds = other1284.fileIds; } -ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other1273) { - fileIds = other1273.fileIds; +ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other1285) { + fileIds = other1285.fileIds; return *this; } void ClearFileMetadataRequest::printTo(std::ostream& out) const { @@ -34729,11 +34857,11 @@ void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) { swap(a.isSupported, b.isSupported); } -CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other1274) noexcept { - isSupported = other1274.isSupported; +CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other1286) noexcept { + isSupported = other1286.isSupported; } -CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other1275) noexcept { - isSupported = other1275.isSupported; +CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other1287) noexcept { + isSupported = other1287.isSupported; return *this; } void CacheFileMetadataResult::printTo(std::ostream& out) const { @@ -34880,19 +35008,19 @@ void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) { swap(a.__isset, b.__isset); } -CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other1276) { - dbName = other1276.dbName; - tblName = other1276.tblName; - partName = other1276.partName; - isAllParts = other1276.isAllParts; - __isset = other1276.__isset; +CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other1288) { + dbName = other1288.dbName; + tblName = other1288.tblName; + partName = other1288.partName; + isAllParts = other1288.isAllParts; + __isset = other1288.__isset; } -CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other1277) { - dbName = other1277.dbName; - tblName = other1277.tblName; - partName = other1277.partName; - isAllParts = other1277.isAllParts; - __isset = other1277.__isset; +CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other1289) { + dbName = other1289.dbName; + tblName = other1289.tblName; + partName = other1289.partName; + isAllParts = other1289.isAllParts; + __isset = other1289.__isset; return *this; } void CacheFileMetadataRequest::printTo(std::ostream& out) const { @@ -34946,14 +35074,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip if (ftype == ::apache::thrift::protocol::T_LIST) { { this->functions.clear(); - uint32_t _size1278; - ::apache::thrift::protocol::TType _etype1281; - xfer += iprot->readListBegin(_etype1281, _size1278); - this->functions.resize(_size1278); - uint32_t _i1282; - for (_i1282 = 0; _i1282 < _size1278; ++_i1282) + uint32_t _size1290; + ::apache::thrift::protocol::TType _etype1293; + xfer += iprot->readListBegin(_etype1293, _size1290); + this->functions.resize(_size1290); + uint32_t _i1294; + for (_i1294 = 0; _i1294 < _size1290; ++_i1294) { - xfer += this->functions[_i1282].read(iprot); + xfer += this->functions[_i1294].read(iprot); } xfer += iprot->readListEnd(); } @@ -34983,10 +35111,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->functions.size())); - std::vector ::const_iterator _iter1283; - for (_iter1283 = this->functions.begin(); _iter1283 != this->functions.end(); ++_iter1283) + std::vector ::const_iterator _iter1295; + for (_iter1295 = this->functions.begin(); _iter1295 != this->functions.end(); ++_iter1295) { - xfer += (*_iter1283).write(oprot); + xfer += (*_iter1295).write(oprot); } xfer += oprot->writeListEnd(); } @@ -35003,13 +35131,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) { swap(a.__isset, b.__isset); } -GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other1284) { - functions = other1284.functions; - __isset = other1284.__isset; +GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other1296) { + functions = other1296.functions; + __isset = other1296.__isset; } -GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other1285) { - functions = other1285.functions; - __isset = other1285.__isset; +GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other1297) { + functions = other1297.functions; + __isset = other1297.__isset; return *this; } void GetAllFunctionsResponse::printTo(std::ostream& out) const { @@ -35060,16 +35188,16 @@ uint32_t ClientCapabilities::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->values.clear(); - uint32_t _size1286; - ::apache::thrift::protocol::TType _etype1289; - xfer += iprot->readListBegin(_etype1289, _size1286); - this->values.resize(_size1286); - uint32_t _i1290; - for (_i1290 = 0; _i1290 < _size1286; ++_i1290) + uint32_t _size1298; + ::apache::thrift::protocol::TType _etype1301; + xfer += iprot->readListBegin(_etype1301, _size1298); + this->values.resize(_size1298); + uint32_t _i1302; + for (_i1302 = 0; _i1302 < _size1298; ++_i1302) { - int32_t ecast1291; - xfer += iprot->readI32(ecast1291); - this->values[_i1290] = static_cast(ecast1291); + int32_t ecast1303; + xfer += iprot->readI32(ecast1303); + this->values[_i1302] = static_cast(ecast1303); } xfer += iprot->readListEnd(); } @@ -35100,10 +35228,10 @@ uint32_t ClientCapabilities::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->values.size())); - std::vector ::const_iterator _iter1292; - for (_iter1292 = this->values.begin(); _iter1292 != this->values.end(); ++_iter1292) + std::vector ::const_iterator _iter1304; + for (_iter1304 = this->values.begin(); _iter1304 != this->values.end(); ++_iter1304) { - xfer += oprot->writeI32(static_cast((*_iter1292))); + xfer += oprot->writeI32(static_cast((*_iter1304))); } xfer += oprot->writeListEnd(); } @@ -35119,11 +35247,11 @@ void swap(ClientCapabilities &a, ClientCapabilities &b) { swap(a.values, b.values); } -ClientCapabilities::ClientCapabilities(const ClientCapabilities& other1293) { - values = other1293.values; +ClientCapabilities::ClientCapabilities(const ClientCapabilities& other1305) { + values = other1305.values; } -ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other1294) { - values = other1294.values; +ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other1306) { + values = other1306.values; return *this; } void ClientCapabilities::printTo(std::ostream& out) const { @@ -35181,14 +35309,14 @@ uint32_t GetProjectionsSpec::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fieldList.clear(); - uint32_t _size1295; - ::apache::thrift::protocol::TType _etype1298; - xfer += iprot->readListBegin(_etype1298, _size1295); - this->fieldList.resize(_size1295); - uint32_t _i1299; - for (_i1299 = 0; _i1299 < _size1295; ++_i1299) + uint32_t _size1307; + ::apache::thrift::protocol::TType _etype1310; + xfer += iprot->readListBegin(_etype1310, _size1307); + this->fieldList.resize(_size1307); + uint32_t _i1311; + for (_i1311 = 0; _i1311 < _size1307; ++_i1311) { - xfer += iprot->readString(this->fieldList[_i1299]); + xfer += iprot->readString(this->fieldList[_i1311]); } xfer += iprot->readListEnd(); } @@ -35233,10 +35361,10 @@ uint32_t GetProjectionsSpec::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("fieldList", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->fieldList.size())); - std::vector ::const_iterator _iter1300; - for (_iter1300 = this->fieldList.begin(); _iter1300 != this->fieldList.end(); ++_iter1300) + std::vector ::const_iterator _iter1312; + for (_iter1312 = this->fieldList.begin(); _iter1312 != this->fieldList.end(); ++_iter1312) { - xfer += oprot->writeString((*_iter1300)); + xfer += oprot->writeString((*_iter1312)); } xfer += oprot->writeListEnd(); } @@ -35263,17 +35391,17 @@ void swap(GetProjectionsSpec &a, GetProjectionsSpec &b) { swap(a.__isset, b.__isset); } -GetProjectionsSpec::GetProjectionsSpec(const GetProjectionsSpec& other1301) { - fieldList = other1301.fieldList; - includeParamKeyPattern = other1301.includeParamKeyPattern; - excludeParamKeyPattern = other1301.excludeParamKeyPattern; - __isset = other1301.__isset; +GetProjectionsSpec::GetProjectionsSpec(const GetProjectionsSpec& other1313) { + fieldList = other1313.fieldList; + includeParamKeyPattern = other1313.includeParamKeyPattern; + excludeParamKeyPattern = other1313.excludeParamKeyPattern; + __isset = other1313.__isset; } -GetProjectionsSpec& GetProjectionsSpec::operator=(const GetProjectionsSpec& other1302) { - fieldList = other1302.fieldList; - includeParamKeyPattern = other1302.includeParamKeyPattern; - excludeParamKeyPattern = other1302.excludeParamKeyPattern; - __isset = other1302.__isset; +GetProjectionsSpec& GetProjectionsSpec::operator=(const GetProjectionsSpec& other1314) { + fieldList = other1314.fieldList; + includeParamKeyPattern = other1314.includeParamKeyPattern; + excludeParamKeyPattern = other1314.excludeParamKeyPattern; + __isset = other1314.__isset; return *this; } void GetProjectionsSpec::printTo(std::ostream& out) const { @@ -35419,14 +35547,14 @@ uint32_t GetTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1303; - ::apache::thrift::protocol::TType _etype1306; - xfer += iprot->readListBegin(_etype1306, _size1303); - this->processorCapabilities.resize(_size1303); - uint32_t _i1307; - for (_i1307 = 0; _i1307 < _size1303; ++_i1307) + uint32_t _size1315; + ::apache::thrift::protocol::TType _etype1318; + xfer += iprot->readListBegin(_etype1318, _size1315); + this->processorCapabilities.resize(_size1315); + uint32_t _i1319; + for (_i1319 = 0; _i1319 < _size1315; ++_i1319) { - xfer += iprot->readString(this->processorCapabilities[_i1307]); + xfer += iprot->readString(this->processorCapabilities[_i1319]); } xfer += iprot->readListEnd(); } @@ -35512,10 +35640,10 @@ uint32_t GetTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) co xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 8); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1308; - for (_iter1308 = this->processorCapabilities.begin(); _iter1308 != this->processorCapabilities.end(); ++_iter1308) + std::vector ::const_iterator _iter1320; + for (_iter1320 = this->processorCapabilities.begin(); _iter1320 != this->processorCapabilities.end(); ++_iter1320) { - xfer += oprot->writeString((*_iter1308)); + xfer += oprot->writeString((*_iter1320)); } xfer += oprot->writeListEnd(); } @@ -35556,31 +35684,31 @@ void swap(GetTableRequest &a, GetTableRequest &b) { swap(a.__isset, b.__isset); } -GetTableRequest::GetTableRequest(const GetTableRequest& other1309) { - dbName = other1309.dbName; - tblName = other1309.tblName; - capabilities = other1309.capabilities; - catName = other1309.catName; - validWriteIdList = other1309.validWriteIdList; - getColumnStats = other1309.getColumnStats; - processorCapabilities = other1309.processorCapabilities; - processorIdentifier = other1309.processorIdentifier; - engine = other1309.engine; - id = other1309.id; - __isset = other1309.__isset; -} -GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other1310) { - dbName = other1310.dbName; - tblName = other1310.tblName; - capabilities = other1310.capabilities; - catName = other1310.catName; - validWriteIdList = other1310.validWriteIdList; - getColumnStats = other1310.getColumnStats; - processorCapabilities = other1310.processorCapabilities; - processorIdentifier = other1310.processorIdentifier; - engine = other1310.engine; - id = other1310.id; - __isset = other1310.__isset; +GetTableRequest::GetTableRequest(const GetTableRequest& other1321) { + dbName = other1321.dbName; + tblName = other1321.tblName; + capabilities = other1321.capabilities; + catName = other1321.catName; + validWriteIdList = other1321.validWriteIdList; + getColumnStats = other1321.getColumnStats; + processorCapabilities = other1321.processorCapabilities; + processorIdentifier = other1321.processorIdentifier; + engine = other1321.engine; + id = other1321.id; + __isset = other1321.__isset; +} +GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other1322) { + dbName = other1322.dbName; + tblName = other1322.tblName; + capabilities = other1322.capabilities; + catName = other1322.catName; + validWriteIdList = other1322.validWriteIdList; + getColumnStats = other1322.getColumnStats; + processorCapabilities = other1322.processorCapabilities; + processorIdentifier = other1322.processorIdentifier; + engine = other1322.engine; + id = other1322.id; + __isset = other1322.__isset; return *this; } void GetTableRequest::printTo(std::ostream& out) const { @@ -35697,15 +35825,15 @@ void swap(GetTableResult &a, GetTableResult &b) { swap(a.__isset, b.__isset); } -GetTableResult::GetTableResult(const GetTableResult& other1311) { - table = other1311.table; - isStatsCompliant = other1311.isStatsCompliant; - __isset = other1311.__isset; +GetTableResult::GetTableResult(const GetTableResult& other1323) { + table = other1323.table; + isStatsCompliant = other1323.isStatsCompliant; + __isset = other1323.__isset; } -GetTableResult& GetTableResult::operator=(const GetTableResult& other1312) { - table = other1312.table; - isStatsCompliant = other1312.isStatsCompliant; - __isset = other1312.__isset; +GetTableResult& GetTableResult::operator=(const GetTableResult& other1324) { + table = other1324.table; + isStatsCompliant = other1324.isStatsCompliant; + __isset = other1324.__isset; return *this; } void GetTableResult::printTo(std::ostream& out) const { @@ -35800,14 +35928,14 @@ uint32_t GetTablesRequest::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tblNames.clear(); - uint32_t _size1313; - ::apache::thrift::protocol::TType _etype1316; - xfer += iprot->readListBegin(_etype1316, _size1313); - this->tblNames.resize(_size1313); - uint32_t _i1317; - for (_i1317 = 0; _i1317 < _size1313; ++_i1317) + uint32_t _size1325; + ::apache::thrift::protocol::TType _etype1328; + xfer += iprot->readListBegin(_etype1328, _size1325); + this->tblNames.resize(_size1325); + uint32_t _i1329; + for (_i1329 = 0; _i1329 < _size1325; ++_i1329) { - xfer += iprot->readString(this->tblNames[_i1317]); + xfer += iprot->readString(this->tblNames[_i1329]); } xfer += iprot->readListEnd(); } @@ -35836,14 +35964,14 @@ uint32_t GetTablesRequest::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1318; - ::apache::thrift::protocol::TType _etype1321; - xfer += iprot->readListBegin(_etype1321, _size1318); - this->processorCapabilities.resize(_size1318); - uint32_t _i1322; - for (_i1322 = 0; _i1322 < _size1318; ++_i1322) + uint32_t _size1330; + ::apache::thrift::protocol::TType _etype1333; + xfer += iprot->readListBegin(_etype1333, _size1330); + this->processorCapabilities.resize(_size1330); + uint32_t _i1334; + for (_i1334 = 0; _i1334 < _size1330; ++_i1334) { - xfer += iprot->readString(this->processorCapabilities[_i1322]); + xfer += iprot->readString(this->processorCapabilities[_i1334]); } xfer += iprot->readListEnd(); } @@ -35903,10 +36031,10 @@ uint32_t GetTablesRequest::write(::apache::thrift::protocol::TProtocol* oprot) c xfer += oprot->writeFieldBegin("tblNames", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tblNames.size())); - std::vector ::const_iterator _iter1323; - for (_iter1323 = this->tblNames.begin(); _iter1323 != this->tblNames.end(); ++_iter1323) + std::vector ::const_iterator _iter1335; + for (_iter1335 = this->tblNames.begin(); _iter1335 != this->tblNames.end(); ++_iter1335) { - xfer += oprot->writeString((*_iter1323)); + xfer += oprot->writeString((*_iter1335)); } xfer += oprot->writeListEnd(); } @@ -35926,10 +36054,10 @@ uint32_t GetTablesRequest::write(::apache::thrift::protocol::TProtocol* oprot) c xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1324; - for (_iter1324 = this->processorCapabilities.begin(); _iter1324 != this->processorCapabilities.end(); ++_iter1324) + std::vector ::const_iterator _iter1336; + for (_iter1336 = this->processorCapabilities.begin(); _iter1336 != this->processorCapabilities.end(); ++_iter1336) { - xfer += oprot->writeString((*_iter1324)); + xfer += oprot->writeString((*_iter1336)); } xfer += oprot->writeListEnd(); } @@ -35968,27 +36096,27 @@ void swap(GetTablesRequest &a, GetTablesRequest &b) { swap(a.__isset, b.__isset); } -GetTablesRequest::GetTablesRequest(const GetTablesRequest& other1325) { - dbName = other1325.dbName; - tblNames = other1325.tblNames; - capabilities = other1325.capabilities; - catName = other1325.catName; - processorCapabilities = other1325.processorCapabilities; - processorIdentifier = other1325.processorIdentifier; - projectionSpec = other1325.projectionSpec; - tablesPattern = other1325.tablesPattern; - __isset = other1325.__isset; -} -GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other1326) { - dbName = other1326.dbName; - tblNames = other1326.tblNames; - capabilities = other1326.capabilities; - catName = other1326.catName; - processorCapabilities = other1326.processorCapabilities; - processorIdentifier = other1326.processorIdentifier; - projectionSpec = other1326.projectionSpec; - tablesPattern = other1326.tablesPattern; - __isset = other1326.__isset; +GetTablesRequest::GetTablesRequest(const GetTablesRequest& other1337) { + dbName = other1337.dbName; + tblNames = other1337.tblNames; + capabilities = other1337.capabilities; + catName = other1337.catName; + processorCapabilities = other1337.processorCapabilities; + processorIdentifier = other1337.processorIdentifier; + projectionSpec = other1337.projectionSpec; + tablesPattern = other1337.tablesPattern; + __isset = other1337.__isset; +} +GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other1338) { + dbName = other1338.dbName; + tblNames = other1338.tblNames; + capabilities = other1338.capabilities; + catName = other1338.catName; + processorCapabilities = other1338.processorCapabilities; + processorIdentifier = other1338.processorIdentifier; + projectionSpec = other1338.projectionSpec; + tablesPattern = other1338.tablesPattern; + __isset = other1338.__isset; return *this; } void GetTablesRequest::printTo(std::ostream& out) const { @@ -36046,14 +36174,14 @@ uint32_t GetTablesResult::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tables.clear(); - uint32_t _size1327; - ::apache::thrift::protocol::TType _etype1330; - xfer += iprot->readListBegin(_etype1330, _size1327); - this->tables.resize(_size1327); - uint32_t _i1331; - for (_i1331 = 0; _i1331 < _size1327; ++_i1331) + uint32_t _size1339; + ::apache::thrift::protocol::TType _etype1342; + xfer += iprot->readListBegin(_etype1342, _size1339); + this->tables.resize(_size1339); + uint32_t _i1343; + for (_i1343 = 0; _i1343 < _size1339; ++_i1343) { - xfer += this->tables[_i1331].read(iprot); + xfer += this->tables[_i1343].read(iprot); } xfer += iprot->readListEnd(); } @@ -36084,10 +36212,10 @@ uint32_t GetTablesResult::write(::apache::thrift::protocol::TProtocol* oprot) co xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->tables.size())); - std::vector
::const_iterator _iter1332; - for (_iter1332 = this->tables.begin(); _iter1332 != this->tables.end(); ++_iter1332) + std::vector
::const_iterator _iter1344; + for (_iter1344 = this->tables.begin(); _iter1344 != this->tables.end(); ++_iter1344) { - xfer += (*_iter1332).write(oprot); + xfer += (*_iter1344).write(oprot); } xfer += oprot->writeListEnd(); } @@ -36103,11 +36231,11 @@ void swap(GetTablesResult &a, GetTablesResult &b) { swap(a.tables, b.tables); } -GetTablesResult::GetTablesResult(const GetTablesResult& other1333) { - tables = other1333.tables; +GetTablesResult::GetTablesResult(const GetTablesResult& other1345) { + tables = other1345.tables; } -GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other1334) { - tables = other1334.tables; +GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other1346) { + tables = other1346.tables; return *this; } void GetTablesResult::printTo(std::ostream& out) const { @@ -36228,14 +36356,14 @@ uint32_t GetTablesExtRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1335; - ::apache::thrift::protocol::TType _etype1338; - xfer += iprot->readListBegin(_etype1338, _size1335); - this->processorCapabilities.resize(_size1335); - uint32_t _i1339; - for (_i1339 = 0; _i1339 < _size1335; ++_i1339) + uint32_t _size1347; + ::apache::thrift::protocol::TType _etype1350; + xfer += iprot->readListBegin(_etype1350, _size1347); + this->processorCapabilities.resize(_size1347); + uint32_t _i1351; + for (_i1351 = 0; _i1351 < _size1347; ++_i1351) { - xfer += iprot->readString(this->processorCapabilities[_i1339]); + xfer += iprot->readString(this->processorCapabilities[_i1351]); } xfer += iprot->readListEnd(); } @@ -36302,10 +36430,10 @@ uint32_t GetTablesExtRequest::write(::apache::thrift::protocol::TProtocol* oprot xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1340; - for (_iter1340 = this->processorCapabilities.begin(); _iter1340 != this->processorCapabilities.end(); ++_iter1340) + std::vector ::const_iterator _iter1352; + for (_iter1352 = this->processorCapabilities.begin(); _iter1352 != this->processorCapabilities.end(); ++_iter1352) { - xfer += oprot->writeString((*_iter1340)); + xfer += oprot->writeString((*_iter1352)); } xfer += oprot->writeListEnd(); } @@ -36333,25 +36461,25 @@ void swap(GetTablesExtRequest &a, GetTablesExtRequest &b) { swap(a.__isset, b.__isset); } -GetTablesExtRequest::GetTablesExtRequest(const GetTablesExtRequest& other1341) { - catalog = other1341.catalog; - database = other1341.database; - tableNamePattern = other1341.tableNamePattern; - requestedFields = other1341.requestedFields; - limit = other1341.limit; - processorCapabilities = other1341.processorCapabilities; - processorIdentifier = other1341.processorIdentifier; - __isset = other1341.__isset; +GetTablesExtRequest::GetTablesExtRequest(const GetTablesExtRequest& other1353) { + catalog = other1353.catalog; + database = other1353.database; + tableNamePattern = other1353.tableNamePattern; + requestedFields = other1353.requestedFields; + limit = other1353.limit; + processorCapabilities = other1353.processorCapabilities; + processorIdentifier = other1353.processorIdentifier; + __isset = other1353.__isset; } -GetTablesExtRequest& GetTablesExtRequest::operator=(const GetTablesExtRequest& other1342) { - catalog = other1342.catalog; - database = other1342.database; - tableNamePattern = other1342.tableNamePattern; - requestedFields = other1342.requestedFields; - limit = other1342.limit; - processorCapabilities = other1342.processorCapabilities; - processorIdentifier = other1342.processorIdentifier; - __isset = other1342.__isset; +GetTablesExtRequest& GetTablesExtRequest::operator=(const GetTablesExtRequest& other1354) { + catalog = other1354.catalog; + database = other1354.database; + tableNamePattern = other1354.tableNamePattern; + requestedFields = other1354.requestedFields; + limit = other1354.limit; + processorCapabilities = other1354.processorCapabilities; + processorIdentifier = other1354.processorIdentifier; + __isset = other1354.__isset; return *this; } void GetTablesExtRequest::printTo(std::ostream& out) const { @@ -36439,14 +36567,14 @@ uint32_t ExtendedTableInfo::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->requiredReadCapabilities.clear(); - uint32_t _size1343; - ::apache::thrift::protocol::TType _etype1346; - xfer += iprot->readListBegin(_etype1346, _size1343); - this->requiredReadCapabilities.resize(_size1343); - uint32_t _i1347; - for (_i1347 = 0; _i1347 < _size1343; ++_i1347) + uint32_t _size1355; + ::apache::thrift::protocol::TType _etype1358; + xfer += iprot->readListBegin(_etype1358, _size1355); + this->requiredReadCapabilities.resize(_size1355); + uint32_t _i1359; + for (_i1359 = 0; _i1359 < _size1355; ++_i1359) { - xfer += iprot->readString(this->requiredReadCapabilities[_i1347]); + xfer += iprot->readString(this->requiredReadCapabilities[_i1359]); } xfer += iprot->readListEnd(); } @@ -36459,14 +36587,14 @@ uint32_t ExtendedTableInfo::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->requiredWriteCapabilities.clear(); - uint32_t _size1348; - ::apache::thrift::protocol::TType _etype1351; - xfer += iprot->readListBegin(_etype1351, _size1348); - this->requiredWriteCapabilities.resize(_size1348); - uint32_t _i1352; - for (_i1352 = 0; _i1352 < _size1348; ++_i1352) + uint32_t _size1360; + ::apache::thrift::protocol::TType _etype1363; + xfer += iprot->readListBegin(_etype1363, _size1360); + this->requiredWriteCapabilities.resize(_size1360); + uint32_t _i1364; + for (_i1364 = 0; _i1364 < _size1360; ++_i1364) { - xfer += iprot->readString(this->requiredWriteCapabilities[_i1352]); + xfer += iprot->readString(this->requiredWriteCapabilities[_i1364]); } xfer += iprot->readListEnd(); } @@ -36507,10 +36635,10 @@ uint32_t ExtendedTableInfo::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("requiredReadCapabilities", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->requiredReadCapabilities.size())); - std::vector ::const_iterator _iter1353; - for (_iter1353 = this->requiredReadCapabilities.begin(); _iter1353 != this->requiredReadCapabilities.end(); ++_iter1353) + std::vector ::const_iterator _iter1365; + for (_iter1365 = this->requiredReadCapabilities.begin(); _iter1365 != this->requiredReadCapabilities.end(); ++_iter1365) { - xfer += oprot->writeString((*_iter1353)); + xfer += oprot->writeString((*_iter1365)); } xfer += oprot->writeListEnd(); } @@ -36520,10 +36648,10 @@ uint32_t ExtendedTableInfo::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("requiredWriteCapabilities", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->requiredWriteCapabilities.size())); - std::vector ::const_iterator _iter1354; - for (_iter1354 = this->requiredWriteCapabilities.begin(); _iter1354 != this->requiredWriteCapabilities.end(); ++_iter1354) + std::vector ::const_iterator _iter1366; + for (_iter1366 = this->requiredWriteCapabilities.begin(); _iter1366 != this->requiredWriteCapabilities.end(); ++_iter1366) { - xfer += oprot->writeString((*_iter1354)); + xfer += oprot->writeString((*_iter1366)); } xfer += oprot->writeListEnd(); } @@ -36543,19 +36671,19 @@ void swap(ExtendedTableInfo &a, ExtendedTableInfo &b) { swap(a.__isset, b.__isset); } -ExtendedTableInfo::ExtendedTableInfo(const ExtendedTableInfo& other1355) { - tblName = other1355.tblName; - accessType = other1355.accessType; - requiredReadCapabilities = other1355.requiredReadCapabilities; - requiredWriteCapabilities = other1355.requiredWriteCapabilities; - __isset = other1355.__isset; +ExtendedTableInfo::ExtendedTableInfo(const ExtendedTableInfo& other1367) { + tblName = other1367.tblName; + accessType = other1367.accessType; + requiredReadCapabilities = other1367.requiredReadCapabilities; + requiredWriteCapabilities = other1367.requiredWriteCapabilities; + __isset = other1367.__isset; } -ExtendedTableInfo& ExtendedTableInfo::operator=(const ExtendedTableInfo& other1356) { - tblName = other1356.tblName; - accessType = other1356.accessType; - requiredReadCapabilities = other1356.requiredReadCapabilities; - requiredWriteCapabilities = other1356.requiredWriteCapabilities; - __isset = other1356.__isset; +ExtendedTableInfo& ExtendedTableInfo::operator=(const ExtendedTableInfo& other1368) { + tblName = other1368.tblName; + accessType = other1368.accessType; + requiredReadCapabilities = other1368.requiredReadCapabilities; + requiredWriteCapabilities = other1368.requiredWriteCapabilities; + __isset = other1368.__isset; return *this; } void ExtendedTableInfo::printTo(std::ostream& out) const { @@ -36640,14 +36768,14 @@ uint32_t GetDatabaseRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1357; - ::apache::thrift::protocol::TType _etype1360; - xfer += iprot->readListBegin(_etype1360, _size1357); - this->processorCapabilities.resize(_size1357); - uint32_t _i1361; - for (_i1361 = 0; _i1361 < _size1357; ++_i1361) + uint32_t _size1369; + ::apache::thrift::protocol::TType _etype1372; + xfer += iprot->readListBegin(_etype1372, _size1369); + this->processorCapabilities.resize(_size1369); + uint32_t _i1373; + for (_i1373 = 0; _i1373 < _size1369; ++_i1373) { - xfer += iprot->readString(this->processorCapabilities[_i1361]); + xfer += iprot->readString(this->processorCapabilities[_i1373]); } xfer += iprot->readListEnd(); } @@ -36695,10 +36823,10 @@ uint32_t GetDatabaseRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1362; - for (_iter1362 = this->processorCapabilities.begin(); _iter1362 != this->processorCapabilities.end(); ++_iter1362) + std::vector ::const_iterator _iter1374; + for (_iter1374 = this->processorCapabilities.begin(); _iter1374 != this->processorCapabilities.end(); ++_iter1374) { - xfer += oprot->writeString((*_iter1362)); + xfer += oprot->writeString((*_iter1374)); } xfer += oprot->writeListEnd(); } @@ -36723,19 +36851,19 @@ void swap(GetDatabaseRequest &a, GetDatabaseRequest &b) { swap(a.__isset, b.__isset); } -GetDatabaseRequest::GetDatabaseRequest(const GetDatabaseRequest& other1363) { - name = other1363.name; - catalogName = other1363.catalogName; - processorCapabilities = other1363.processorCapabilities; - processorIdentifier = other1363.processorIdentifier; - __isset = other1363.__isset; +GetDatabaseRequest::GetDatabaseRequest(const GetDatabaseRequest& other1375) { + name = other1375.name; + catalogName = other1375.catalogName; + processorCapabilities = other1375.processorCapabilities; + processorIdentifier = other1375.processorIdentifier; + __isset = other1375.__isset; } -GetDatabaseRequest& GetDatabaseRequest::operator=(const GetDatabaseRequest& other1364) { - name = other1364.name; - catalogName = other1364.catalogName; - processorCapabilities = other1364.processorCapabilities; - processorIdentifier = other1364.processorIdentifier; - __isset = other1364.__isset; +GetDatabaseRequest& GetDatabaseRequest::operator=(const GetDatabaseRequest& other1376) { + name = other1376.name; + catalogName = other1376.catalogName; + processorCapabilities = other1376.processorCapabilities; + processorIdentifier = other1376.processorIdentifier; + __isset = other1376.__isset; return *this; } void GetDatabaseRequest::printTo(std::ostream& out) const { @@ -36963,27 +37091,27 @@ void swap(DropDatabaseRequest &a, DropDatabaseRequest &b) { swap(a.__isset, b.__isset); } -DropDatabaseRequest::DropDatabaseRequest(const DropDatabaseRequest& other1365) { - name = other1365.name; - catalogName = other1365.catalogName; - ignoreUnknownDb = other1365.ignoreUnknownDb; - deleteData = other1365.deleteData; - cascade = other1365.cascade; - softDelete = other1365.softDelete; - txnId = other1365.txnId; - deleteManagedDir = other1365.deleteManagedDir; - __isset = other1365.__isset; -} -DropDatabaseRequest& DropDatabaseRequest::operator=(const DropDatabaseRequest& other1366) { - name = other1366.name; - catalogName = other1366.catalogName; - ignoreUnknownDb = other1366.ignoreUnknownDb; - deleteData = other1366.deleteData; - cascade = other1366.cascade; - softDelete = other1366.softDelete; - txnId = other1366.txnId; - deleteManagedDir = other1366.deleteManagedDir; - __isset = other1366.__isset; +DropDatabaseRequest::DropDatabaseRequest(const DropDatabaseRequest& other1377) { + name = other1377.name; + catalogName = other1377.catalogName; + ignoreUnknownDb = other1377.ignoreUnknownDb; + deleteData = other1377.deleteData; + cascade = other1377.cascade; + softDelete = other1377.softDelete; + txnId = other1377.txnId; + deleteManagedDir = other1377.deleteManagedDir; + __isset = other1377.__isset; +} +DropDatabaseRequest& DropDatabaseRequest::operator=(const DropDatabaseRequest& other1378) { + name = other1378.name; + catalogName = other1378.catalogName; + ignoreUnknownDb = other1378.ignoreUnknownDb; + deleteData = other1378.deleteData; + cascade = other1378.cascade; + softDelete = other1378.softDelete; + txnId = other1378.txnId; + deleteManagedDir = other1378.deleteManagedDir; + __isset = other1378.__isset; return *this; } void DropDatabaseRequest::printTo(std::ostream& out) const { @@ -37098,13 +37226,13 @@ void swap(CmRecycleRequest &a, CmRecycleRequest &b) { swap(a.purge, b.purge); } -CmRecycleRequest::CmRecycleRequest(const CmRecycleRequest& other1367) { - dataPath = other1367.dataPath; - purge = other1367.purge; +CmRecycleRequest::CmRecycleRequest(const CmRecycleRequest& other1379) { + dataPath = other1379.dataPath; + purge = other1379.purge; } -CmRecycleRequest& CmRecycleRequest::operator=(const CmRecycleRequest& other1368) { - dataPath = other1368.dataPath; - purge = other1368.purge; +CmRecycleRequest& CmRecycleRequest::operator=(const CmRecycleRequest& other1380) { + dataPath = other1380.dataPath; + purge = other1380.purge; return *this; } void CmRecycleRequest::printTo(std::ostream& out) const { @@ -37170,11 +37298,11 @@ void swap(CmRecycleResponse &a, CmRecycleResponse &b) { (void) b; } -CmRecycleResponse::CmRecycleResponse(const CmRecycleResponse& other1369) noexcept { - (void) other1369; +CmRecycleResponse::CmRecycleResponse(const CmRecycleResponse& other1381) noexcept { + (void) other1381; } -CmRecycleResponse& CmRecycleResponse::operator=(const CmRecycleResponse& other1370) noexcept { - (void) other1370; +CmRecycleResponse& CmRecycleResponse::operator=(const CmRecycleResponse& other1382) noexcept { + (void) other1382; return *this; } void CmRecycleResponse::printTo(std::ostream& out) const { @@ -37300,9 +37428,9 @@ uint32_t TableMeta::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 7: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1371; - xfer += iprot->readI32(ecast1371); - this->ownerType = static_cast(ecast1371); + int32_t ecast1383; + xfer += iprot->readI32(ecast1383); + this->ownerType = static_cast(ecast1383); this->__isset.ownerType = true; } else { xfer += iprot->skip(ftype); @@ -37380,25 +37508,25 @@ void swap(TableMeta &a, TableMeta &b) { swap(a.__isset, b.__isset); } -TableMeta::TableMeta(const TableMeta& other1372) { - dbName = other1372.dbName; - tableName = other1372.tableName; - tableType = other1372.tableType; - comments = other1372.comments; - catName = other1372.catName; - ownerName = other1372.ownerName; - ownerType = other1372.ownerType; - __isset = other1372.__isset; +TableMeta::TableMeta(const TableMeta& other1384) { + dbName = other1384.dbName; + tableName = other1384.tableName; + tableType = other1384.tableType; + comments = other1384.comments; + catName = other1384.catName; + ownerName = other1384.ownerName; + ownerType = other1384.ownerType; + __isset = other1384.__isset; } -TableMeta& TableMeta::operator=(const TableMeta& other1373) { - dbName = other1373.dbName; - tableName = other1373.tableName; - tableType = other1373.tableType; - comments = other1373.comments; - catName = other1373.catName; - ownerName = other1373.ownerName; - ownerType = other1373.ownerType; - __isset = other1373.__isset; +TableMeta& TableMeta::operator=(const TableMeta& other1385) { + dbName = other1385.dbName; + tableName = other1385.tableName; + tableType = other1385.tableType; + comments = other1385.comments; + catName = other1385.catName; + ownerName = other1385.ownerName; + ownerType = other1385.ownerType; + __isset = other1385.__isset; return *this; } void TableMeta::printTo(std::ostream& out) const { @@ -37512,13 +37640,13 @@ void swap(Materialization &a, Materialization &b) { swap(a.sourceTablesCompacted, b.sourceTablesCompacted); } -Materialization::Materialization(const Materialization& other1374) noexcept { - sourceTablesUpdateDeleteModified = other1374.sourceTablesUpdateDeleteModified; - sourceTablesCompacted = other1374.sourceTablesCompacted; +Materialization::Materialization(const Materialization& other1386) noexcept { + sourceTablesUpdateDeleteModified = other1386.sourceTablesUpdateDeleteModified; + sourceTablesCompacted = other1386.sourceTablesCompacted; } -Materialization& Materialization::operator=(const Materialization& other1375) noexcept { - sourceTablesUpdateDeleteModified = other1375.sourceTablesUpdateDeleteModified; - sourceTablesCompacted = other1375.sourceTablesCompacted; +Materialization& Materialization::operator=(const Materialization& other1387) noexcept { + sourceTablesUpdateDeleteModified = other1387.sourceTablesUpdateDeleteModified; + sourceTablesCompacted = other1387.sourceTablesCompacted; return *this; } void Materialization::printTo(std::ostream& out) const { @@ -37596,9 +37724,9 @@ uint32_t WMResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1376; - xfer += iprot->readI32(ecast1376); - this->status = static_cast(ecast1376); + int32_t ecast1388; + xfer += iprot->readI32(ecast1388); + this->status = static_cast(ecast1388); this->__isset.status = true; } else { xfer += iprot->skip(ftype); @@ -37686,21 +37814,21 @@ void swap(WMResourcePlan &a, WMResourcePlan &b) { swap(a.__isset, b.__isset); } -WMResourcePlan::WMResourcePlan(const WMResourcePlan& other1377) { - name = other1377.name; - status = other1377.status; - queryParallelism = other1377.queryParallelism; - defaultPoolPath = other1377.defaultPoolPath; - ns = other1377.ns; - __isset = other1377.__isset; +WMResourcePlan::WMResourcePlan(const WMResourcePlan& other1389) { + name = other1389.name; + status = other1389.status; + queryParallelism = other1389.queryParallelism; + defaultPoolPath = other1389.defaultPoolPath; + ns = other1389.ns; + __isset = other1389.__isset; } -WMResourcePlan& WMResourcePlan::operator=(const WMResourcePlan& other1378) { - name = other1378.name; - status = other1378.status; - queryParallelism = other1378.queryParallelism; - defaultPoolPath = other1378.defaultPoolPath; - ns = other1378.ns; - __isset = other1378.__isset; +WMResourcePlan& WMResourcePlan::operator=(const WMResourcePlan& other1390) { + name = other1390.name; + status = other1390.status; + queryParallelism = other1390.queryParallelism; + defaultPoolPath = other1390.defaultPoolPath; + ns = other1390.ns; + __isset = other1390.__isset; return *this; } void WMResourcePlan::printTo(std::ostream& out) const { @@ -37791,9 +37919,9 @@ uint32_t WMNullableResourcePlan::read(::apache::thrift::protocol::TProtocol* ipr break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1379; - xfer += iprot->readI32(ecast1379); - this->status = static_cast(ecast1379); + int32_t ecast1391; + xfer += iprot->readI32(ecast1391); + this->status = static_cast(ecast1391); this->__isset.status = true; } else { xfer += iprot->skip(ftype); @@ -37908,25 +38036,25 @@ void swap(WMNullableResourcePlan &a, WMNullableResourcePlan &b) { swap(a.__isset, b.__isset); } -WMNullableResourcePlan::WMNullableResourcePlan(const WMNullableResourcePlan& other1380) { - name = other1380.name; - status = other1380.status; - queryParallelism = other1380.queryParallelism; - isSetQueryParallelism = other1380.isSetQueryParallelism; - defaultPoolPath = other1380.defaultPoolPath; - isSetDefaultPoolPath = other1380.isSetDefaultPoolPath; - ns = other1380.ns; - __isset = other1380.__isset; +WMNullableResourcePlan::WMNullableResourcePlan(const WMNullableResourcePlan& other1392) { + name = other1392.name; + status = other1392.status; + queryParallelism = other1392.queryParallelism; + isSetQueryParallelism = other1392.isSetQueryParallelism; + defaultPoolPath = other1392.defaultPoolPath; + isSetDefaultPoolPath = other1392.isSetDefaultPoolPath; + ns = other1392.ns; + __isset = other1392.__isset; } -WMNullableResourcePlan& WMNullableResourcePlan::operator=(const WMNullableResourcePlan& other1381) { - name = other1381.name; - status = other1381.status; - queryParallelism = other1381.queryParallelism; - isSetQueryParallelism = other1381.isSetQueryParallelism; - defaultPoolPath = other1381.defaultPoolPath; - isSetDefaultPoolPath = other1381.isSetDefaultPoolPath; - ns = other1381.ns; - __isset = other1381.__isset; +WMNullableResourcePlan& WMNullableResourcePlan::operator=(const WMNullableResourcePlan& other1393) { + name = other1393.name; + status = other1393.status; + queryParallelism = other1393.queryParallelism; + isSetQueryParallelism = other1393.isSetQueryParallelism; + defaultPoolPath = other1393.defaultPoolPath; + isSetDefaultPoolPath = other1393.isSetDefaultPoolPath; + ns = other1393.ns; + __isset = other1393.__isset; return *this; } void WMNullableResourcePlan::printTo(std::ostream& out) const { @@ -38117,23 +38245,23 @@ void swap(WMPool &a, WMPool &b) { swap(a.__isset, b.__isset); } -WMPool::WMPool(const WMPool& other1382) { - resourcePlanName = other1382.resourcePlanName; - poolPath = other1382.poolPath; - allocFraction = other1382.allocFraction; - queryParallelism = other1382.queryParallelism; - schedulingPolicy = other1382.schedulingPolicy; - ns = other1382.ns; - __isset = other1382.__isset; +WMPool::WMPool(const WMPool& other1394) { + resourcePlanName = other1394.resourcePlanName; + poolPath = other1394.poolPath; + allocFraction = other1394.allocFraction; + queryParallelism = other1394.queryParallelism; + schedulingPolicy = other1394.schedulingPolicy; + ns = other1394.ns; + __isset = other1394.__isset; } -WMPool& WMPool::operator=(const WMPool& other1383) { - resourcePlanName = other1383.resourcePlanName; - poolPath = other1383.poolPath; - allocFraction = other1383.allocFraction; - queryParallelism = other1383.queryParallelism; - schedulingPolicy = other1383.schedulingPolicy; - ns = other1383.ns; - __isset = other1383.__isset; +WMPool& WMPool::operator=(const WMPool& other1395) { + resourcePlanName = other1395.resourcePlanName; + poolPath = other1395.poolPath; + allocFraction = other1395.allocFraction; + queryParallelism = other1395.queryParallelism; + schedulingPolicy = other1395.schedulingPolicy; + ns = other1395.ns; + __isset = other1395.__isset; return *this; } void WMPool::printTo(std::ostream& out) const { @@ -38342,25 +38470,25 @@ void swap(WMNullablePool &a, WMNullablePool &b) { swap(a.__isset, b.__isset); } -WMNullablePool::WMNullablePool(const WMNullablePool& other1384) { - resourcePlanName = other1384.resourcePlanName; - poolPath = other1384.poolPath; - allocFraction = other1384.allocFraction; - queryParallelism = other1384.queryParallelism; - schedulingPolicy = other1384.schedulingPolicy; - isSetSchedulingPolicy = other1384.isSetSchedulingPolicy; - ns = other1384.ns; - __isset = other1384.__isset; +WMNullablePool::WMNullablePool(const WMNullablePool& other1396) { + resourcePlanName = other1396.resourcePlanName; + poolPath = other1396.poolPath; + allocFraction = other1396.allocFraction; + queryParallelism = other1396.queryParallelism; + schedulingPolicy = other1396.schedulingPolicy; + isSetSchedulingPolicy = other1396.isSetSchedulingPolicy; + ns = other1396.ns; + __isset = other1396.__isset; } -WMNullablePool& WMNullablePool::operator=(const WMNullablePool& other1385) { - resourcePlanName = other1385.resourcePlanName; - poolPath = other1385.poolPath; - allocFraction = other1385.allocFraction; - queryParallelism = other1385.queryParallelism; - schedulingPolicy = other1385.schedulingPolicy; - isSetSchedulingPolicy = other1385.isSetSchedulingPolicy; - ns = other1385.ns; - __isset = other1385.__isset; +WMNullablePool& WMNullablePool::operator=(const WMNullablePool& other1397) { + resourcePlanName = other1397.resourcePlanName; + poolPath = other1397.poolPath; + allocFraction = other1397.allocFraction; + queryParallelism = other1397.queryParallelism; + schedulingPolicy = other1397.schedulingPolicy; + isSetSchedulingPolicy = other1397.isSetSchedulingPolicy; + ns = other1397.ns; + __isset = other1397.__isset; return *this; } void WMNullablePool::printTo(std::ostream& out) const { @@ -38551,23 +38679,23 @@ void swap(WMTrigger &a, WMTrigger &b) { swap(a.__isset, b.__isset); } -WMTrigger::WMTrigger(const WMTrigger& other1386) { - resourcePlanName = other1386.resourcePlanName; - triggerName = other1386.triggerName; - triggerExpression = other1386.triggerExpression; - actionExpression = other1386.actionExpression; - isInUnmanaged = other1386.isInUnmanaged; - ns = other1386.ns; - __isset = other1386.__isset; +WMTrigger::WMTrigger(const WMTrigger& other1398) { + resourcePlanName = other1398.resourcePlanName; + triggerName = other1398.triggerName; + triggerExpression = other1398.triggerExpression; + actionExpression = other1398.actionExpression; + isInUnmanaged = other1398.isInUnmanaged; + ns = other1398.ns; + __isset = other1398.__isset; } -WMTrigger& WMTrigger::operator=(const WMTrigger& other1387) { - resourcePlanName = other1387.resourcePlanName; - triggerName = other1387.triggerName; - triggerExpression = other1387.triggerExpression; - actionExpression = other1387.actionExpression; - isInUnmanaged = other1387.isInUnmanaged; - ns = other1387.ns; - __isset = other1387.__isset; +WMTrigger& WMTrigger::operator=(const WMTrigger& other1399) { + resourcePlanName = other1399.resourcePlanName; + triggerName = other1399.triggerName; + triggerExpression = other1399.triggerExpression; + actionExpression = other1399.actionExpression; + isInUnmanaged = other1399.isInUnmanaged; + ns = other1399.ns; + __isset = other1399.__isset; return *this; } void WMTrigger::printTo(std::ostream& out) const { @@ -38758,23 +38886,23 @@ void swap(WMMapping &a, WMMapping &b) { swap(a.__isset, b.__isset); } -WMMapping::WMMapping(const WMMapping& other1388) { - resourcePlanName = other1388.resourcePlanName; - entityType = other1388.entityType; - entityName = other1388.entityName; - poolPath = other1388.poolPath; - ordering = other1388.ordering; - ns = other1388.ns; - __isset = other1388.__isset; -} -WMMapping& WMMapping::operator=(const WMMapping& other1389) { - resourcePlanName = other1389.resourcePlanName; - entityType = other1389.entityType; - entityName = other1389.entityName; - poolPath = other1389.poolPath; - ordering = other1389.ordering; - ns = other1389.ns; - __isset = other1389.__isset; +WMMapping::WMMapping(const WMMapping& other1400) { + resourcePlanName = other1400.resourcePlanName; + entityType = other1400.entityType; + entityName = other1400.entityName; + poolPath = other1400.poolPath; + ordering = other1400.ordering; + ns = other1400.ns; + __isset = other1400.__isset; +} +WMMapping& WMMapping::operator=(const WMMapping& other1401) { + resourcePlanName = other1401.resourcePlanName; + entityType = other1401.entityType; + entityName = other1401.entityName; + poolPath = other1401.poolPath; + ordering = other1401.ordering; + ns = other1401.ns; + __isset = other1401.__isset; return *this; } void WMMapping::printTo(std::ostream& out) const { @@ -38907,17 +39035,17 @@ void swap(WMPoolTrigger &a, WMPoolTrigger &b) { swap(a.__isset, b.__isset); } -WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other1390) { - pool = other1390.pool; - trigger = other1390.trigger; - ns = other1390.ns; - __isset = other1390.__isset; +WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other1402) { + pool = other1402.pool; + trigger = other1402.trigger; + ns = other1402.ns; + __isset = other1402.__isset; } -WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other1391) { - pool = other1391.pool; - trigger = other1391.trigger; - ns = other1391.ns; - __isset = other1391.__isset; +WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other1403) { + pool = other1403.pool; + trigger = other1403.trigger; + ns = other1403.ns; + __isset = other1403.__isset; return *this; } void WMPoolTrigger::printTo(std::ostream& out) const { @@ -38998,14 +39126,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->pools.clear(); - uint32_t _size1392; - ::apache::thrift::protocol::TType _etype1395; - xfer += iprot->readListBegin(_etype1395, _size1392); - this->pools.resize(_size1392); - uint32_t _i1396; - for (_i1396 = 0; _i1396 < _size1392; ++_i1396) + uint32_t _size1404; + ::apache::thrift::protocol::TType _etype1407; + xfer += iprot->readListBegin(_etype1407, _size1404); + this->pools.resize(_size1404); + uint32_t _i1408; + for (_i1408 = 0; _i1408 < _size1404; ++_i1408) { - xfer += this->pools[_i1396].read(iprot); + xfer += this->pools[_i1408].read(iprot); } xfer += iprot->readListEnd(); } @@ -39018,14 +39146,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->mappings.clear(); - uint32_t _size1397; - ::apache::thrift::protocol::TType _etype1400; - xfer += iprot->readListBegin(_etype1400, _size1397); - this->mappings.resize(_size1397); - uint32_t _i1401; - for (_i1401 = 0; _i1401 < _size1397; ++_i1401) + uint32_t _size1409; + ::apache::thrift::protocol::TType _etype1412; + xfer += iprot->readListBegin(_etype1412, _size1409); + this->mappings.resize(_size1409); + uint32_t _i1413; + for (_i1413 = 0; _i1413 < _size1409; ++_i1413) { - xfer += this->mappings[_i1401].read(iprot); + xfer += this->mappings[_i1413].read(iprot); } xfer += iprot->readListEnd(); } @@ -39038,14 +39166,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->triggers.clear(); - uint32_t _size1402; - ::apache::thrift::protocol::TType _etype1405; - xfer += iprot->readListBegin(_etype1405, _size1402); - this->triggers.resize(_size1402); - uint32_t _i1406; - for (_i1406 = 0; _i1406 < _size1402; ++_i1406) + uint32_t _size1414; + ::apache::thrift::protocol::TType _etype1417; + xfer += iprot->readListBegin(_etype1417, _size1414); + this->triggers.resize(_size1414); + uint32_t _i1418; + for (_i1418 = 0; _i1418 < _size1414; ++_i1418) { - xfer += this->triggers[_i1406].read(iprot); + xfer += this->triggers[_i1418].read(iprot); } xfer += iprot->readListEnd(); } @@ -39058,14 +39186,14 @@ uint32_t WMFullResourcePlan::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->poolTriggers.clear(); - uint32_t _size1407; - ::apache::thrift::protocol::TType _etype1410; - xfer += iprot->readListBegin(_etype1410, _size1407); - this->poolTriggers.resize(_size1407); - uint32_t _i1411; - for (_i1411 = 0; _i1411 < _size1407; ++_i1411) + uint32_t _size1419; + ::apache::thrift::protocol::TType _etype1422; + xfer += iprot->readListBegin(_etype1422, _size1419); + this->poolTriggers.resize(_size1419); + uint32_t _i1423; + for (_i1423 = 0; _i1423 < _size1419; ++_i1423) { - xfer += this->poolTriggers[_i1411].read(iprot); + xfer += this->poolTriggers[_i1423].read(iprot); } xfer += iprot->readListEnd(); } @@ -39102,10 +39230,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("pools", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->pools.size())); - std::vector ::const_iterator _iter1412; - for (_iter1412 = this->pools.begin(); _iter1412 != this->pools.end(); ++_iter1412) + std::vector ::const_iterator _iter1424; + for (_iter1424 = this->pools.begin(); _iter1424 != this->pools.end(); ++_iter1424) { - xfer += (*_iter1412).write(oprot); + xfer += (*_iter1424).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39115,10 +39243,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("mappings", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->mappings.size())); - std::vector ::const_iterator _iter1413; - for (_iter1413 = this->mappings.begin(); _iter1413 != this->mappings.end(); ++_iter1413) + std::vector ::const_iterator _iter1425; + for (_iter1425 = this->mappings.begin(); _iter1425 != this->mappings.end(); ++_iter1425) { - xfer += (*_iter1413).write(oprot); + xfer += (*_iter1425).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39128,10 +39256,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->triggers.size())); - std::vector ::const_iterator _iter1414; - for (_iter1414 = this->triggers.begin(); _iter1414 != this->triggers.end(); ++_iter1414) + std::vector ::const_iterator _iter1426; + for (_iter1426 = this->triggers.begin(); _iter1426 != this->triggers.end(); ++_iter1426) { - xfer += (*_iter1414).write(oprot); + xfer += (*_iter1426).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39141,10 +39269,10 @@ uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("poolTriggers", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->poolTriggers.size())); - std::vector ::const_iterator _iter1415; - for (_iter1415 = this->poolTriggers.begin(); _iter1415 != this->poolTriggers.end(); ++_iter1415) + std::vector ::const_iterator _iter1427; + for (_iter1427 = this->poolTriggers.begin(); _iter1427 != this->poolTriggers.end(); ++_iter1427) { - xfer += (*_iter1415).write(oprot); + xfer += (*_iter1427).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39165,21 +39293,21 @@ void swap(WMFullResourcePlan &a, WMFullResourcePlan &b) { swap(a.__isset, b.__isset); } -WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other1416) { - plan = other1416.plan; - pools = other1416.pools; - mappings = other1416.mappings; - triggers = other1416.triggers; - poolTriggers = other1416.poolTriggers; - __isset = other1416.__isset; +WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other1428) { + plan = other1428.plan; + pools = other1428.pools; + mappings = other1428.mappings; + triggers = other1428.triggers; + poolTriggers = other1428.poolTriggers; + __isset = other1428.__isset; } -WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other1417) { - plan = other1417.plan; - pools = other1417.pools; - mappings = other1417.mappings; - triggers = other1417.triggers; - poolTriggers = other1417.poolTriggers; - __isset = other1417.__isset; +WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other1429) { + plan = other1429.plan; + pools = other1429.pools; + mappings = other1429.mappings; + triggers = other1429.triggers; + poolTriggers = other1429.poolTriggers; + __isset = other1429.__isset; return *this; } void WMFullResourcePlan::printTo(std::ostream& out) const { @@ -39290,15 +39418,15 @@ void swap(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b) { swap(a.__isset, b.__isset); } -WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other1418) { - resourcePlan = other1418.resourcePlan; - copyFrom = other1418.copyFrom; - __isset = other1418.__isset; +WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other1430) { + resourcePlan = other1430.resourcePlan; + copyFrom = other1430.copyFrom; + __isset = other1430.__isset; } -WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other1419) { - resourcePlan = other1419.resourcePlan; - copyFrom = other1419.copyFrom; - __isset = other1419.__isset; +WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other1431) { + resourcePlan = other1431.resourcePlan; + copyFrom = other1431.copyFrom; + __isset = other1431.__isset; return *this; } void WMCreateResourcePlanRequest::printTo(std::ostream& out) const { @@ -39364,11 +39492,11 @@ void swap(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b) { (void) b; } -WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other1420) noexcept { - (void) other1420; +WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other1432) noexcept { + (void) other1432; } -WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other1421) noexcept { - (void) other1421; +WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other1433) noexcept { + (void) other1433; return *this; } void WMCreateResourcePlanResponse::printTo(std::ostream& out) const { @@ -39455,13 +39583,13 @@ void swap(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b) swap(a.__isset, b.__isset); } -WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other1422) { - ns = other1422.ns; - __isset = other1422.__isset; +WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other1434) { + ns = other1434.ns; + __isset = other1434.__isset; } -WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other1423) { - ns = other1423.ns; - __isset = other1423.__isset; +WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other1435) { + ns = other1435.ns; + __isset = other1435.__isset; return *this; } void WMGetActiveResourcePlanRequest::printTo(std::ostream& out) const { @@ -39549,13 +39677,13 @@ void swap(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b swap(a.__isset, b.__isset); } -WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other1424) { - resourcePlan = other1424.resourcePlan; - __isset = other1424.__isset; +WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other1436) { + resourcePlan = other1436.resourcePlan; + __isset = other1436.__isset; } -WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other1425) { - resourcePlan = other1425.resourcePlan; - __isset = other1425.__isset; +WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other1437) { + resourcePlan = other1437.resourcePlan; + __isset = other1437.__isset; return *this; } void WMGetActiveResourcePlanResponse::printTo(std::ostream& out) const { @@ -39662,15 +39790,15 @@ void swap(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b) { swap(a.__isset, b.__isset); } -WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other1426) { - resourcePlanName = other1426.resourcePlanName; - ns = other1426.ns; - __isset = other1426.__isset; +WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other1438) { + resourcePlanName = other1438.resourcePlanName; + ns = other1438.ns; + __isset = other1438.__isset; } -WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other1427) { - resourcePlanName = other1427.resourcePlanName; - ns = other1427.ns; - __isset = other1427.__isset; +WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other1439) { + resourcePlanName = other1439.resourcePlanName; + ns = other1439.ns; + __isset = other1439.__isset; return *this; } void WMGetResourcePlanRequest::printTo(std::ostream& out) const { @@ -39759,13 +39887,13 @@ void swap(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b) { swap(a.__isset, b.__isset); } -WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other1428) { - resourcePlan = other1428.resourcePlan; - __isset = other1428.__isset; +WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other1440) { + resourcePlan = other1440.resourcePlan; + __isset = other1440.__isset; } -WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other1429) { - resourcePlan = other1429.resourcePlan; - __isset = other1429.__isset; +WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other1441) { + resourcePlan = other1441.resourcePlan; + __isset = other1441.__isset; return *this; } void WMGetResourcePlanResponse::printTo(std::ostream& out) const { @@ -39853,13 +39981,13 @@ void swap(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b) { swap(a.__isset, b.__isset); } -WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other1430) { - ns = other1430.ns; - __isset = other1430.__isset; +WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other1442) { + ns = other1442.ns; + __isset = other1442.__isset; } -WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other1431) { - ns = other1431.ns; - __isset = other1431.__isset; +WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other1443) { + ns = other1443.ns; + __isset = other1443.__isset; return *this; } void WMGetAllResourcePlanRequest::printTo(std::ostream& out) const { @@ -39910,14 +40038,14 @@ uint32_t WMGetAllResourcePlanResponse::read(::apache::thrift::protocol::TProtoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->resourcePlans.clear(); - uint32_t _size1432; - ::apache::thrift::protocol::TType _etype1435; - xfer += iprot->readListBegin(_etype1435, _size1432); - this->resourcePlans.resize(_size1432); - uint32_t _i1436; - for (_i1436 = 0; _i1436 < _size1432; ++_i1436) + uint32_t _size1444; + ::apache::thrift::protocol::TType _etype1447; + xfer += iprot->readListBegin(_etype1447, _size1444); + this->resourcePlans.resize(_size1444); + uint32_t _i1448; + for (_i1448 = 0; _i1448 < _size1444; ++_i1448) { - xfer += this->resourcePlans[_i1436].read(iprot); + xfer += this->resourcePlans[_i1448].read(iprot); } xfer += iprot->readListEnd(); } @@ -39947,10 +40075,10 @@ uint32_t WMGetAllResourcePlanResponse::write(::apache::thrift::protocol::TProtoc xfer += oprot->writeFieldBegin("resourcePlans", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->resourcePlans.size())); - std::vector ::const_iterator _iter1437; - for (_iter1437 = this->resourcePlans.begin(); _iter1437 != this->resourcePlans.end(); ++_iter1437) + std::vector ::const_iterator _iter1449; + for (_iter1449 = this->resourcePlans.begin(); _iter1449 != this->resourcePlans.end(); ++_iter1449) { - xfer += (*_iter1437).write(oprot); + xfer += (*_iter1449).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39967,13 +40095,13 @@ void swap(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b) { swap(a.__isset, b.__isset); } -WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other1438) { - resourcePlans = other1438.resourcePlans; - __isset = other1438.__isset; +WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other1450) { + resourcePlans = other1450.resourcePlans; + __isset = other1450.__isset; } -WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other1439) { - resourcePlans = other1439.resourcePlans; - __isset = other1439.__isset; +WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other1451) { + resourcePlans = other1451.resourcePlans; + __isset = other1451.__isset; return *this; } void WMGetAllResourcePlanResponse::printTo(std::ostream& out) const { @@ -40156,23 +40284,23 @@ void swap(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b) { swap(a.__isset, b.__isset); } -WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other1440) { - resourcePlanName = other1440.resourcePlanName; - resourcePlan = other1440.resourcePlan; - isEnableAndActivate = other1440.isEnableAndActivate; - isForceDeactivate = other1440.isForceDeactivate; - isReplace = other1440.isReplace; - ns = other1440.ns; - __isset = other1440.__isset; +WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other1452) { + resourcePlanName = other1452.resourcePlanName; + resourcePlan = other1452.resourcePlan; + isEnableAndActivate = other1452.isEnableAndActivate; + isForceDeactivate = other1452.isForceDeactivate; + isReplace = other1452.isReplace; + ns = other1452.ns; + __isset = other1452.__isset; } -WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other1441) { - resourcePlanName = other1441.resourcePlanName; - resourcePlan = other1441.resourcePlan; - isEnableAndActivate = other1441.isEnableAndActivate; - isForceDeactivate = other1441.isForceDeactivate; - isReplace = other1441.isReplace; - ns = other1441.ns; - __isset = other1441.__isset; +WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other1453) { + resourcePlanName = other1453.resourcePlanName; + resourcePlan = other1453.resourcePlan; + isEnableAndActivate = other1453.isEnableAndActivate; + isForceDeactivate = other1453.isForceDeactivate; + isReplace = other1453.isReplace; + ns = other1453.ns; + __isset = other1453.__isset; return *this; } void WMAlterResourcePlanRequest::printTo(std::ostream& out) const { @@ -40265,13 +40393,13 @@ void swap(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b) { swap(a.__isset, b.__isset); } -WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other1442) { - fullResourcePlan = other1442.fullResourcePlan; - __isset = other1442.__isset; +WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other1454) { + fullResourcePlan = other1454.fullResourcePlan; + __isset = other1454.__isset; } -WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other1443) { - fullResourcePlan = other1443.fullResourcePlan; - __isset = other1443.__isset; +WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other1455) { + fullResourcePlan = other1455.fullResourcePlan; + __isset = other1455.__isset; return *this; } void WMAlterResourcePlanResponse::printTo(std::ostream& out) const { @@ -40378,15 +40506,15 @@ void swap(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b) { swap(a.__isset, b.__isset); } -WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other1444) { - resourcePlanName = other1444.resourcePlanName; - ns = other1444.ns; - __isset = other1444.__isset; +WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other1456) { + resourcePlanName = other1456.resourcePlanName; + ns = other1456.ns; + __isset = other1456.__isset; } -WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other1445) { - resourcePlanName = other1445.resourcePlanName; - ns = other1445.ns; - __isset = other1445.__isset; +WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other1457) { + resourcePlanName = other1457.resourcePlanName; + ns = other1457.ns; + __isset = other1457.__isset; return *this; } void WMValidateResourcePlanRequest::printTo(std::ostream& out) const { @@ -40443,14 +40571,14 @@ uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->errors.clear(); - uint32_t _size1446; - ::apache::thrift::protocol::TType _etype1449; - xfer += iprot->readListBegin(_etype1449, _size1446); - this->errors.resize(_size1446); - uint32_t _i1450; - for (_i1450 = 0; _i1450 < _size1446; ++_i1450) + uint32_t _size1458; + ::apache::thrift::protocol::TType _etype1461; + xfer += iprot->readListBegin(_etype1461, _size1458); + this->errors.resize(_size1458); + uint32_t _i1462; + for (_i1462 = 0; _i1462 < _size1458; ++_i1462) { - xfer += iprot->readString(this->errors[_i1450]); + xfer += iprot->readString(this->errors[_i1462]); } xfer += iprot->readListEnd(); } @@ -40463,14 +40591,14 @@ uint32_t WMValidateResourcePlanResponse::read(::apache::thrift::protocol::TProto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->warnings.clear(); - uint32_t _size1451; - ::apache::thrift::protocol::TType _etype1454; - xfer += iprot->readListBegin(_etype1454, _size1451); - this->warnings.resize(_size1451); - uint32_t _i1455; - for (_i1455 = 0; _i1455 < _size1451; ++_i1455) + uint32_t _size1463; + ::apache::thrift::protocol::TType _etype1466; + xfer += iprot->readListBegin(_etype1466, _size1463); + this->warnings.resize(_size1463); + uint32_t _i1467; + for (_i1467 = 0; _i1467 < _size1463; ++_i1467) { - xfer += iprot->readString(this->warnings[_i1455]); + xfer += iprot->readString(this->warnings[_i1467]); } xfer += iprot->readListEnd(); } @@ -40500,10 +40628,10 @@ uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProt xfer += oprot->writeFieldBegin("errors", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->errors.size())); - std::vector ::const_iterator _iter1456; - for (_iter1456 = this->errors.begin(); _iter1456 != this->errors.end(); ++_iter1456) + std::vector ::const_iterator _iter1468; + for (_iter1468 = this->errors.begin(); _iter1468 != this->errors.end(); ++_iter1468) { - xfer += oprot->writeString((*_iter1456)); + xfer += oprot->writeString((*_iter1468)); } xfer += oprot->writeListEnd(); } @@ -40513,10 +40641,10 @@ uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProt xfer += oprot->writeFieldBegin("warnings", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->warnings.size())); - std::vector ::const_iterator _iter1457; - for (_iter1457 = this->warnings.begin(); _iter1457 != this->warnings.end(); ++_iter1457) + std::vector ::const_iterator _iter1469; + for (_iter1469 = this->warnings.begin(); _iter1469 != this->warnings.end(); ++_iter1469) { - xfer += oprot->writeString((*_iter1457)); + xfer += oprot->writeString((*_iter1469)); } xfer += oprot->writeListEnd(); } @@ -40534,15 +40662,15 @@ void swap(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b) swap(a.__isset, b.__isset); } -WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other1458) { - errors = other1458.errors; - warnings = other1458.warnings; - __isset = other1458.__isset; +WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other1470) { + errors = other1470.errors; + warnings = other1470.warnings; + __isset = other1470.__isset; } -WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other1459) { - errors = other1459.errors; - warnings = other1459.warnings; - __isset = other1459.__isset; +WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other1471) { + errors = other1471.errors; + warnings = other1471.warnings; + __isset = other1471.__isset; return *this; } void WMValidateResourcePlanResponse::printTo(std::ostream& out) const { @@ -40650,15 +40778,15 @@ void swap(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b) { swap(a.__isset, b.__isset); } -WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other1460) { - resourcePlanName = other1460.resourcePlanName; - ns = other1460.ns; - __isset = other1460.__isset; +WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other1472) { + resourcePlanName = other1472.resourcePlanName; + ns = other1472.ns; + __isset = other1472.__isset; } -WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other1461) { - resourcePlanName = other1461.resourcePlanName; - ns = other1461.ns; - __isset = other1461.__isset; +WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other1473) { + resourcePlanName = other1473.resourcePlanName; + ns = other1473.ns; + __isset = other1473.__isset; return *this; } void WMDropResourcePlanRequest::printTo(std::ostream& out) const { @@ -40724,11 +40852,11 @@ void swap(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b) { (void) b; } -WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other1462) noexcept { - (void) other1462; +WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other1474) noexcept { + (void) other1474; } -WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other1463) noexcept { - (void) other1463; +WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other1475) noexcept { + (void) other1475; return *this; } void WMDropResourcePlanResponse::printTo(std::ostream& out) const { @@ -40815,13 +40943,13 @@ void swap(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b) { swap(a.__isset, b.__isset); } -WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other1464) { - trigger = other1464.trigger; - __isset = other1464.__isset; +WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other1476) { + trigger = other1476.trigger; + __isset = other1476.__isset; } -WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other1465) { - trigger = other1465.trigger; - __isset = other1465.__isset; +WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other1477) { + trigger = other1477.trigger; + __isset = other1477.__isset; return *this; } void WMCreateTriggerRequest::printTo(std::ostream& out) const { @@ -40886,11 +41014,11 @@ void swap(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b) { (void) b; } -WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other1466) noexcept { - (void) other1466; +WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other1478) noexcept { + (void) other1478; } -WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other1467) noexcept { - (void) other1467; +WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other1479) noexcept { + (void) other1479; return *this; } void WMCreateTriggerResponse::printTo(std::ostream& out) const { @@ -40977,13 +41105,13 @@ void swap(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b) { swap(a.__isset, b.__isset); } -WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other1468) { - trigger = other1468.trigger; - __isset = other1468.__isset; +WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other1480) { + trigger = other1480.trigger; + __isset = other1480.__isset; } -WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other1469) { - trigger = other1469.trigger; - __isset = other1469.__isset; +WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other1481) { + trigger = other1481.trigger; + __isset = other1481.__isset; return *this; } void WMAlterTriggerRequest::printTo(std::ostream& out) const { @@ -41048,11 +41176,11 @@ void swap(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b) { (void) b; } -WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other1470) noexcept { - (void) other1470; +WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other1482) noexcept { + (void) other1482; } -WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other1471) noexcept { - (void) other1471; +WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other1483) noexcept { + (void) other1483; return *this; } void WMAlterTriggerResponse::printTo(std::ostream& out) const { @@ -41177,17 +41305,17 @@ void swap(WMDropTriggerRequest &a, WMDropTriggerRequest &b) { swap(a.__isset, b.__isset); } -WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other1472) { - resourcePlanName = other1472.resourcePlanName; - triggerName = other1472.triggerName; - ns = other1472.ns; - __isset = other1472.__isset; +WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other1484) { + resourcePlanName = other1484.resourcePlanName; + triggerName = other1484.triggerName; + ns = other1484.ns; + __isset = other1484.__isset; } -WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other1473) { - resourcePlanName = other1473.resourcePlanName; - triggerName = other1473.triggerName; - ns = other1473.ns; - __isset = other1473.__isset; +WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other1485) { + resourcePlanName = other1485.resourcePlanName; + triggerName = other1485.triggerName; + ns = other1485.ns; + __isset = other1485.__isset; return *this; } void WMDropTriggerRequest::printTo(std::ostream& out) const { @@ -41254,11 +41382,11 @@ void swap(WMDropTriggerResponse &a, WMDropTriggerResponse &b) { (void) b; } -WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other1474) noexcept { - (void) other1474; +WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other1486) noexcept { + (void) other1486; } -WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other1475) noexcept { - (void) other1475; +WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other1487) noexcept { + (void) other1487; return *this; } void WMDropTriggerResponse::printTo(std::ostream& out) const { @@ -41364,15 +41492,15 @@ void swap(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequ swap(a.__isset, b.__isset); } -WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other1476) { - resourcePlanName = other1476.resourcePlanName; - ns = other1476.ns; - __isset = other1476.__isset; +WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other1488) { + resourcePlanName = other1488.resourcePlanName; + ns = other1488.ns; + __isset = other1488.__isset; } -WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other1477) { - resourcePlanName = other1477.resourcePlanName; - ns = other1477.ns; - __isset = other1477.__isset; +WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other1489) { + resourcePlanName = other1489.resourcePlanName; + ns = other1489.ns; + __isset = other1489.__isset; return *this; } void WMGetTriggersForResourePlanRequest::printTo(std::ostream& out) const { @@ -41424,14 +41552,14 @@ uint32_t WMGetTriggersForResourePlanResponse::read(::apache::thrift::protocol::T if (ftype == ::apache::thrift::protocol::T_LIST) { { this->triggers.clear(); - uint32_t _size1478; - ::apache::thrift::protocol::TType _etype1481; - xfer += iprot->readListBegin(_etype1481, _size1478); - this->triggers.resize(_size1478); - uint32_t _i1482; - for (_i1482 = 0; _i1482 < _size1478; ++_i1482) + uint32_t _size1490; + ::apache::thrift::protocol::TType _etype1493; + xfer += iprot->readListBegin(_etype1493, _size1490); + this->triggers.resize(_size1490); + uint32_t _i1494; + for (_i1494 = 0; _i1494 < _size1490; ++_i1494) { - xfer += this->triggers[_i1482].read(iprot); + xfer += this->triggers[_i1494].read(iprot); } xfer += iprot->readListEnd(); } @@ -41461,10 +41589,10 @@ uint32_t WMGetTriggersForResourePlanResponse::write(::apache::thrift::protocol:: xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->triggers.size())); - std::vector ::const_iterator _iter1483; - for (_iter1483 = this->triggers.begin(); _iter1483 != this->triggers.end(); ++_iter1483) + std::vector ::const_iterator _iter1495; + for (_iter1495 = this->triggers.begin(); _iter1495 != this->triggers.end(); ++_iter1495) { - xfer += (*_iter1483).write(oprot); + xfer += (*_iter1495).write(oprot); } xfer += oprot->writeListEnd(); } @@ -41481,13 +41609,13 @@ void swap(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanRes swap(a.__isset, b.__isset); } -WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other1484) { - triggers = other1484.triggers; - __isset = other1484.__isset; +WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other1496) { + triggers = other1496.triggers; + __isset = other1496.__isset; } -WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other1485) { - triggers = other1485.triggers; - __isset = other1485.__isset; +WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other1497) { + triggers = other1497.triggers; + __isset = other1497.__isset; return *this; } void WMGetTriggersForResourePlanResponse::printTo(std::ostream& out) const { @@ -41575,13 +41703,13 @@ void swap(WMCreatePoolRequest &a, WMCreatePoolRequest &b) { swap(a.__isset, b.__isset); } -WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other1486) { - pool = other1486.pool; - __isset = other1486.__isset; +WMCreatePoolRequest::WMCreatePoolRequest(const WMCreatePoolRequest& other1498) { + pool = other1498.pool; + __isset = other1498.__isset; } -WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other1487) { - pool = other1487.pool; - __isset = other1487.__isset; +WMCreatePoolRequest& WMCreatePoolRequest::operator=(const WMCreatePoolRequest& other1499) { + pool = other1499.pool; + __isset = other1499.__isset; return *this; } void WMCreatePoolRequest::printTo(std::ostream& out) const { @@ -41646,11 +41774,11 @@ void swap(WMCreatePoolResponse &a, WMCreatePoolResponse &b) { (void) b; } -WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other1488) noexcept { - (void) other1488; +WMCreatePoolResponse::WMCreatePoolResponse(const WMCreatePoolResponse& other1500) noexcept { + (void) other1500; } -WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other1489) noexcept { - (void) other1489; +WMCreatePoolResponse& WMCreatePoolResponse::operator=(const WMCreatePoolResponse& other1501) noexcept { + (void) other1501; return *this; } void WMCreatePoolResponse::printTo(std::ostream& out) const { @@ -41756,15 +41884,15 @@ void swap(WMAlterPoolRequest &a, WMAlterPoolRequest &b) { swap(a.__isset, b.__isset); } -WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other1490) { - pool = other1490.pool; - poolPath = other1490.poolPath; - __isset = other1490.__isset; +WMAlterPoolRequest::WMAlterPoolRequest(const WMAlterPoolRequest& other1502) { + pool = other1502.pool; + poolPath = other1502.poolPath; + __isset = other1502.__isset; } -WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other1491) { - pool = other1491.pool; - poolPath = other1491.poolPath; - __isset = other1491.__isset; +WMAlterPoolRequest& WMAlterPoolRequest::operator=(const WMAlterPoolRequest& other1503) { + pool = other1503.pool; + poolPath = other1503.poolPath; + __isset = other1503.__isset; return *this; } void WMAlterPoolRequest::printTo(std::ostream& out) const { @@ -41830,11 +41958,11 @@ void swap(WMAlterPoolResponse &a, WMAlterPoolResponse &b) { (void) b; } -WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1492) noexcept { - (void) other1492; +WMAlterPoolResponse::WMAlterPoolResponse(const WMAlterPoolResponse& other1504) noexcept { + (void) other1504; } -WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1493) noexcept { - (void) other1493; +WMAlterPoolResponse& WMAlterPoolResponse::operator=(const WMAlterPoolResponse& other1505) noexcept { + (void) other1505; return *this; } void WMAlterPoolResponse::printTo(std::ostream& out) const { @@ -41959,17 +42087,17 @@ void swap(WMDropPoolRequest &a, WMDropPoolRequest &b) { swap(a.__isset, b.__isset); } -WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1494) { - resourcePlanName = other1494.resourcePlanName; - poolPath = other1494.poolPath; - ns = other1494.ns; - __isset = other1494.__isset; +WMDropPoolRequest::WMDropPoolRequest(const WMDropPoolRequest& other1506) { + resourcePlanName = other1506.resourcePlanName; + poolPath = other1506.poolPath; + ns = other1506.ns; + __isset = other1506.__isset; } -WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1495) { - resourcePlanName = other1495.resourcePlanName; - poolPath = other1495.poolPath; - ns = other1495.ns; - __isset = other1495.__isset; +WMDropPoolRequest& WMDropPoolRequest::operator=(const WMDropPoolRequest& other1507) { + resourcePlanName = other1507.resourcePlanName; + poolPath = other1507.poolPath; + ns = other1507.ns; + __isset = other1507.__isset; return *this; } void WMDropPoolRequest::printTo(std::ostream& out) const { @@ -42036,11 +42164,11 @@ void swap(WMDropPoolResponse &a, WMDropPoolResponse &b) { (void) b; } -WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1496) noexcept { - (void) other1496; +WMDropPoolResponse::WMDropPoolResponse(const WMDropPoolResponse& other1508) noexcept { + (void) other1508; } -WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1497) noexcept { - (void) other1497; +WMDropPoolResponse& WMDropPoolResponse::operator=(const WMDropPoolResponse& other1509) noexcept { + (void) other1509; return *this; } void WMDropPoolResponse::printTo(std::ostream& out) const { @@ -42146,15 +42274,15 @@ void swap(WMCreateOrUpdateMappingRequest &a, WMCreateOrUpdateMappingRequest &b) swap(a.__isset, b.__isset); } -WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1498) { - mapping = other1498.mapping; - update = other1498.update; - __isset = other1498.__isset; +WMCreateOrUpdateMappingRequest::WMCreateOrUpdateMappingRequest(const WMCreateOrUpdateMappingRequest& other1510) { + mapping = other1510.mapping; + update = other1510.update; + __isset = other1510.__isset; } -WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1499) { - mapping = other1499.mapping; - update = other1499.update; - __isset = other1499.__isset; +WMCreateOrUpdateMappingRequest& WMCreateOrUpdateMappingRequest::operator=(const WMCreateOrUpdateMappingRequest& other1511) { + mapping = other1511.mapping; + update = other1511.update; + __isset = other1511.__isset; return *this; } void WMCreateOrUpdateMappingRequest::printTo(std::ostream& out) const { @@ -42220,11 +42348,11 @@ void swap(WMCreateOrUpdateMappingResponse &a, WMCreateOrUpdateMappingResponse &b (void) b; } -WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1500) noexcept { - (void) other1500; +WMCreateOrUpdateMappingResponse::WMCreateOrUpdateMappingResponse(const WMCreateOrUpdateMappingResponse& other1512) noexcept { + (void) other1512; } -WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1501) noexcept { - (void) other1501; +WMCreateOrUpdateMappingResponse& WMCreateOrUpdateMappingResponse::operator=(const WMCreateOrUpdateMappingResponse& other1513) noexcept { + (void) other1513; return *this; } void WMCreateOrUpdateMappingResponse::printTo(std::ostream& out) const { @@ -42311,13 +42439,13 @@ void swap(WMDropMappingRequest &a, WMDropMappingRequest &b) { swap(a.__isset, b.__isset); } -WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1502) { - mapping = other1502.mapping; - __isset = other1502.__isset; +WMDropMappingRequest::WMDropMappingRequest(const WMDropMappingRequest& other1514) { + mapping = other1514.mapping; + __isset = other1514.__isset; } -WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1503) { - mapping = other1503.mapping; - __isset = other1503.__isset; +WMDropMappingRequest& WMDropMappingRequest::operator=(const WMDropMappingRequest& other1515) { + mapping = other1515.mapping; + __isset = other1515.__isset; return *this; } void WMDropMappingRequest::printTo(std::ostream& out) const { @@ -42382,11 +42510,11 @@ void swap(WMDropMappingResponse &a, WMDropMappingResponse &b) { (void) b; } -WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1504) noexcept { - (void) other1504; +WMDropMappingResponse::WMDropMappingResponse(const WMDropMappingResponse& other1516) noexcept { + (void) other1516; } -WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1505) noexcept { - (void) other1505; +WMDropMappingResponse& WMDropMappingResponse::operator=(const WMDropMappingResponse& other1517) noexcept { + (void) other1517; return *this; } void WMDropMappingResponse::printTo(std::ostream& out) const { @@ -42549,21 +42677,21 @@ void swap(WMCreateOrDropTriggerToPoolMappingRequest &a, WMCreateOrDropTriggerToP swap(a.__isset, b.__isset); } -WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1506) { - resourcePlanName = other1506.resourcePlanName; - triggerName = other1506.triggerName; - poolPath = other1506.poolPath; - drop = other1506.drop; - ns = other1506.ns; - __isset = other1506.__isset; +WMCreateOrDropTriggerToPoolMappingRequest::WMCreateOrDropTriggerToPoolMappingRequest(const WMCreateOrDropTriggerToPoolMappingRequest& other1518) { + resourcePlanName = other1518.resourcePlanName; + triggerName = other1518.triggerName; + poolPath = other1518.poolPath; + drop = other1518.drop; + ns = other1518.ns; + __isset = other1518.__isset; } -WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1507) { - resourcePlanName = other1507.resourcePlanName; - triggerName = other1507.triggerName; - poolPath = other1507.poolPath; - drop = other1507.drop; - ns = other1507.ns; - __isset = other1507.__isset; +WMCreateOrDropTriggerToPoolMappingRequest& WMCreateOrDropTriggerToPoolMappingRequest::operator=(const WMCreateOrDropTriggerToPoolMappingRequest& other1519) { + resourcePlanName = other1519.resourcePlanName; + triggerName = other1519.triggerName; + poolPath = other1519.poolPath; + drop = other1519.drop; + ns = other1519.ns; + __isset = other1519.__isset; return *this; } void WMCreateOrDropTriggerToPoolMappingRequest::printTo(std::ostream& out) const { @@ -42632,11 +42760,11 @@ void swap(WMCreateOrDropTriggerToPoolMappingResponse &a, WMCreateOrDropTriggerTo (void) b; } -WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1508) noexcept { - (void) other1508; +WMCreateOrDropTriggerToPoolMappingResponse::WMCreateOrDropTriggerToPoolMappingResponse(const WMCreateOrDropTriggerToPoolMappingResponse& other1520) noexcept { + (void) other1520; } -WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1509) noexcept { - (void) other1509; +WMCreateOrDropTriggerToPoolMappingResponse& WMCreateOrDropTriggerToPoolMappingResponse::operator=(const WMCreateOrDropTriggerToPoolMappingResponse& other1521) noexcept { + (void) other1521; return *this; } void WMCreateOrDropTriggerToPoolMappingResponse::printTo(std::ostream& out) const { @@ -42717,9 +42845,9 @@ uint32_t ISchema::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1510; - xfer += iprot->readI32(ecast1510); - this->schemaType = static_cast(ecast1510); + int32_t ecast1522; + xfer += iprot->readI32(ecast1522); + this->schemaType = static_cast(ecast1522); this->__isset.schemaType = true; } else { xfer += iprot->skip(ftype); @@ -42751,9 +42879,9 @@ uint32_t ISchema::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 5: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1511; - xfer += iprot->readI32(ecast1511); - this->compatibility = static_cast(ecast1511); + int32_t ecast1523; + xfer += iprot->readI32(ecast1523); + this->compatibility = static_cast(ecast1523); this->__isset.compatibility = true; } else { xfer += iprot->skip(ftype); @@ -42761,9 +42889,9 @@ uint32_t ISchema::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 6: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1512; - xfer += iprot->readI32(ecast1512); - this->validationLevel = static_cast(ecast1512); + int32_t ecast1524; + xfer += iprot->readI32(ecast1524); + this->validationLevel = static_cast(ecast1524); this->__isset.validationLevel = true; } else { xfer += iprot->skip(ftype); @@ -42867,29 +42995,29 @@ void swap(ISchema &a, ISchema &b) { swap(a.__isset, b.__isset); } -ISchema::ISchema(const ISchema& other1513) { - schemaType = other1513.schemaType; - name = other1513.name; - catName = other1513.catName; - dbName = other1513.dbName; - compatibility = other1513.compatibility; - validationLevel = other1513.validationLevel; - canEvolve = other1513.canEvolve; - schemaGroup = other1513.schemaGroup; - description = other1513.description; - __isset = other1513.__isset; -} -ISchema& ISchema::operator=(const ISchema& other1514) { - schemaType = other1514.schemaType; - name = other1514.name; - catName = other1514.catName; - dbName = other1514.dbName; - compatibility = other1514.compatibility; - validationLevel = other1514.validationLevel; - canEvolve = other1514.canEvolve; - schemaGroup = other1514.schemaGroup; - description = other1514.description; - __isset = other1514.__isset; +ISchema::ISchema(const ISchema& other1525) { + schemaType = other1525.schemaType; + name = other1525.name; + catName = other1525.catName; + dbName = other1525.dbName; + compatibility = other1525.compatibility; + validationLevel = other1525.validationLevel; + canEvolve = other1525.canEvolve; + schemaGroup = other1525.schemaGroup; + description = other1525.description; + __isset = other1525.__isset; +} +ISchema& ISchema::operator=(const ISchema& other1526) { + schemaType = other1526.schemaType; + name = other1526.name; + catName = other1526.catName; + dbName = other1526.dbName; + compatibility = other1526.compatibility; + validationLevel = other1526.validationLevel; + canEvolve = other1526.canEvolve; + schemaGroup = other1526.schemaGroup; + description = other1526.description; + __isset = other1526.__isset; return *this; } void ISchema::printTo(std::ostream& out) const { @@ -43017,17 +43145,17 @@ void swap(ISchemaName &a, ISchemaName &b) { swap(a.__isset, b.__isset); } -ISchemaName::ISchemaName(const ISchemaName& other1515) { - catName = other1515.catName; - dbName = other1515.dbName; - schemaName = other1515.schemaName; - __isset = other1515.__isset; +ISchemaName::ISchemaName(const ISchemaName& other1527) { + catName = other1527.catName; + dbName = other1527.dbName; + schemaName = other1527.schemaName; + __isset = other1527.__isset; } -ISchemaName& ISchemaName::operator=(const ISchemaName& other1516) { - catName = other1516.catName; - dbName = other1516.dbName; - schemaName = other1516.schemaName; - __isset = other1516.__isset; +ISchemaName& ISchemaName::operator=(const ISchemaName& other1528) { + catName = other1528.catName; + dbName = other1528.dbName; + schemaName = other1528.schemaName; + __isset = other1528.__isset; return *this; } void ISchemaName::printTo(std::ostream& out) const { @@ -43132,15 +43260,15 @@ void swap(AlterISchemaRequest &a, AlterISchemaRequest &b) { swap(a.__isset, b.__isset); } -AlterISchemaRequest::AlterISchemaRequest(const AlterISchemaRequest& other1517) { - name = other1517.name; - newSchema = other1517.newSchema; - __isset = other1517.__isset; +AlterISchemaRequest::AlterISchemaRequest(const AlterISchemaRequest& other1529) { + name = other1529.name; + newSchema = other1529.newSchema; + __isset = other1529.__isset; } -AlterISchemaRequest& AlterISchemaRequest::operator=(const AlterISchemaRequest& other1518) { - name = other1518.name; - newSchema = other1518.newSchema; - __isset = other1518.__isset; +AlterISchemaRequest& AlterISchemaRequest::operator=(const AlterISchemaRequest& other1530) { + name = other1530.name; + newSchema = other1530.newSchema; + __isset = other1530.__isset; return *this; } void AlterISchemaRequest::printTo(std::ostream& out) const { @@ -43257,14 +43385,14 @@ uint32_t SchemaVersion::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->cols.clear(); - uint32_t _size1519; - ::apache::thrift::protocol::TType _etype1522; - xfer += iprot->readListBegin(_etype1522, _size1519); - this->cols.resize(_size1519); - uint32_t _i1523; - for (_i1523 = 0; _i1523 < _size1519; ++_i1523) + uint32_t _size1531; + ::apache::thrift::protocol::TType _etype1534; + xfer += iprot->readListBegin(_etype1534, _size1531); + this->cols.resize(_size1531); + uint32_t _i1535; + for (_i1535 = 0; _i1535 < _size1531; ++_i1535) { - xfer += this->cols[_i1523].read(iprot); + xfer += this->cols[_i1535].read(iprot); } xfer += iprot->readListEnd(); } @@ -43275,9 +43403,9 @@ uint32_t SchemaVersion::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 5: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1524; - xfer += iprot->readI32(ecast1524); - this->state = static_cast(ecast1524); + int32_t ecast1536; + xfer += iprot->readI32(ecast1536); + this->state = static_cast(ecast1536); this->__isset.state = true; } else { xfer += iprot->skip(ftype); @@ -43355,10 +43483,10 @@ uint32_t SchemaVersion::write(::apache::thrift::protocol::TProtocol* oprot) cons xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->cols.size())); - std::vector ::const_iterator _iter1525; - for (_iter1525 = this->cols.begin(); _iter1525 != this->cols.end(); ++_iter1525) + std::vector ::const_iterator _iter1537; + for (_iter1537 = this->cols.begin(); _iter1537 != this->cols.end(); ++_iter1537) { - xfer += (*_iter1525).write(oprot); + xfer += (*_iter1537).write(oprot); } xfer += oprot->writeListEnd(); } @@ -43414,31 +43542,31 @@ void swap(SchemaVersion &a, SchemaVersion &b) { swap(a.__isset, b.__isset); } -SchemaVersion::SchemaVersion(const SchemaVersion& other1526) { - schema = other1526.schema; - version = other1526.version; - createdAt = other1526.createdAt; - cols = other1526.cols; - state = other1526.state; - description = other1526.description; - schemaText = other1526.schemaText; - fingerprint = other1526.fingerprint; - name = other1526.name; - serDe = other1526.serDe; - __isset = other1526.__isset; +SchemaVersion::SchemaVersion(const SchemaVersion& other1538) { + schema = other1538.schema; + version = other1538.version; + createdAt = other1538.createdAt; + cols = other1538.cols; + state = other1538.state; + description = other1538.description; + schemaText = other1538.schemaText; + fingerprint = other1538.fingerprint; + name = other1538.name; + serDe = other1538.serDe; + __isset = other1538.__isset; } -SchemaVersion& SchemaVersion::operator=(const SchemaVersion& other1527) { - schema = other1527.schema; - version = other1527.version; - createdAt = other1527.createdAt; - cols = other1527.cols; - state = other1527.state; - description = other1527.description; - schemaText = other1527.schemaText; - fingerprint = other1527.fingerprint; - name = other1527.name; - serDe = other1527.serDe; - __isset = other1527.__isset; +SchemaVersion& SchemaVersion::operator=(const SchemaVersion& other1539) { + schema = other1539.schema; + version = other1539.version; + createdAt = other1539.createdAt; + cols = other1539.cols; + state = other1539.state; + description = other1539.description; + schemaText = other1539.schemaText; + fingerprint = other1539.fingerprint; + name = other1539.name; + serDe = other1539.serDe; + __isset = other1539.__isset; return *this; } void SchemaVersion::printTo(std::ostream& out) const { @@ -43550,15 +43678,15 @@ void swap(SchemaVersionDescriptor &a, SchemaVersionDescriptor &b) { swap(a.__isset, b.__isset); } -SchemaVersionDescriptor::SchemaVersionDescriptor(const SchemaVersionDescriptor& other1528) { - schema = other1528.schema; - version = other1528.version; - __isset = other1528.__isset; +SchemaVersionDescriptor::SchemaVersionDescriptor(const SchemaVersionDescriptor& other1540) { + schema = other1540.schema; + version = other1540.version; + __isset = other1540.__isset; } -SchemaVersionDescriptor& SchemaVersionDescriptor::operator=(const SchemaVersionDescriptor& other1529) { - schema = other1529.schema; - version = other1529.version; - __isset = other1529.__isset; +SchemaVersionDescriptor& SchemaVersionDescriptor::operator=(const SchemaVersionDescriptor& other1541) { + schema = other1541.schema; + version = other1541.version; + __isset = other1541.__isset; return *this; } void SchemaVersionDescriptor::printTo(std::ostream& out) const { @@ -43685,17 +43813,17 @@ void swap(FindSchemasByColsRqst &a, FindSchemasByColsRqst &b) { swap(a.__isset, b.__isset); } -FindSchemasByColsRqst::FindSchemasByColsRqst(const FindSchemasByColsRqst& other1530) { - colName = other1530.colName; - colNamespace = other1530.colNamespace; - type = other1530.type; - __isset = other1530.__isset; +FindSchemasByColsRqst::FindSchemasByColsRqst(const FindSchemasByColsRqst& other1542) { + colName = other1542.colName; + colNamespace = other1542.colNamespace; + type = other1542.type; + __isset = other1542.__isset; } -FindSchemasByColsRqst& FindSchemasByColsRqst::operator=(const FindSchemasByColsRqst& other1531) { - colName = other1531.colName; - colNamespace = other1531.colNamespace; - type = other1531.type; - __isset = other1531.__isset; +FindSchemasByColsRqst& FindSchemasByColsRqst::operator=(const FindSchemasByColsRqst& other1543) { + colName = other1543.colName; + colNamespace = other1543.colNamespace; + type = other1543.type; + __isset = other1543.__isset; return *this; } void FindSchemasByColsRqst::printTo(std::ostream& out) const { @@ -43747,14 +43875,14 @@ uint32_t FindSchemasByColsResp::read(::apache::thrift::protocol::TProtocol* ipro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->schemaVersions.clear(); - uint32_t _size1532; - ::apache::thrift::protocol::TType _etype1535; - xfer += iprot->readListBegin(_etype1535, _size1532); - this->schemaVersions.resize(_size1532); - uint32_t _i1536; - for (_i1536 = 0; _i1536 < _size1532; ++_i1536) + uint32_t _size1544; + ::apache::thrift::protocol::TType _etype1547; + xfer += iprot->readListBegin(_etype1547, _size1544); + this->schemaVersions.resize(_size1544); + uint32_t _i1548; + for (_i1548 = 0; _i1548 < _size1544; ++_i1548) { - xfer += this->schemaVersions[_i1536].read(iprot); + xfer += this->schemaVersions[_i1548].read(iprot); } xfer += iprot->readListEnd(); } @@ -43783,10 +43911,10 @@ uint32_t FindSchemasByColsResp::write(::apache::thrift::protocol::TProtocol* opr xfer += oprot->writeFieldBegin("schemaVersions", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->schemaVersions.size())); - std::vector ::const_iterator _iter1537; - for (_iter1537 = this->schemaVersions.begin(); _iter1537 != this->schemaVersions.end(); ++_iter1537) + std::vector ::const_iterator _iter1549; + for (_iter1549 = this->schemaVersions.begin(); _iter1549 != this->schemaVersions.end(); ++_iter1549) { - xfer += (*_iter1537).write(oprot); + xfer += (*_iter1549).write(oprot); } xfer += oprot->writeListEnd(); } @@ -43803,13 +43931,13 @@ void swap(FindSchemasByColsResp &a, FindSchemasByColsResp &b) { swap(a.__isset, b.__isset); } -FindSchemasByColsResp::FindSchemasByColsResp(const FindSchemasByColsResp& other1538) { - schemaVersions = other1538.schemaVersions; - __isset = other1538.__isset; +FindSchemasByColsResp::FindSchemasByColsResp(const FindSchemasByColsResp& other1550) { + schemaVersions = other1550.schemaVersions; + __isset = other1550.__isset; } -FindSchemasByColsResp& FindSchemasByColsResp::operator=(const FindSchemasByColsResp& other1539) { - schemaVersions = other1539.schemaVersions; - __isset = other1539.__isset; +FindSchemasByColsResp& FindSchemasByColsResp::operator=(const FindSchemasByColsResp& other1551) { + schemaVersions = other1551.schemaVersions; + __isset = other1551.__isset; return *this; } void FindSchemasByColsResp::printTo(std::ostream& out) const { @@ -43912,15 +44040,15 @@ void swap(MapSchemaVersionToSerdeRequest &a, MapSchemaVersionToSerdeRequest &b) swap(a.__isset, b.__isset); } -MapSchemaVersionToSerdeRequest::MapSchemaVersionToSerdeRequest(const MapSchemaVersionToSerdeRequest& other1540) { - schemaVersion = other1540.schemaVersion; - serdeName = other1540.serdeName; - __isset = other1540.__isset; +MapSchemaVersionToSerdeRequest::MapSchemaVersionToSerdeRequest(const MapSchemaVersionToSerdeRequest& other1552) { + schemaVersion = other1552.schemaVersion; + serdeName = other1552.serdeName; + __isset = other1552.__isset; } -MapSchemaVersionToSerdeRequest& MapSchemaVersionToSerdeRequest::operator=(const MapSchemaVersionToSerdeRequest& other1541) { - schemaVersion = other1541.schemaVersion; - serdeName = other1541.serdeName; - __isset = other1541.__isset; +MapSchemaVersionToSerdeRequest& MapSchemaVersionToSerdeRequest::operator=(const MapSchemaVersionToSerdeRequest& other1553) { + schemaVersion = other1553.schemaVersion; + serdeName = other1553.serdeName; + __isset = other1553.__isset; return *this; } void MapSchemaVersionToSerdeRequest::printTo(std::ostream& out) const { @@ -43981,9 +44109,9 @@ uint32_t SetSchemaVersionStateRequest::read(::apache::thrift::protocol::TProtoco break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1542; - xfer += iprot->readI32(ecast1542); - this->state = static_cast(ecast1542); + int32_t ecast1554; + xfer += iprot->readI32(ecast1554); + this->state = static_cast(ecast1554); this->__isset.state = true; } else { xfer += iprot->skip(ftype); @@ -44026,15 +44154,15 @@ void swap(SetSchemaVersionStateRequest &a, SetSchemaVersionStateRequest &b) { swap(a.__isset, b.__isset); } -SetSchemaVersionStateRequest::SetSchemaVersionStateRequest(const SetSchemaVersionStateRequest& other1543) { - schemaVersion = other1543.schemaVersion; - state = other1543.state; - __isset = other1543.__isset; +SetSchemaVersionStateRequest::SetSchemaVersionStateRequest(const SetSchemaVersionStateRequest& other1555) { + schemaVersion = other1555.schemaVersion; + state = other1555.state; + __isset = other1555.__isset; } -SetSchemaVersionStateRequest& SetSchemaVersionStateRequest::operator=(const SetSchemaVersionStateRequest& other1544) { - schemaVersion = other1544.schemaVersion; - state = other1544.state; - __isset = other1544.__isset; +SetSchemaVersionStateRequest& SetSchemaVersionStateRequest::operator=(const SetSchemaVersionStateRequest& other1556) { + schemaVersion = other1556.schemaVersion; + state = other1556.state; + __isset = other1556.__isset; return *this; } void SetSchemaVersionStateRequest::printTo(std::ostream& out) const { @@ -44121,13 +44249,13 @@ void swap(GetSerdeRequest &a, GetSerdeRequest &b) { swap(a.__isset, b.__isset); } -GetSerdeRequest::GetSerdeRequest(const GetSerdeRequest& other1545) { - serdeName = other1545.serdeName; - __isset = other1545.__isset; +GetSerdeRequest::GetSerdeRequest(const GetSerdeRequest& other1557) { + serdeName = other1557.serdeName; + __isset = other1557.__isset; } -GetSerdeRequest& GetSerdeRequest::operator=(const GetSerdeRequest& other1546) { - serdeName = other1546.serdeName; - __isset = other1546.__isset; +GetSerdeRequest& GetSerdeRequest::operator=(const GetSerdeRequest& other1558) { + serdeName = other1558.serdeName; + __isset = other1558.__isset; return *this; } void GetSerdeRequest::printTo(std::ostream& out) const { @@ -44255,17 +44383,17 @@ void swap(RuntimeStat &a, RuntimeStat &b) { swap(a.__isset, b.__isset); } -RuntimeStat::RuntimeStat(const RuntimeStat& other1547) { - createTime = other1547.createTime; - weight = other1547.weight; - payload = other1547.payload; - __isset = other1547.__isset; +RuntimeStat::RuntimeStat(const RuntimeStat& other1559) { + createTime = other1559.createTime; + weight = other1559.weight; + payload = other1559.payload; + __isset = other1559.__isset; } -RuntimeStat& RuntimeStat::operator=(const RuntimeStat& other1548) { - createTime = other1548.createTime; - weight = other1548.weight; - payload = other1548.payload; - __isset = other1548.__isset; +RuntimeStat& RuntimeStat::operator=(const RuntimeStat& other1560) { + createTime = other1560.createTime; + weight = other1560.weight; + payload = other1560.payload; + __isset = other1560.__isset; return *this; } void RuntimeStat::printTo(std::ostream& out) const { @@ -44375,13 +44503,13 @@ void swap(GetRuntimeStatsRequest &a, GetRuntimeStatsRequest &b) { swap(a.maxCreateTime, b.maxCreateTime); } -GetRuntimeStatsRequest::GetRuntimeStatsRequest(const GetRuntimeStatsRequest& other1549) noexcept { - maxWeight = other1549.maxWeight; - maxCreateTime = other1549.maxCreateTime; +GetRuntimeStatsRequest::GetRuntimeStatsRequest(const GetRuntimeStatsRequest& other1561) noexcept { + maxWeight = other1561.maxWeight; + maxCreateTime = other1561.maxCreateTime; } -GetRuntimeStatsRequest& GetRuntimeStatsRequest::operator=(const GetRuntimeStatsRequest& other1550) noexcept { - maxWeight = other1550.maxWeight; - maxCreateTime = other1550.maxCreateTime; +GetRuntimeStatsRequest& GetRuntimeStatsRequest::operator=(const GetRuntimeStatsRequest& other1562) noexcept { + maxWeight = other1562.maxWeight; + maxCreateTime = other1562.maxCreateTime; return *this; } void GetRuntimeStatsRequest::printTo(std::ostream& out) const { @@ -44494,14 +44622,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->primaryKeys.clear(); - uint32_t _size1551; - ::apache::thrift::protocol::TType _etype1554; - xfer += iprot->readListBegin(_etype1554, _size1551); - this->primaryKeys.resize(_size1551); - uint32_t _i1555; - for (_i1555 = 0; _i1555 < _size1551; ++_i1555) + uint32_t _size1563; + ::apache::thrift::protocol::TType _etype1566; + xfer += iprot->readListBegin(_etype1566, _size1563); + this->primaryKeys.resize(_size1563); + uint32_t _i1567; + for (_i1567 = 0; _i1567 < _size1563; ++_i1567) { - xfer += this->primaryKeys[_i1555].read(iprot); + xfer += this->primaryKeys[_i1567].read(iprot); } xfer += iprot->readListEnd(); } @@ -44514,14 +44642,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->foreignKeys.clear(); - uint32_t _size1556; - ::apache::thrift::protocol::TType _etype1559; - xfer += iprot->readListBegin(_etype1559, _size1556); - this->foreignKeys.resize(_size1556); - uint32_t _i1560; - for (_i1560 = 0; _i1560 < _size1556; ++_i1560) + uint32_t _size1568; + ::apache::thrift::protocol::TType _etype1571; + xfer += iprot->readListBegin(_etype1571, _size1568); + this->foreignKeys.resize(_size1568); + uint32_t _i1572; + for (_i1572 = 0; _i1572 < _size1568; ++_i1572) { - xfer += this->foreignKeys[_i1560].read(iprot); + xfer += this->foreignKeys[_i1572].read(iprot); } xfer += iprot->readListEnd(); } @@ -44534,14 +44662,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->uniqueConstraints.clear(); - uint32_t _size1561; - ::apache::thrift::protocol::TType _etype1564; - xfer += iprot->readListBegin(_etype1564, _size1561); - this->uniqueConstraints.resize(_size1561); - uint32_t _i1565; - for (_i1565 = 0; _i1565 < _size1561; ++_i1565) + uint32_t _size1573; + ::apache::thrift::protocol::TType _etype1576; + xfer += iprot->readListBegin(_etype1576, _size1573); + this->uniqueConstraints.resize(_size1573); + uint32_t _i1577; + for (_i1577 = 0; _i1577 < _size1573; ++_i1577) { - xfer += this->uniqueConstraints[_i1565].read(iprot); + xfer += this->uniqueConstraints[_i1577].read(iprot); } xfer += iprot->readListEnd(); } @@ -44554,14 +44682,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->notNullConstraints.clear(); - uint32_t _size1566; - ::apache::thrift::protocol::TType _etype1569; - xfer += iprot->readListBegin(_etype1569, _size1566); - this->notNullConstraints.resize(_size1566); - uint32_t _i1570; - for (_i1570 = 0; _i1570 < _size1566; ++_i1570) + uint32_t _size1578; + ::apache::thrift::protocol::TType _etype1581; + xfer += iprot->readListBegin(_etype1581, _size1578); + this->notNullConstraints.resize(_size1578); + uint32_t _i1582; + for (_i1582 = 0; _i1582 < _size1578; ++_i1582) { - xfer += this->notNullConstraints[_i1570].read(iprot); + xfer += this->notNullConstraints[_i1582].read(iprot); } xfer += iprot->readListEnd(); } @@ -44574,14 +44702,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->defaultConstraints.clear(); - uint32_t _size1571; - ::apache::thrift::protocol::TType _etype1574; - xfer += iprot->readListBegin(_etype1574, _size1571); - this->defaultConstraints.resize(_size1571); - uint32_t _i1575; - for (_i1575 = 0; _i1575 < _size1571; ++_i1575) + uint32_t _size1583; + ::apache::thrift::protocol::TType _etype1586; + xfer += iprot->readListBegin(_etype1586, _size1583); + this->defaultConstraints.resize(_size1583); + uint32_t _i1587; + for (_i1587 = 0; _i1587 < _size1583; ++_i1587) { - xfer += this->defaultConstraints[_i1575].read(iprot); + xfer += this->defaultConstraints[_i1587].read(iprot); } xfer += iprot->readListEnd(); } @@ -44594,14 +44722,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->checkConstraints.clear(); - uint32_t _size1576; - ::apache::thrift::protocol::TType _etype1579; - xfer += iprot->readListBegin(_etype1579, _size1576); - this->checkConstraints.resize(_size1576); - uint32_t _i1580; - for (_i1580 = 0; _i1580 < _size1576; ++_i1580) + uint32_t _size1588; + ::apache::thrift::protocol::TType _etype1591; + xfer += iprot->readListBegin(_etype1591, _size1588); + this->checkConstraints.resize(_size1588); + uint32_t _i1592; + for (_i1592 = 0; _i1592 < _size1588; ++_i1592) { - xfer += this->checkConstraints[_i1580].read(iprot); + xfer += this->checkConstraints[_i1592].read(iprot); } xfer += iprot->readListEnd(); } @@ -44614,14 +44742,14 @@ uint32_t CreateTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1581; - ::apache::thrift::protocol::TType _etype1584; - xfer += iprot->readListBegin(_etype1584, _size1581); - this->processorCapabilities.resize(_size1581); - uint32_t _i1585; - for (_i1585 = 0; _i1585 < _size1581; ++_i1585) + uint32_t _size1593; + ::apache::thrift::protocol::TType _etype1596; + xfer += iprot->readListBegin(_etype1596, _size1593); + this->processorCapabilities.resize(_size1593); + uint32_t _i1597; + for (_i1597 = 0; _i1597 < _size1593; ++_i1597) { - xfer += iprot->readString(this->processorCapabilities[_i1585]); + xfer += iprot->readString(this->processorCapabilities[_i1597]); } xfer += iprot->readListEnd(); } @@ -44670,10 +44798,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); - std::vector ::const_iterator _iter1586; - for (_iter1586 = this->primaryKeys.begin(); _iter1586 != this->primaryKeys.end(); ++_iter1586) + std::vector ::const_iterator _iter1598; + for (_iter1598 = this->primaryKeys.begin(); _iter1598 != this->primaryKeys.end(); ++_iter1598) { - xfer += (*_iter1586).write(oprot); + xfer += (*_iter1598).write(oprot); } xfer += oprot->writeListEnd(); } @@ -44683,10 +44811,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); - std::vector ::const_iterator _iter1587; - for (_iter1587 = this->foreignKeys.begin(); _iter1587 != this->foreignKeys.end(); ++_iter1587) + std::vector ::const_iterator _iter1599; + for (_iter1599 = this->foreignKeys.begin(); _iter1599 != this->foreignKeys.end(); ++_iter1599) { - xfer += (*_iter1587).write(oprot); + xfer += (*_iter1599).write(oprot); } xfer += oprot->writeListEnd(); } @@ -44696,10 +44824,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); - std::vector ::const_iterator _iter1588; - for (_iter1588 = this->uniqueConstraints.begin(); _iter1588 != this->uniqueConstraints.end(); ++_iter1588) + std::vector ::const_iterator _iter1600; + for (_iter1600 = this->uniqueConstraints.begin(); _iter1600 != this->uniqueConstraints.end(); ++_iter1600) { - xfer += (*_iter1588).write(oprot); + xfer += (*_iter1600).write(oprot); } xfer += oprot->writeListEnd(); } @@ -44709,10 +44837,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); - std::vector ::const_iterator _iter1589; - for (_iter1589 = this->notNullConstraints.begin(); _iter1589 != this->notNullConstraints.end(); ++_iter1589) + std::vector ::const_iterator _iter1601; + for (_iter1601 = this->notNullConstraints.begin(); _iter1601 != this->notNullConstraints.end(); ++_iter1601) { - xfer += (*_iter1589).write(oprot); + xfer += (*_iter1601).write(oprot); } xfer += oprot->writeListEnd(); } @@ -44722,10 +44850,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->defaultConstraints.size())); - std::vector ::const_iterator _iter1590; - for (_iter1590 = this->defaultConstraints.begin(); _iter1590 != this->defaultConstraints.end(); ++_iter1590) + std::vector ::const_iterator _iter1602; + for (_iter1602 = this->defaultConstraints.begin(); _iter1602 != this->defaultConstraints.end(); ++_iter1602) { - xfer += (*_iter1590).write(oprot); + xfer += (*_iter1602).write(oprot); } xfer += oprot->writeListEnd(); } @@ -44735,10 +44863,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 8); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->checkConstraints.size())); - std::vector ::const_iterator _iter1591; - for (_iter1591 = this->checkConstraints.begin(); _iter1591 != this->checkConstraints.end(); ++_iter1591) + std::vector ::const_iterator _iter1603; + for (_iter1603 = this->checkConstraints.begin(); _iter1603 != this->checkConstraints.end(); ++_iter1603) { - xfer += (*_iter1591).write(oprot); + xfer += (*_iter1603).write(oprot); } xfer += oprot->writeListEnd(); } @@ -44748,10 +44876,10 @@ uint32_t CreateTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 9); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1592; - for (_iter1592 = this->processorCapabilities.begin(); _iter1592 != this->processorCapabilities.end(); ++_iter1592) + std::vector ::const_iterator _iter1604; + for (_iter1604 = this->processorCapabilities.begin(); _iter1604 != this->processorCapabilities.end(); ++_iter1604) { - xfer += oprot->writeString((*_iter1592)); + xfer += oprot->writeString((*_iter1604)); } xfer += oprot->writeListEnd(); } @@ -44782,31 +44910,31 @@ void swap(CreateTableRequest &a, CreateTableRequest &b) { swap(a.__isset, b.__isset); } -CreateTableRequest::CreateTableRequest(const CreateTableRequest& other1593) { - table = other1593.table; - envContext = other1593.envContext; - primaryKeys = other1593.primaryKeys; - foreignKeys = other1593.foreignKeys; - uniqueConstraints = other1593.uniqueConstraints; - notNullConstraints = other1593.notNullConstraints; - defaultConstraints = other1593.defaultConstraints; - checkConstraints = other1593.checkConstraints; - processorCapabilities = other1593.processorCapabilities; - processorIdentifier = other1593.processorIdentifier; - __isset = other1593.__isset; -} -CreateTableRequest& CreateTableRequest::operator=(const CreateTableRequest& other1594) { - table = other1594.table; - envContext = other1594.envContext; - primaryKeys = other1594.primaryKeys; - foreignKeys = other1594.foreignKeys; - uniqueConstraints = other1594.uniqueConstraints; - notNullConstraints = other1594.notNullConstraints; - defaultConstraints = other1594.defaultConstraints; - checkConstraints = other1594.checkConstraints; - processorCapabilities = other1594.processorCapabilities; - processorIdentifier = other1594.processorIdentifier; - __isset = other1594.__isset; +CreateTableRequest::CreateTableRequest(const CreateTableRequest& other1605) { + table = other1605.table; + envContext = other1605.envContext; + primaryKeys = other1605.primaryKeys; + foreignKeys = other1605.foreignKeys; + uniqueConstraints = other1605.uniqueConstraints; + notNullConstraints = other1605.notNullConstraints; + defaultConstraints = other1605.defaultConstraints; + checkConstraints = other1605.checkConstraints; + processorCapabilities = other1605.processorCapabilities; + processorIdentifier = other1605.processorIdentifier; + __isset = other1605.__isset; +} +CreateTableRequest& CreateTableRequest::operator=(const CreateTableRequest& other1606) { + table = other1606.table; + envContext = other1606.envContext; + primaryKeys = other1606.primaryKeys; + foreignKeys = other1606.foreignKeys; + uniqueConstraints = other1606.uniqueConstraints; + notNullConstraints = other1606.notNullConstraints; + defaultConstraints = other1606.defaultConstraints; + checkConstraints = other1606.checkConstraints; + processorCapabilities = other1606.processorCapabilities; + processorIdentifier = other1606.processorIdentifier; + __isset = other1606.__isset; return *this; } void CreateTableRequest::printTo(std::ostream& out) const { @@ -44945,17 +45073,17 @@ uint32_t CreateDatabaseRequest::read(::apache::thrift::protocol::TProtocol* ipro if (ftype == ::apache::thrift::protocol::T_MAP) { { this->parameters.clear(); - uint32_t _size1595; - ::apache::thrift::protocol::TType _ktype1596; - ::apache::thrift::protocol::TType _vtype1597; - xfer += iprot->readMapBegin(_ktype1596, _vtype1597, _size1595); - uint32_t _i1599; - for (_i1599 = 0; _i1599 < _size1595; ++_i1599) + uint32_t _size1607; + ::apache::thrift::protocol::TType _ktype1608; + ::apache::thrift::protocol::TType _vtype1609; + xfer += iprot->readMapBegin(_ktype1608, _vtype1609, _size1607); + uint32_t _i1611; + for (_i1611 = 0; _i1611 < _size1607; ++_i1611) { - std::string _key1600; - xfer += iprot->readString(_key1600); - std::string& _val1601 = this->parameters[_key1600]; - xfer += iprot->readString(_val1601); + std::string _key1612; + xfer += iprot->readString(_key1612); + std::string& _val1613 = this->parameters[_key1612]; + xfer += iprot->readString(_val1613); } xfer += iprot->readMapEnd(); } @@ -44982,9 +45110,9 @@ uint32_t CreateDatabaseRequest::read(::apache::thrift::protocol::TProtocol* ipro break; case 7: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1602; - xfer += iprot->readI32(ecast1602); - this->ownerType = static_cast(ecast1602); + int32_t ecast1614; + xfer += iprot->readI32(ecast1614); + this->ownerType = static_cast(ecast1614); this->__isset.ownerType = true; } else { xfer += iprot->skip(ftype); @@ -45067,11 +45195,11 @@ uint32_t CreateDatabaseRequest::write(::apache::thrift::protocol::TProtocol* opr xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 4); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter1603; - for (_iter1603 = this->parameters.begin(); _iter1603 != this->parameters.end(); ++_iter1603) + std::map ::const_iterator _iter1615; + for (_iter1615 = this->parameters.begin(); _iter1615 != this->parameters.end(); ++_iter1615) { - xfer += oprot->writeString(_iter1603->first); - xfer += oprot->writeString(_iter1603->second); + xfer += oprot->writeString(_iter1615->first); + xfer += oprot->writeString(_iter1615->second); } xfer += oprot->writeMapEnd(); } @@ -45139,35 +45267,35 @@ void swap(CreateDatabaseRequest &a, CreateDatabaseRequest &b) { swap(a.__isset, b.__isset); } -CreateDatabaseRequest::CreateDatabaseRequest(const CreateDatabaseRequest& other1604) { - databaseName = other1604.databaseName; - description = other1604.description; - locationUri = other1604.locationUri; - parameters = other1604.parameters; - privileges = other1604.privileges; - ownerName = other1604.ownerName; - ownerType = other1604.ownerType; - catalogName = other1604.catalogName; - createTime = other1604.createTime; - managedLocationUri = other1604.managedLocationUri; - type = other1604.type; - dataConnectorName = other1604.dataConnectorName; - __isset = other1604.__isset; -} -CreateDatabaseRequest& CreateDatabaseRequest::operator=(const CreateDatabaseRequest& other1605) { - databaseName = other1605.databaseName; - description = other1605.description; - locationUri = other1605.locationUri; - parameters = other1605.parameters; - privileges = other1605.privileges; - ownerName = other1605.ownerName; - ownerType = other1605.ownerType; - catalogName = other1605.catalogName; - createTime = other1605.createTime; - managedLocationUri = other1605.managedLocationUri; - type = other1605.type; - dataConnectorName = other1605.dataConnectorName; - __isset = other1605.__isset; +CreateDatabaseRequest::CreateDatabaseRequest(const CreateDatabaseRequest& other1616) { + databaseName = other1616.databaseName; + description = other1616.description; + locationUri = other1616.locationUri; + parameters = other1616.parameters; + privileges = other1616.privileges; + ownerName = other1616.ownerName; + ownerType = other1616.ownerType; + catalogName = other1616.catalogName; + createTime = other1616.createTime; + managedLocationUri = other1616.managedLocationUri; + type = other1616.type; + dataConnectorName = other1616.dataConnectorName; + __isset = other1616.__isset; +} +CreateDatabaseRequest& CreateDatabaseRequest::operator=(const CreateDatabaseRequest& other1617) { + databaseName = other1617.databaseName; + description = other1617.description; + locationUri = other1617.locationUri; + parameters = other1617.parameters; + privileges = other1617.privileges; + ownerName = other1617.ownerName; + ownerType = other1617.ownerType; + catalogName = other1617.catalogName; + createTime = other1617.createTime; + managedLocationUri = other1617.managedLocationUri; + type = other1617.type; + dataConnectorName = other1617.dataConnectorName; + __isset = other1617.__isset; return *this; } void CreateDatabaseRequest::printTo(std::ostream& out) const { @@ -45264,13 +45392,13 @@ void swap(CreateDataConnectorRequest &a, CreateDataConnectorRequest &b) { swap(a.__isset, b.__isset); } -CreateDataConnectorRequest::CreateDataConnectorRequest(const CreateDataConnectorRequest& other1606) { - connector = other1606.connector; - __isset = other1606.__isset; +CreateDataConnectorRequest::CreateDataConnectorRequest(const CreateDataConnectorRequest& other1618) { + connector = other1618.connector; + __isset = other1618.__isset; } -CreateDataConnectorRequest& CreateDataConnectorRequest::operator=(const CreateDataConnectorRequest& other1607) { - connector = other1607.connector; - __isset = other1607.__isset; +CreateDataConnectorRequest& CreateDataConnectorRequest::operator=(const CreateDataConnectorRequest& other1619) { + connector = other1619.connector; + __isset = other1619.__isset; return *this; } void CreateDataConnectorRequest::printTo(std::ostream& out) const { @@ -45358,11 +45486,11 @@ void swap(GetDataConnectorRequest &a, GetDataConnectorRequest &b) { swap(a.connectorName, b.connectorName); } -GetDataConnectorRequest::GetDataConnectorRequest(const GetDataConnectorRequest& other1608) { - connectorName = other1608.connectorName; +GetDataConnectorRequest::GetDataConnectorRequest(const GetDataConnectorRequest& other1620) { + connectorName = other1620.connectorName; } -GetDataConnectorRequest& GetDataConnectorRequest::operator=(const GetDataConnectorRequest& other1609) { - connectorName = other1609.connectorName; +GetDataConnectorRequest& GetDataConnectorRequest::operator=(const GetDataConnectorRequest& other1621) { + connectorName = other1621.connectorName; return *this; } void GetDataConnectorRequest::printTo(std::ostream& out) const { @@ -45450,11 +45578,11 @@ void swap(ScheduledQueryPollRequest &a, ScheduledQueryPollRequest &b) { swap(a.clusterNamespace, b.clusterNamespace); } -ScheduledQueryPollRequest::ScheduledQueryPollRequest(const ScheduledQueryPollRequest& other1610) { - clusterNamespace = other1610.clusterNamespace; +ScheduledQueryPollRequest::ScheduledQueryPollRequest(const ScheduledQueryPollRequest& other1622) { + clusterNamespace = other1622.clusterNamespace; } -ScheduledQueryPollRequest& ScheduledQueryPollRequest::operator=(const ScheduledQueryPollRequest& other1611) { - clusterNamespace = other1611.clusterNamespace; +ScheduledQueryPollRequest& ScheduledQueryPollRequest::operator=(const ScheduledQueryPollRequest& other1623) { + clusterNamespace = other1623.clusterNamespace; return *this; } void ScheduledQueryPollRequest::printTo(std::ostream& out) const { @@ -45562,13 +45690,13 @@ void swap(ScheduledQueryKey &a, ScheduledQueryKey &b) { swap(a.clusterNamespace, b.clusterNamespace); } -ScheduledQueryKey::ScheduledQueryKey(const ScheduledQueryKey& other1612) { - scheduleName = other1612.scheduleName; - clusterNamespace = other1612.clusterNamespace; +ScheduledQueryKey::ScheduledQueryKey(const ScheduledQueryKey& other1624) { + scheduleName = other1624.scheduleName; + clusterNamespace = other1624.clusterNamespace; } -ScheduledQueryKey& ScheduledQueryKey::operator=(const ScheduledQueryKey& other1613) { - scheduleName = other1613.scheduleName; - clusterNamespace = other1613.clusterNamespace; +ScheduledQueryKey& ScheduledQueryKey::operator=(const ScheduledQueryKey& other1625) { + scheduleName = other1625.scheduleName; + clusterNamespace = other1625.clusterNamespace; return *this; } void ScheduledQueryKey::printTo(std::ostream& out) const { @@ -45714,19 +45842,19 @@ void swap(ScheduledQueryPollResponse &a, ScheduledQueryPollResponse &b) { swap(a.__isset, b.__isset); } -ScheduledQueryPollResponse::ScheduledQueryPollResponse(const ScheduledQueryPollResponse& other1614) { - scheduleKey = other1614.scheduleKey; - executionId = other1614.executionId; - query = other1614.query; - user = other1614.user; - __isset = other1614.__isset; +ScheduledQueryPollResponse::ScheduledQueryPollResponse(const ScheduledQueryPollResponse& other1626) { + scheduleKey = other1626.scheduleKey; + executionId = other1626.executionId; + query = other1626.query; + user = other1626.user; + __isset = other1626.__isset; } -ScheduledQueryPollResponse& ScheduledQueryPollResponse::operator=(const ScheduledQueryPollResponse& other1615) { - scheduleKey = other1615.scheduleKey; - executionId = other1615.executionId; - query = other1615.query; - user = other1615.user; - __isset = other1615.__isset; +ScheduledQueryPollResponse& ScheduledQueryPollResponse::operator=(const ScheduledQueryPollResponse& other1627) { + scheduleKey = other1627.scheduleKey; + executionId = other1627.executionId; + query = other1627.query; + user = other1627.user; + __isset = other1627.__isset; return *this; } void ScheduledQueryPollResponse::printTo(std::ostream& out) const { @@ -45913,23 +46041,23 @@ void swap(ScheduledQuery &a, ScheduledQuery &b) { swap(a.__isset, b.__isset); } -ScheduledQuery::ScheduledQuery(const ScheduledQuery& other1616) { - scheduleKey = other1616.scheduleKey; - enabled = other1616.enabled; - schedule = other1616.schedule; - user = other1616.user; - query = other1616.query; - nextExecution = other1616.nextExecution; - __isset = other1616.__isset; +ScheduledQuery::ScheduledQuery(const ScheduledQuery& other1628) { + scheduleKey = other1628.scheduleKey; + enabled = other1628.enabled; + schedule = other1628.schedule; + user = other1628.user; + query = other1628.query; + nextExecution = other1628.nextExecution; + __isset = other1628.__isset; } -ScheduledQuery& ScheduledQuery::operator=(const ScheduledQuery& other1617) { - scheduleKey = other1617.scheduleKey; - enabled = other1617.enabled; - schedule = other1617.schedule; - user = other1617.user; - query = other1617.query; - nextExecution = other1617.nextExecution; - __isset = other1617.__isset; +ScheduledQuery& ScheduledQuery::operator=(const ScheduledQuery& other1629) { + scheduleKey = other1629.scheduleKey; + enabled = other1629.enabled; + schedule = other1629.schedule; + user = other1629.user; + query = other1629.query; + nextExecution = other1629.nextExecution; + __isset = other1629.__isset; return *this; } void ScheduledQuery::printTo(std::ostream& out) const { @@ -45988,9 +46116,9 @@ uint32_t ScheduledQueryMaintenanceRequest::read(::apache::thrift::protocol::TPro { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1618; - xfer += iprot->readI32(ecast1618); - this->type = static_cast(ecast1618); + int32_t ecast1630; + xfer += iprot->readI32(ecast1630); + this->type = static_cast(ecast1630); isset_type = true; } else { xfer += iprot->skip(ftype); @@ -46044,13 +46172,13 @@ void swap(ScheduledQueryMaintenanceRequest &a, ScheduledQueryMaintenanceRequest swap(a.scheduledQuery, b.scheduledQuery); } -ScheduledQueryMaintenanceRequest::ScheduledQueryMaintenanceRequest(const ScheduledQueryMaintenanceRequest& other1619) { - type = other1619.type; - scheduledQuery = other1619.scheduledQuery; +ScheduledQueryMaintenanceRequest::ScheduledQueryMaintenanceRequest(const ScheduledQueryMaintenanceRequest& other1631) { + type = other1631.type; + scheduledQuery = other1631.scheduledQuery; } -ScheduledQueryMaintenanceRequest& ScheduledQueryMaintenanceRequest::operator=(const ScheduledQueryMaintenanceRequest& other1620) { - type = other1620.type; - scheduledQuery = other1620.scheduledQuery; +ScheduledQueryMaintenanceRequest& ScheduledQueryMaintenanceRequest::operator=(const ScheduledQueryMaintenanceRequest& other1632) { + type = other1632.type; + scheduledQuery = other1632.scheduledQuery; return *this; } void ScheduledQueryMaintenanceRequest::printTo(std::ostream& out) const { @@ -46123,9 +46251,9 @@ uint32_t ScheduledQueryProgressInfo::read(::apache::thrift::protocol::TProtocol* break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1621; - xfer += iprot->readI32(ecast1621); - this->state = static_cast(ecast1621); + int32_t ecast1633; + xfer += iprot->readI32(ecast1633); + this->state = static_cast(ecast1633); isset_state = true; } else { xfer += iprot->skip(ftype); @@ -46201,19 +46329,19 @@ void swap(ScheduledQueryProgressInfo &a, ScheduledQueryProgressInfo &b) { swap(a.__isset, b.__isset); } -ScheduledQueryProgressInfo::ScheduledQueryProgressInfo(const ScheduledQueryProgressInfo& other1622) { - scheduledExecutionId = other1622.scheduledExecutionId; - state = other1622.state; - executorQueryId = other1622.executorQueryId; - errorMessage = other1622.errorMessage; - __isset = other1622.__isset; +ScheduledQueryProgressInfo::ScheduledQueryProgressInfo(const ScheduledQueryProgressInfo& other1634) { + scheduledExecutionId = other1634.scheduledExecutionId; + state = other1634.state; + executorQueryId = other1634.executorQueryId; + errorMessage = other1634.errorMessage; + __isset = other1634.__isset; } -ScheduledQueryProgressInfo& ScheduledQueryProgressInfo::operator=(const ScheduledQueryProgressInfo& other1623) { - scheduledExecutionId = other1623.scheduledExecutionId; - state = other1623.state; - executorQueryId = other1623.executorQueryId; - errorMessage = other1623.errorMessage; - __isset = other1623.__isset; +ScheduledQueryProgressInfo& ScheduledQueryProgressInfo::operator=(const ScheduledQueryProgressInfo& other1635) { + scheduledExecutionId = other1635.scheduledExecutionId; + state = other1635.state; + executorQueryId = other1635.executorQueryId; + errorMessage = other1635.errorMessage; + __isset = other1635.__isset; return *this; } void ScheduledQueryProgressInfo::printTo(std::ostream& out) const { @@ -46331,14 +46459,14 @@ uint32_t AlterPartitionsRequest::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitions.clear(); - uint32_t _size1624; - ::apache::thrift::protocol::TType _etype1627; - xfer += iprot->readListBegin(_etype1627, _size1624); - this->partitions.resize(_size1624); - uint32_t _i1628; - for (_i1628 = 0; _i1628 < _size1624; ++_i1628) + uint32_t _size1636; + ::apache::thrift::protocol::TType _etype1639; + xfer += iprot->readListBegin(_etype1639, _size1636); + this->partitions.resize(_size1636); + uint32_t _i1640; + for (_i1640 = 0; _i1640 < _size1636; ++_i1640) { - xfer += this->partitions[_i1628].read(iprot); + xfer += this->partitions[_i1640].read(iprot); } xfer += iprot->readListEnd(); } @@ -46383,14 +46511,14 @@ uint32_t AlterPartitionsRequest::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitionColSchema.clear(); - uint32_t _size1629; - ::apache::thrift::protocol::TType _etype1632; - xfer += iprot->readListBegin(_etype1632, _size1629); - this->partitionColSchema.resize(_size1629); - uint32_t _i1633; - for (_i1633 = 0; _i1633 < _size1629; ++_i1633) + uint32_t _size1641; + ::apache::thrift::protocol::TType _etype1644; + xfer += iprot->readListBegin(_etype1644, _size1641); + this->partitionColSchema.resize(_size1641); + uint32_t _i1645; + for (_i1645 = 0; _i1645 < _size1641; ++_i1645) { - xfer += this->partitionColSchema[_i1633].read(iprot); + xfer += this->partitionColSchema[_i1645].read(iprot); } xfer += iprot->readListEnd(); } @@ -46438,10 +46566,10 @@ uint32_t AlterPartitionsRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter1634; - for (_iter1634 = this->partitions.begin(); _iter1634 != this->partitions.end(); ++_iter1634) + std::vector ::const_iterator _iter1646; + for (_iter1646 = this->partitions.begin(); _iter1646 != this->partitions.end(); ++_iter1646) { - xfer += (*_iter1634).write(oprot); + xfer += (*_iter1646).write(oprot); } xfer += oprot->writeListEnd(); } @@ -46471,10 +46599,10 @@ uint32_t AlterPartitionsRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("partitionColSchema", ::apache::thrift::protocol::T_LIST, 9); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionColSchema.size())); - std::vector ::const_iterator _iter1635; - for (_iter1635 = this->partitionColSchema.begin(); _iter1635 != this->partitionColSchema.end(); ++_iter1635) + std::vector ::const_iterator _iter1647; + for (_iter1647 = this->partitionColSchema.begin(); _iter1647 != this->partitionColSchema.end(); ++_iter1647) { - xfer += (*_iter1635).write(oprot); + xfer += (*_iter1647).write(oprot); } xfer += oprot->writeListEnd(); } @@ -46499,29 +46627,29 @@ void swap(AlterPartitionsRequest &a, AlterPartitionsRequest &b) { swap(a.__isset, b.__isset); } -AlterPartitionsRequest::AlterPartitionsRequest(const AlterPartitionsRequest& other1636) { - catName = other1636.catName; - dbName = other1636.dbName; - tableName = other1636.tableName; - partitions = other1636.partitions; - environmentContext = other1636.environmentContext; - writeId = other1636.writeId; - validWriteIdList = other1636.validWriteIdList; - skipColumnSchemaForPartition = other1636.skipColumnSchemaForPartition; - partitionColSchema = other1636.partitionColSchema; - __isset = other1636.__isset; -} -AlterPartitionsRequest& AlterPartitionsRequest::operator=(const AlterPartitionsRequest& other1637) { - catName = other1637.catName; - dbName = other1637.dbName; - tableName = other1637.tableName; - partitions = other1637.partitions; - environmentContext = other1637.environmentContext; - writeId = other1637.writeId; - validWriteIdList = other1637.validWriteIdList; - skipColumnSchemaForPartition = other1637.skipColumnSchemaForPartition; - partitionColSchema = other1637.partitionColSchema; - __isset = other1637.__isset; +AlterPartitionsRequest::AlterPartitionsRequest(const AlterPartitionsRequest& other1648) { + catName = other1648.catName; + dbName = other1648.dbName; + tableName = other1648.tableName; + partitions = other1648.partitions; + environmentContext = other1648.environmentContext; + writeId = other1648.writeId; + validWriteIdList = other1648.validWriteIdList; + skipColumnSchemaForPartition = other1648.skipColumnSchemaForPartition; + partitionColSchema = other1648.partitionColSchema; + __isset = other1648.__isset; +} +AlterPartitionsRequest& AlterPartitionsRequest::operator=(const AlterPartitionsRequest& other1649) { + catName = other1649.catName; + dbName = other1649.dbName; + tableName = other1649.tableName; + partitions = other1649.partitions; + environmentContext = other1649.environmentContext; + writeId = other1649.writeId; + validWriteIdList = other1649.validWriteIdList; + skipColumnSchemaForPartition = other1649.skipColumnSchemaForPartition; + partitionColSchema = other1649.partitionColSchema; + __isset = other1649.__isset; return *this; } void AlterPartitionsRequest::printTo(std::ostream& out) const { @@ -46594,11 +46722,11 @@ void swap(AlterPartitionsResponse &a, AlterPartitionsResponse &b) { (void) b; } -AlterPartitionsResponse::AlterPartitionsResponse(const AlterPartitionsResponse& other1638) noexcept { - (void) other1638; +AlterPartitionsResponse::AlterPartitionsResponse(const AlterPartitionsResponse& other1650) noexcept { + (void) other1650; } -AlterPartitionsResponse& AlterPartitionsResponse::operator=(const AlterPartitionsResponse& other1639) noexcept { - (void) other1639; +AlterPartitionsResponse& AlterPartitionsResponse::operator=(const AlterPartitionsResponse& other1651) noexcept { + (void) other1651; return *this; } void AlterPartitionsResponse::printTo(std::ostream& out) const { @@ -46707,14 +46835,14 @@ uint32_t RenamePartitionRequest::read(::apache::thrift::protocol::TProtocol* ipr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partVals.clear(); - uint32_t _size1640; - ::apache::thrift::protocol::TType _etype1643; - xfer += iprot->readListBegin(_etype1643, _size1640); - this->partVals.resize(_size1640); - uint32_t _i1644; - for (_i1644 = 0; _i1644 < _size1640; ++_i1644) + uint32_t _size1652; + ::apache::thrift::protocol::TType _etype1655; + xfer += iprot->readListBegin(_etype1655, _size1652); + this->partVals.resize(_size1652); + uint32_t _i1656; + for (_i1656 = 0; _i1656 < _size1652; ++_i1656) { - xfer += iprot->readString(this->partVals[_i1644]); + xfer += iprot->readString(this->partVals[_i1656]); } xfer += iprot->readListEnd(); } @@ -46796,10 +46924,10 @@ uint32_t RenamePartitionRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeFieldBegin("partVals", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partVals.size())); - std::vector ::const_iterator _iter1645; - for (_iter1645 = this->partVals.begin(); _iter1645 != this->partVals.end(); ++_iter1645) + std::vector ::const_iterator _iter1657; + for (_iter1657 = this->partVals.begin(); _iter1657 != this->partVals.end(); ++_iter1657) { - xfer += oprot->writeString((*_iter1645)); + xfer += oprot->writeString((*_iter1657)); } xfer += oprot->writeListEnd(); } @@ -46842,27 +46970,27 @@ void swap(RenamePartitionRequest &a, RenamePartitionRequest &b) { swap(a.__isset, b.__isset); } -RenamePartitionRequest::RenamePartitionRequest(const RenamePartitionRequest& other1646) { - catName = other1646.catName; - dbName = other1646.dbName; - tableName = other1646.tableName; - partVals = other1646.partVals; - newPart = other1646.newPart; - validWriteIdList = other1646.validWriteIdList; - txnId = other1646.txnId; - clonePart = other1646.clonePart; - __isset = other1646.__isset; -} -RenamePartitionRequest& RenamePartitionRequest::operator=(const RenamePartitionRequest& other1647) { - catName = other1647.catName; - dbName = other1647.dbName; - tableName = other1647.tableName; - partVals = other1647.partVals; - newPart = other1647.newPart; - validWriteIdList = other1647.validWriteIdList; - txnId = other1647.txnId; - clonePart = other1647.clonePart; - __isset = other1647.__isset; +RenamePartitionRequest::RenamePartitionRequest(const RenamePartitionRequest& other1658) { + catName = other1658.catName; + dbName = other1658.dbName; + tableName = other1658.tableName; + partVals = other1658.partVals; + newPart = other1658.newPart; + validWriteIdList = other1658.validWriteIdList; + txnId = other1658.txnId; + clonePart = other1658.clonePart; + __isset = other1658.__isset; +} +RenamePartitionRequest& RenamePartitionRequest::operator=(const RenamePartitionRequest& other1659) { + catName = other1659.catName; + dbName = other1659.dbName; + tableName = other1659.tableName; + partVals = other1659.partVals; + newPart = other1659.newPart; + validWriteIdList = other1659.validWriteIdList; + txnId = other1659.txnId; + clonePart = other1659.clonePart; + __isset = other1659.__isset; return *this; } void RenamePartitionRequest::printTo(std::ostream& out) const { @@ -46934,11 +47062,11 @@ void swap(RenamePartitionResponse &a, RenamePartitionResponse &b) { (void) b; } -RenamePartitionResponse::RenamePartitionResponse(const RenamePartitionResponse& other1648) noexcept { - (void) other1648; +RenamePartitionResponse::RenamePartitionResponse(const RenamePartitionResponse& other1660) noexcept { + (void) other1660; } -RenamePartitionResponse& RenamePartitionResponse::operator=(const RenamePartitionResponse& other1649) noexcept { - (void) other1649; +RenamePartitionResponse& RenamePartitionResponse::operator=(const RenamePartitionResponse& other1661) noexcept { + (void) other1661; return *this; } void RenamePartitionResponse::printTo(std::ostream& out) const { @@ -47094,14 +47222,14 @@ uint32_t AlterTableRequest::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1650; - ::apache::thrift::protocol::TType _etype1653; - xfer += iprot->readListBegin(_etype1653, _size1650); - this->processorCapabilities.resize(_size1650); - uint32_t _i1654; - for (_i1654 = 0; _i1654 < _size1650; ++_i1654) + uint32_t _size1662; + ::apache::thrift::protocol::TType _etype1665; + xfer += iprot->readListBegin(_etype1665, _size1662); + this->processorCapabilities.resize(_size1662); + uint32_t _i1666; + for (_i1666 = 0; _i1666 < _size1662; ++_i1666) { - xfer += iprot->readString(this->processorCapabilities[_i1654]); + xfer += iprot->readString(this->processorCapabilities[_i1666]); } xfer += iprot->readListEnd(); } @@ -47193,10 +47321,10 @@ uint32_t AlterTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 8); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1655; - for (_iter1655 = this->processorCapabilities.begin(); _iter1655 != this->processorCapabilities.end(); ++_iter1655) + std::vector ::const_iterator _iter1667; + for (_iter1667 = this->processorCapabilities.begin(); _iter1667 != this->processorCapabilities.end(); ++_iter1667) { - xfer += oprot->writeString((*_iter1655)); + xfer += oprot->writeString((*_iter1667)); } xfer += oprot->writeListEnd(); } @@ -47238,33 +47366,33 @@ void swap(AlterTableRequest &a, AlterTableRequest &b) { swap(a.__isset, b.__isset); } -AlterTableRequest::AlterTableRequest(const AlterTableRequest& other1656) { - catName = other1656.catName; - dbName = other1656.dbName; - tableName = other1656.tableName; - table = other1656.table; - environmentContext = other1656.environmentContext; - writeId = other1656.writeId; - validWriteIdList = other1656.validWriteIdList; - processorCapabilities = other1656.processorCapabilities; - processorIdentifier = other1656.processorIdentifier; - expectedParameterKey = other1656.expectedParameterKey; - expectedParameterValue = other1656.expectedParameterValue; - __isset = other1656.__isset; -} -AlterTableRequest& AlterTableRequest::operator=(const AlterTableRequest& other1657) { - catName = other1657.catName; - dbName = other1657.dbName; - tableName = other1657.tableName; - table = other1657.table; - environmentContext = other1657.environmentContext; - writeId = other1657.writeId; - validWriteIdList = other1657.validWriteIdList; - processorCapabilities = other1657.processorCapabilities; - processorIdentifier = other1657.processorIdentifier; - expectedParameterKey = other1657.expectedParameterKey; - expectedParameterValue = other1657.expectedParameterValue; - __isset = other1657.__isset; +AlterTableRequest::AlterTableRequest(const AlterTableRequest& other1668) { + catName = other1668.catName; + dbName = other1668.dbName; + tableName = other1668.tableName; + table = other1668.table; + environmentContext = other1668.environmentContext; + writeId = other1668.writeId; + validWriteIdList = other1668.validWriteIdList; + processorCapabilities = other1668.processorCapabilities; + processorIdentifier = other1668.processorIdentifier; + expectedParameterKey = other1668.expectedParameterKey; + expectedParameterValue = other1668.expectedParameterValue; + __isset = other1668.__isset; +} +AlterTableRequest& AlterTableRequest::operator=(const AlterTableRequest& other1669) { + catName = other1669.catName; + dbName = other1669.dbName; + tableName = other1669.tableName; + table = other1669.table; + environmentContext = other1669.environmentContext; + writeId = other1669.writeId; + validWriteIdList = other1669.validWriteIdList; + processorCapabilities = other1669.processorCapabilities; + processorIdentifier = other1669.processorIdentifier; + expectedParameterKey = other1669.expectedParameterKey; + expectedParameterValue = other1669.expectedParameterValue; + __isset = other1669.__isset; return *this; } void AlterTableRequest::printTo(std::ostream& out) const { @@ -47339,11 +47467,11 @@ void swap(AlterTableResponse &a, AlterTableResponse &b) { (void) b; } -AlterTableResponse::AlterTableResponse(const AlterTableResponse& other1658) noexcept { - (void) other1658; +AlterTableResponse::AlterTableResponse(const AlterTableResponse& other1670) noexcept { + (void) other1670; } -AlterTableResponse& AlterTableResponse::operator=(const AlterTableResponse& other1659) noexcept { - (void) other1659; +AlterTableResponse& AlterTableResponse::operator=(const AlterTableResponse& other1671) noexcept { + (void) other1671; return *this; } void AlterTableResponse::printTo(std::ostream& out) const { @@ -47396,9 +47524,9 @@ uint32_t GetPartitionsFilterSpec::read(::apache::thrift::protocol::TProtocol* ip { case 7: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1660; - xfer += iprot->readI32(ecast1660); - this->filterMode = static_cast(ecast1660); + int32_t ecast1672; + xfer += iprot->readI32(ecast1672); + this->filterMode = static_cast(ecast1672); this->__isset.filterMode = true; } else { xfer += iprot->skip(ftype); @@ -47408,14 +47536,14 @@ uint32_t GetPartitionsFilterSpec::read(::apache::thrift::protocol::TProtocol* ip if (ftype == ::apache::thrift::protocol::T_LIST) { { this->filters.clear(); - uint32_t _size1661; - ::apache::thrift::protocol::TType _etype1664; - xfer += iprot->readListBegin(_etype1664, _size1661); - this->filters.resize(_size1661); - uint32_t _i1665; - for (_i1665 = 0; _i1665 < _size1661; ++_i1665) + uint32_t _size1673; + ::apache::thrift::protocol::TType _etype1676; + xfer += iprot->readListBegin(_etype1676, _size1673); + this->filters.resize(_size1673); + uint32_t _i1677; + for (_i1677 = 0; _i1677 < _size1673; ++_i1677) { - xfer += iprot->readString(this->filters[_i1665]); + xfer += iprot->readString(this->filters[_i1677]); } xfer += iprot->readListEnd(); } @@ -47450,10 +47578,10 @@ uint32_t GetPartitionsFilterSpec::write(::apache::thrift::protocol::TProtocol* o xfer += oprot->writeFieldBegin("filters", ::apache::thrift::protocol::T_LIST, 8); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filters.size())); - std::vector ::const_iterator _iter1666; - for (_iter1666 = this->filters.begin(); _iter1666 != this->filters.end(); ++_iter1666) + std::vector ::const_iterator _iter1678; + for (_iter1678 = this->filters.begin(); _iter1678 != this->filters.end(); ++_iter1678) { - xfer += oprot->writeString((*_iter1666)); + xfer += oprot->writeString((*_iter1678)); } xfer += oprot->writeListEnd(); } @@ -47471,15 +47599,15 @@ void swap(GetPartitionsFilterSpec &a, GetPartitionsFilterSpec &b) { swap(a.__isset, b.__isset); } -GetPartitionsFilterSpec::GetPartitionsFilterSpec(const GetPartitionsFilterSpec& other1667) { - filterMode = other1667.filterMode; - filters = other1667.filters; - __isset = other1667.__isset; +GetPartitionsFilterSpec::GetPartitionsFilterSpec(const GetPartitionsFilterSpec& other1679) { + filterMode = other1679.filterMode; + filters = other1679.filters; + __isset = other1679.__isset; } -GetPartitionsFilterSpec& GetPartitionsFilterSpec::operator=(const GetPartitionsFilterSpec& other1668) { - filterMode = other1668.filterMode; - filters = other1668.filters; - __isset = other1668.__isset; +GetPartitionsFilterSpec& GetPartitionsFilterSpec::operator=(const GetPartitionsFilterSpec& other1680) { + filterMode = other1680.filterMode; + filters = other1680.filters; + __isset = other1680.__isset; return *this; } void GetPartitionsFilterSpec::printTo(std::ostream& out) const { @@ -47530,14 +47658,14 @@ uint32_t GetPartitionsResponse::read(::apache::thrift::protocol::TProtocol* ipro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitionSpec.clear(); - uint32_t _size1669; - ::apache::thrift::protocol::TType _etype1672; - xfer += iprot->readListBegin(_etype1672, _size1669); - this->partitionSpec.resize(_size1669); - uint32_t _i1673; - for (_i1673 = 0; _i1673 < _size1669; ++_i1673) + uint32_t _size1681; + ::apache::thrift::protocol::TType _etype1684; + xfer += iprot->readListBegin(_etype1684, _size1681); + this->partitionSpec.resize(_size1681); + uint32_t _i1685; + for (_i1685 = 0; _i1685 < _size1681; ++_i1685) { - xfer += this->partitionSpec[_i1673].read(iprot); + xfer += this->partitionSpec[_i1685].read(iprot); } xfer += iprot->readListEnd(); } @@ -47566,10 +47694,10 @@ uint32_t GetPartitionsResponse::write(::apache::thrift::protocol::TProtocol* opr xfer += oprot->writeFieldBegin("partitionSpec", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionSpec.size())); - std::vector ::const_iterator _iter1674; - for (_iter1674 = this->partitionSpec.begin(); _iter1674 != this->partitionSpec.end(); ++_iter1674) + std::vector ::const_iterator _iter1686; + for (_iter1686 = this->partitionSpec.begin(); _iter1686 != this->partitionSpec.end(); ++_iter1686) { - xfer += (*_iter1674).write(oprot); + xfer += (*_iter1686).write(oprot); } xfer += oprot->writeListEnd(); } @@ -47586,13 +47714,13 @@ void swap(GetPartitionsResponse &a, GetPartitionsResponse &b) { swap(a.__isset, b.__isset); } -GetPartitionsResponse::GetPartitionsResponse(const GetPartitionsResponse& other1675) { - partitionSpec = other1675.partitionSpec; - __isset = other1675.__isset; +GetPartitionsResponse::GetPartitionsResponse(const GetPartitionsResponse& other1687) { + partitionSpec = other1687.partitionSpec; + __isset = other1687.__isset; } -GetPartitionsResponse& GetPartitionsResponse::operator=(const GetPartitionsResponse& other1676) { - partitionSpec = other1676.partitionSpec; - __isset = other1676.__isset; +GetPartitionsResponse& GetPartitionsResponse::operator=(const GetPartitionsResponse& other1688) { + partitionSpec = other1688.partitionSpec; + __isset = other1688.__isset; return *this; } void GetPartitionsResponse::printTo(std::ostream& out) const { @@ -47729,14 +47857,14 @@ uint32_t GetPartitionsRequest::read(::apache::thrift::protocol::TProtocol* iprot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->groupNames.clear(); - uint32_t _size1677; - ::apache::thrift::protocol::TType _etype1680; - xfer += iprot->readListBegin(_etype1680, _size1677); - this->groupNames.resize(_size1677); - uint32_t _i1681; - for (_i1681 = 0; _i1681 < _size1677; ++_i1681) + uint32_t _size1689; + ::apache::thrift::protocol::TType _etype1692; + xfer += iprot->readListBegin(_etype1692, _size1689); + this->groupNames.resize(_size1689); + uint32_t _i1693; + for (_i1693 = 0; _i1693 < _size1689; ++_i1693) { - xfer += iprot->readString(this->groupNames[_i1681]); + xfer += iprot->readString(this->groupNames[_i1693]); } xfer += iprot->readListEnd(); } @@ -47765,14 +47893,14 @@ uint32_t GetPartitionsRequest::read(::apache::thrift::protocol::TProtocol* iprot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->processorCapabilities.clear(); - uint32_t _size1682; - ::apache::thrift::protocol::TType _etype1685; - xfer += iprot->readListBegin(_etype1685, _size1682); - this->processorCapabilities.resize(_size1682); - uint32_t _i1686; - for (_i1686 = 0; _i1686 < _size1682; ++_i1686) + uint32_t _size1694; + ::apache::thrift::protocol::TType _etype1697; + xfer += iprot->readListBegin(_etype1697, _size1694); + this->processorCapabilities.resize(_size1694); + uint32_t _i1698; + for (_i1698 = 0; _i1698 < _size1694; ++_i1698) { - xfer += iprot->readString(this->processorCapabilities[_i1686]); + xfer += iprot->readString(this->processorCapabilities[_i1698]); } xfer += iprot->readListEnd(); } @@ -47841,10 +47969,10 @@ uint32_t GetPartitionsRequest::write(::apache::thrift::protocol::TProtocol* opro xfer += oprot->writeFieldBegin("groupNames", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->groupNames.size())); - std::vector ::const_iterator _iter1687; - for (_iter1687 = this->groupNames.begin(); _iter1687 != this->groupNames.end(); ++_iter1687) + std::vector ::const_iterator _iter1699; + for (_iter1699 = this->groupNames.begin(); _iter1699 != this->groupNames.end(); ++_iter1699) { - xfer += oprot->writeString((*_iter1687)); + xfer += oprot->writeString((*_iter1699)); } xfer += oprot->writeListEnd(); } @@ -47862,10 +47990,10 @@ uint32_t GetPartitionsRequest::write(::apache::thrift::protocol::TProtocol* opro xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 9); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->processorCapabilities.size())); - std::vector ::const_iterator _iter1688; - for (_iter1688 = this->processorCapabilities.begin(); _iter1688 != this->processorCapabilities.end(); ++_iter1688) + std::vector ::const_iterator _iter1700; + for (_iter1700 = this->processorCapabilities.begin(); _iter1700 != this->processorCapabilities.end(); ++_iter1700) { - xfer += oprot->writeString((*_iter1688)); + xfer += oprot->writeString((*_iter1700)); } xfer += oprot->writeListEnd(); } @@ -47902,33 +48030,33 @@ void swap(GetPartitionsRequest &a, GetPartitionsRequest &b) { swap(a.__isset, b.__isset); } -GetPartitionsRequest::GetPartitionsRequest(const GetPartitionsRequest& other1689) { - catName = other1689.catName; - dbName = other1689.dbName; - tblName = other1689.tblName; - withAuth = other1689.withAuth; - user = other1689.user; - groupNames = other1689.groupNames; - projectionSpec = other1689.projectionSpec; - filterSpec = other1689.filterSpec; - processorCapabilities = other1689.processorCapabilities; - processorIdentifier = other1689.processorIdentifier; - validWriteIdList = other1689.validWriteIdList; - __isset = other1689.__isset; -} -GetPartitionsRequest& GetPartitionsRequest::operator=(const GetPartitionsRequest& other1690) { - catName = other1690.catName; - dbName = other1690.dbName; - tblName = other1690.tblName; - withAuth = other1690.withAuth; - user = other1690.user; - groupNames = other1690.groupNames; - projectionSpec = other1690.projectionSpec; - filterSpec = other1690.filterSpec; - processorCapabilities = other1690.processorCapabilities; - processorIdentifier = other1690.processorIdentifier; - validWriteIdList = other1690.validWriteIdList; - __isset = other1690.__isset; +GetPartitionsRequest::GetPartitionsRequest(const GetPartitionsRequest& other1701) { + catName = other1701.catName; + dbName = other1701.dbName; + tblName = other1701.tblName; + withAuth = other1701.withAuth; + user = other1701.user; + groupNames = other1701.groupNames; + projectionSpec = other1701.projectionSpec; + filterSpec = other1701.filterSpec; + processorCapabilities = other1701.processorCapabilities; + processorIdentifier = other1701.processorIdentifier; + validWriteIdList = other1701.validWriteIdList; + __isset = other1701.__isset; +} +GetPartitionsRequest& GetPartitionsRequest::operator=(const GetPartitionsRequest& other1702) { + catName = other1702.catName; + dbName = other1702.dbName; + tblName = other1702.tblName; + withAuth = other1702.withAuth; + user = other1702.user; + groupNames = other1702.groupNames; + projectionSpec = other1702.projectionSpec; + filterSpec = other1702.filterSpec; + processorCapabilities = other1702.processorCapabilities; + processorIdentifier = other1702.processorIdentifier; + validWriteIdList = other1702.validWriteIdList; + __isset = other1702.__isset; return *this; } void GetPartitionsRequest::printTo(std::ostream& out) const { @@ -48123,23 +48251,23 @@ void swap(GetFieldsRequest &a, GetFieldsRequest &b) { swap(a.__isset, b.__isset); } -GetFieldsRequest::GetFieldsRequest(const GetFieldsRequest& other1691) { - catName = other1691.catName; - dbName = other1691.dbName; - tblName = other1691.tblName; - envContext = other1691.envContext; - validWriteIdList = other1691.validWriteIdList; - id = other1691.id; - __isset = other1691.__isset; +GetFieldsRequest::GetFieldsRequest(const GetFieldsRequest& other1703) { + catName = other1703.catName; + dbName = other1703.dbName; + tblName = other1703.tblName; + envContext = other1703.envContext; + validWriteIdList = other1703.validWriteIdList; + id = other1703.id; + __isset = other1703.__isset; } -GetFieldsRequest& GetFieldsRequest::operator=(const GetFieldsRequest& other1692) { - catName = other1692.catName; - dbName = other1692.dbName; - tblName = other1692.tblName; - envContext = other1692.envContext; - validWriteIdList = other1692.validWriteIdList; - id = other1692.id; - __isset = other1692.__isset; +GetFieldsRequest& GetFieldsRequest::operator=(const GetFieldsRequest& other1704) { + catName = other1704.catName; + dbName = other1704.dbName; + tblName = other1704.tblName; + envContext = other1704.envContext; + validWriteIdList = other1704.validWriteIdList; + id = other1704.id; + __isset = other1704.__isset; return *this; } void GetFieldsRequest::printTo(std::ostream& out) const { @@ -48195,14 +48323,14 @@ uint32_t GetFieldsResponse::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fields.clear(); - uint32_t _size1693; - ::apache::thrift::protocol::TType _etype1696; - xfer += iprot->readListBegin(_etype1696, _size1693); - this->fields.resize(_size1693); - uint32_t _i1697; - for (_i1697 = 0; _i1697 < _size1693; ++_i1697) + uint32_t _size1705; + ::apache::thrift::protocol::TType _etype1708; + xfer += iprot->readListBegin(_etype1708, _size1705); + this->fields.resize(_size1705); + uint32_t _i1709; + for (_i1709 = 0; _i1709 < _size1705; ++_i1709) { - xfer += this->fields[_i1697].read(iprot); + xfer += this->fields[_i1709].read(iprot); } xfer += iprot->readListEnd(); } @@ -48233,10 +48361,10 @@ uint32_t GetFieldsResponse::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->fields.size())); - std::vector ::const_iterator _iter1698; - for (_iter1698 = this->fields.begin(); _iter1698 != this->fields.end(); ++_iter1698) + std::vector ::const_iterator _iter1710; + for (_iter1710 = this->fields.begin(); _iter1710 != this->fields.end(); ++_iter1710) { - xfer += (*_iter1698).write(oprot); + xfer += (*_iter1710).write(oprot); } xfer += oprot->writeListEnd(); } @@ -48252,11 +48380,11 @@ void swap(GetFieldsResponse &a, GetFieldsResponse &b) { swap(a.fields, b.fields); } -GetFieldsResponse::GetFieldsResponse(const GetFieldsResponse& other1699) { - fields = other1699.fields; +GetFieldsResponse::GetFieldsResponse(const GetFieldsResponse& other1711) { + fields = other1711.fields; } -GetFieldsResponse& GetFieldsResponse::operator=(const GetFieldsResponse& other1700) { - fields = other1700.fields; +GetFieldsResponse& GetFieldsResponse::operator=(const GetFieldsResponse& other1712) { + fields = other1712.fields; return *this; } void GetFieldsResponse::printTo(std::ostream& out) const { @@ -48441,23 +48569,23 @@ void swap(GetSchemaRequest &a, GetSchemaRequest &b) { swap(a.__isset, b.__isset); } -GetSchemaRequest::GetSchemaRequest(const GetSchemaRequest& other1701) { - catName = other1701.catName; - dbName = other1701.dbName; - tblName = other1701.tblName; - envContext = other1701.envContext; - validWriteIdList = other1701.validWriteIdList; - id = other1701.id; - __isset = other1701.__isset; +GetSchemaRequest::GetSchemaRequest(const GetSchemaRequest& other1713) { + catName = other1713.catName; + dbName = other1713.dbName; + tblName = other1713.tblName; + envContext = other1713.envContext; + validWriteIdList = other1713.validWriteIdList; + id = other1713.id; + __isset = other1713.__isset; } -GetSchemaRequest& GetSchemaRequest::operator=(const GetSchemaRequest& other1702) { - catName = other1702.catName; - dbName = other1702.dbName; - tblName = other1702.tblName; - envContext = other1702.envContext; - validWriteIdList = other1702.validWriteIdList; - id = other1702.id; - __isset = other1702.__isset; +GetSchemaRequest& GetSchemaRequest::operator=(const GetSchemaRequest& other1714) { + catName = other1714.catName; + dbName = other1714.dbName; + tblName = other1714.tblName; + envContext = other1714.envContext; + validWriteIdList = other1714.validWriteIdList; + id = other1714.id; + __isset = other1714.__isset; return *this; } void GetSchemaRequest::printTo(std::ostream& out) const { @@ -48513,14 +48641,14 @@ uint32_t GetSchemaResponse::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->fields.clear(); - uint32_t _size1703; - ::apache::thrift::protocol::TType _etype1706; - xfer += iprot->readListBegin(_etype1706, _size1703); - this->fields.resize(_size1703); - uint32_t _i1707; - for (_i1707 = 0; _i1707 < _size1703; ++_i1707) + uint32_t _size1715; + ::apache::thrift::protocol::TType _etype1718; + xfer += iprot->readListBegin(_etype1718, _size1715); + this->fields.resize(_size1715); + uint32_t _i1719; + for (_i1719 = 0; _i1719 < _size1715; ++_i1719) { - xfer += this->fields[_i1707].read(iprot); + xfer += this->fields[_i1719].read(iprot); } xfer += iprot->readListEnd(); } @@ -48551,10 +48679,10 @@ uint32_t GetSchemaResponse::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->fields.size())); - std::vector ::const_iterator _iter1708; - for (_iter1708 = this->fields.begin(); _iter1708 != this->fields.end(); ++_iter1708) + std::vector ::const_iterator _iter1720; + for (_iter1720 = this->fields.begin(); _iter1720 != this->fields.end(); ++_iter1720) { - xfer += (*_iter1708).write(oprot); + xfer += (*_iter1720).write(oprot); } xfer += oprot->writeListEnd(); } @@ -48570,11 +48698,11 @@ void swap(GetSchemaResponse &a, GetSchemaResponse &b) { swap(a.fields, b.fields); } -GetSchemaResponse::GetSchemaResponse(const GetSchemaResponse& other1709) { - fields = other1709.fields; +GetSchemaResponse::GetSchemaResponse(const GetSchemaResponse& other1721) { + fields = other1721.fields; } -GetSchemaResponse& GetSchemaResponse::operator=(const GetSchemaResponse& other1710) { - fields = other1710.fields; +GetSchemaResponse& GetSchemaResponse::operator=(const GetSchemaResponse& other1722) { + fields = other1722.fields; return *this; } void GetSchemaResponse::printTo(std::ostream& out) const { @@ -48674,14 +48802,14 @@ uint32_t GetPartitionRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partVals.clear(); - uint32_t _size1711; - ::apache::thrift::protocol::TType _etype1714; - xfer += iprot->readListBegin(_etype1714, _size1711); - this->partVals.resize(_size1711); - uint32_t _i1715; - for (_i1715 = 0; _i1715 < _size1711; ++_i1715) + uint32_t _size1723; + ::apache::thrift::protocol::TType _etype1726; + xfer += iprot->readListBegin(_etype1726, _size1723); + this->partVals.resize(_size1723); + uint32_t _i1727; + for (_i1727 = 0; _i1727 < _size1723; ++_i1727) { - xfer += iprot->readString(this->partVals[_i1715]); + xfer += iprot->readString(this->partVals[_i1727]); } xfer += iprot->readListEnd(); } @@ -48745,10 +48873,10 @@ uint32_t GetPartitionRequest::write(::apache::thrift::protocol::TProtocol* oprot xfer += oprot->writeFieldBegin("partVals", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partVals.size())); - std::vector ::const_iterator _iter1716; - for (_iter1716 = this->partVals.begin(); _iter1716 != this->partVals.end(); ++_iter1716) + std::vector ::const_iterator _iter1728; + for (_iter1728 = this->partVals.begin(); _iter1728 != this->partVals.end(); ++_iter1728) { - xfer += oprot->writeString((*_iter1716)); + xfer += oprot->writeString((*_iter1728)); } xfer += oprot->writeListEnd(); } @@ -48780,23 +48908,23 @@ void swap(GetPartitionRequest &a, GetPartitionRequest &b) { swap(a.__isset, b.__isset); } -GetPartitionRequest::GetPartitionRequest(const GetPartitionRequest& other1717) { - catName = other1717.catName; - dbName = other1717.dbName; - tblName = other1717.tblName; - partVals = other1717.partVals; - validWriteIdList = other1717.validWriteIdList; - id = other1717.id; - __isset = other1717.__isset; +GetPartitionRequest::GetPartitionRequest(const GetPartitionRequest& other1729) { + catName = other1729.catName; + dbName = other1729.dbName; + tblName = other1729.tblName; + partVals = other1729.partVals; + validWriteIdList = other1729.validWriteIdList; + id = other1729.id; + __isset = other1729.__isset; } -GetPartitionRequest& GetPartitionRequest::operator=(const GetPartitionRequest& other1718) { - catName = other1718.catName; - dbName = other1718.dbName; - tblName = other1718.tblName; - partVals = other1718.partVals; - validWriteIdList = other1718.validWriteIdList; - id = other1718.id; - __isset = other1718.__isset; +GetPartitionRequest& GetPartitionRequest::operator=(const GetPartitionRequest& other1730) { + catName = other1730.catName; + dbName = other1730.dbName; + tblName = other1730.tblName; + partVals = other1730.partVals; + validWriteIdList = other1730.validWriteIdList; + id = other1730.id; + __isset = other1730.__isset; return *this; } void GetPartitionRequest::printTo(std::ostream& out) const { @@ -48889,11 +49017,11 @@ void swap(GetPartitionResponse &a, GetPartitionResponse &b) { swap(a.partition, b.partition); } -GetPartitionResponse::GetPartitionResponse(const GetPartitionResponse& other1719) { - partition = other1719.partition; +GetPartitionResponse::GetPartitionResponse(const GetPartitionResponse& other1731) { + partition = other1731.partition; } -GetPartitionResponse& GetPartitionResponse::operator=(const GetPartitionResponse& other1720) { - partition = other1720.partition; +GetPartitionResponse& GetPartitionResponse::operator=(const GetPartitionResponse& other1732) { + partition = other1732.partition; return *this; } void GetPartitionResponse::printTo(std::ostream& out) const { @@ -49097,25 +49225,25 @@ void swap(PartitionsRequest &a, PartitionsRequest &b) { swap(a.__isset, b.__isset); } -PartitionsRequest::PartitionsRequest(const PartitionsRequest& other1721) { - catName = other1721.catName; - dbName = other1721.dbName; - tblName = other1721.tblName; - maxParts = other1721.maxParts; - validWriteIdList = other1721.validWriteIdList; - id = other1721.id; - skipColumnSchemaForPartition = other1721.skipColumnSchemaForPartition; - __isset = other1721.__isset; +PartitionsRequest::PartitionsRequest(const PartitionsRequest& other1733) { + catName = other1733.catName; + dbName = other1733.dbName; + tblName = other1733.tblName; + maxParts = other1733.maxParts; + validWriteIdList = other1733.validWriteIdList; + id = other1733.id; + skipColumnSchemaForPartition = other1733.skipColumnSchemaForPartition; + __isset = other1733.__isset; } -PartitionsRequest& PartitionsRequest::operator=(const PartitionsRequest& other1722) { - catName = other1722.catName; - dbName = other1722.dbName; - tblName = other1722.tblName; - maxParts = other1722.maxParts; - validWriteIdList = other1722.validWriteIdList; - id = other1722.id; - skipColumnSchemaForPartition = other1722.skipColumnSchemaForPartition; - __isset = other1722.__isset; +PartitionsRequest& PartitionsRequest::operator=(const PartitionsRequest& other1734) { + catName = other1734.catName; + dbName = other1734.dbName; + tblName = other1734.tblName; + maxParts = other1734.maxParts; + validWriteIdList = other1734.validWriteIdList; + id = other1734.id; + skipColumnSchemaForPartition = other1734.skipColumnSchemaForPartition; + __isset = other1734.__isset; return *this; } void PartitionsRequest::printTo(std::ostream& out) const { @@ -49172,14 +49300,14 @@ uint32_t PartitionsResponse::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitions.clear(); - uint32_t _size1723; - ::apache::thrift::protocol::TType _etype1726; - xfer += iprot->readListBegin(_etype1726, _size1723); - this->partitions.resize(_size1723); - uint32_t _i1727; - for (_i1727 = 0; _i1727 < _size1723; ++_i1727) + uint32_t _size1735; + ::apache::thrift::protocol::TType _etype1738; + xfer += iprot->readListBegin(_etype1738, _size1735); + this->partitions.resize(_size1735); + uint32_t _i1739; + for (_i1739 = 0; _i1739 < _size1735; ++_i1739) { - xfer += this->partitions[_i1727].read(iprot); + xfer += this->partitions[_i1739].read(iprot); } xfer += iprot->readListEnd(); } @@ -49210,10 +49338,10 @@ uint32_t PartitionsResponse::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter1728; - for (_iter1728 = this->partitions.begin(); _iter1728 != this->partitions.end(); ++_iter1728) + std::vector ::const_iterator _iter1740; + for (_iter1740 = this->partitions.begin(); _iter1740 != this->partitions.end(); ++_iter1740) { - xfer += (*_iter1728).write(oprot); + xfer += (*_iter1740).write(oprot); } xfer += oprot->writeListEnd(); } @@ -49229,11 +49357,11 @@ void swap(PartitionsResponse &a, PartitionsResponse &b) { swap(a.partitions, b.partitions); } -PartitionsResponse::PartitionsResponse(const PartitionsResponse& other1729) { - partitions = other1729.partitions; +PartitionsResponse::PartitionsResponse(const PartitionsResponse& other1741) { + partitions = other1741.partitions; } -PartitionsResponse& PartitionsResponse::operator=(const PartitionsResponse& other1730) { - partitions = other1730.partitions; +PartitionsResponse& PartitionsResponse::operator=(const PartitionsResponse& other1742) { + partitions = other1742.partitions; return *this; } void PartitionsResponse::printTo(std::ostream& out) const { @@ -49410,23 +49538,23 @@ void swap(GetPartitionsByFilterRequest &a, GetPartitionsByFilterRequest &b) { swap(a.__isset, b.__isset); } -GetPartitionsByFilterRequest::GetPartitionsByFilterRequest(const GetPartitionsByFilterRequest& other1731) { - catName = other1731.catName; - dbName = other1731.dbName; - tblName = other1731.tblName; - filter = other1731.filter; - maxParts = other1731.maxParts; - skipColumnSchemaForPartition = other1731.skipColumnSchemaForPartition; - __isset = other1731.__isset; +GetPartitionsByFilterRequest::GetPartitionsByFilterRequest(const GetPartitionsByFilterRequest& other1743) { + catName = other1743.catName; + dbName = other1743.dbName; + tblName = other1743.tblName; + filter = other1743.filter; + maxParts = other1743.maxParts; + skipColumnSchemaForPartition = other1743.skipColumnSchemaForPartition; + __isset = other1743.__isset; } -GetPartitionsByFilterRequest& GetPartitionsByFilterRequest::operator=(const GetPartitionsByFilterRequest& other1732) { - catName = other1732.catName; - dbName = other1732.dbName; - tblName = other1732.tblName; - filter = other1732.filter; - maxParts = other1732.maxParts; - skipColumnSchemaForPartition = other1732.skipColumnSchemaForPartition; - __isset = other1732.__isset; +GetPartitionsByFilterRequest& GetPartitionsByFilterRequest::operator=(const GetPartitionsByFilterRequest& other1744) { + catName = other1744.catName; + dbName = other1744.dbName; + tblName = other1744.tblName; + filter = other1744.filter; + maxParts = other1744.maxParts; + skipColumnSchemaForPartition = other1744.skipColumnSchemaForPartition; + __isset = other1744.__isset; return *this; } void GetPartitionsByFilterRequest::printTo(std::ostream& out) const { @@ -49536,14 +49664,14 @@ uint32_t GetPartitionNamesPsRequest::read(::apache::thrift::protocol::TProtocol* if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partValues.clear(); - uint32_t _size1733; - ::apache::thrift::protocol::TType _etype1736; - xfer += iprot->readListBegin(_etype1736, _size1733); - this->partValues.resize(_size1733); - uint32_t _i1737; - for (_i1737 = 0; _i1737 < _size1733; ++_i1737) + uint32_t _size1745; + ::apache::thrift::protocol::TType _etype1748; + xfer += iprot->readListBegin(_etype1748, _size1745); + this->partValues.resize(_size1745); + uint32_t _i1749; + for (_i1749 = 0; _i1749 < _size1745; ++_i1749) { - xfer += iprot->readString(this->partValues[_i1737]); + xfer += iprot->readString(this->partValues[_i1749]); } xfer += iprot->readListEnd(); } @@ -49614,10 +49742,10 @@ uint32_t GetPartitionNamesPsRequest::write(::apache::thrift::protocol::TProtocol xfer += oprot->writeFieldBegin("partValues", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partValues.size())); - std::vector ::const_iterator _iter1738; - for (_iter1738 = this->partValues.begin(); _iter1738 != this->partValues.end(); ++_iter1738) + std::vector ::const_iterator _iter1750; + for (_iter1750 = this->partValues.begin(); _iter1750 != this->partValues.end(); ++_iter1750) { - xfer += oprot->writeString((*_iter1738)); + xfer += oprot->writeString((*_iter1750)); } xfer += oprot->writeListEnd(); } @@ -49655,25 +49783,25 @@ void swap(GetPartitionNamesPsRequest &a, GetPartitionNamesPsRequest &b) { swap(a.__isset, b.__isset); } -GetPartitionNamesPsRequest::GetPartitionNamesPsRequest(const GetPartitionNamesPsRequest& other1739) { - catName = other1739.catName; - dbName = other1739.dbName; - tblName = other1739.tblName; - partValues = other1739.partValues; - maxParts = other1739.maxParts; - validWriteIdList = other1739.validWriteIdList; - id = other1739.id; - __isset = other1739.__isset; +GetPartitionNamesPsRequest::GetPartitionNamesPsRequest(const GetPartitionNamesPsRequest& other1751) { + catName = other1751.catName; + dbName = other1751.dbName; + tblName = other1751.tblName; + partValues = other1751.partValues; + maxParts = other1751.maxParts; + validWriteIdList = other1751.validWriteIdList; + id = other1751.id; + __isset = other1751.__isset; } -GetPartitionNamesPsRequest& GetPartitionNamesPsRequest::operator=(const GetPartitionNamesPsRequest& other1740) { - catName = other1740.catName; - dbName = other1740.dbName; - tblName = other1740.tblName; - partValues = other1740.partValues; - maxParts = other1740.maxParts; - validWriteIdList = other1740.validWriteIdList; - id = other1740.id; - __isset = other1740.__isset; +GetPartitionNamesPsRequest& GetPartitionNamesPsRequest::operator=(const GetPartitionNamesPsRequest& other1752) { + catName = other1752.catName; + dbName = other1752.dbName; + tblName = other1752.tblName; + partValues = other1752.partValues; + maxParts = other1752.maxParts; + validWriteIdList = other1752.validWriteIdList; + id = other1752.id; + __isset = other1752.__isset; return *this; } void GetPartitionNamesPsRequest::printTo(std::ostream& out) const { @@ -49730,14 +49858,14 @@ uint32_t GetPartitionNamesPsResponse::read(::apache::thrift::protocol::TProtocol if (ftype == ::apache::thrift::protocol::T_LIST) { { this->names.clear(); - uint32_t _size1741; - ::apache::thrift::protocol::TType _etype1744; - xfer += iprot->readListBegin(_etype1744, _size1741); - this->names.resize(_size1741); - uint32_t _i1745; - for (_i1745 = 0; _i1745 < _size1741; ++_i1745) + uint32_t _size1753; + ::apache::thrift::protocol::TType _etype1756; + xfer += iprot->readListBegin(_etype1756, _size1753); + this->names.resize(_size1753); + uint32_t _i1757; + for (_i1757 = 0; _i1757 < _size1753; ++_i1757) { - xfer += iprot->readString(this->names[_i1745]); + xfer += iprot->readString(this->names[_i1757]); } xfer += iprot->readListEnd(); } @@ -49768,10 +49896,10 @@ uint32_t GetPartitionNamesPsResponse::write(::apache::thrift::protocol::TProtoco xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); - std::vector ::const_iterator _iter1746; - for (_iter1746 = this->names.begin(); _iter1746 != this->names.end(); ++_iter1746) + std::vector ::const_iterator _iter1758; + for (_iter1758 = this->names.begin(); _iter1758 != this->names.end(); ++_iter1758) { - xfer += oprot->writeString((*_iter1746)); + xfer += oprot->writeString((*_iter1758)); } xfer += oprot->writeListEnd(); } @@ -49787,11 +49915,11 @@ void swap(GetPartitionNamesPsResponse &a, GetPartitionNamesPsResponse &b) { swap(a.names, b.names); } -GetPartitionNamesPsResponse::GetPartitionNamesPsResponse(const GetPartitionNamesPsResponse& other1747) { - names = other1747.names; +GetPartitionNamesPsResponse::GetPartitionNamesPsResponse(const GetPartitionNamesPsResponse& other1759) { + names = other1759.names; } -GetPartitionNamesPsResponse& GetPartitionNamesPsResponse::operator=(const GetPartitionNamesPsResponse& other1748) { - names = other1748.names; +GetPartitionNamesPsResponse& GetPartitionNamesPsResponse::operator=(const GetPartitionNamesPsResponse& other1760) { + names = other1760.names; return *this; } void GetPartitionNamesPsResponse::printTo(std::ostream& out) const { @@ -49911,14 +50039,14 @@ uint32_t GetPartitionsPsWithAuthRequest::read(::apache::thrift::protocol::TProto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partVals.clear(); - uint32_t _size1749; - ::apache::thrift::protocol::TType _etype1752; - xfer += iprot->readListBegin(_etype1752, _size1749); - this->partVals.resize(_size1749); - uint32_t _i1753; - for (_i1753 = 0; _i1753 < _size1749; ++_i1753) + uint32_t _size1761; + ::apache::thrift::protocol::TType _etype1764; + xfer += iprot->readListBegin(_etype1764, _size1761); + this->partVals.resize(_size1761); + uint32_t _i1765; + for (_i1765 = 0; _i1765 < _size1761; ++_i1765) { - xfer += iprot->readString(this->partVals[_i1753]); + xfer += iprot->readString(this->partVals[_i1765]); } xfer += iprot->readListEnd(); } @@ -49947,14 +50075,14 @@ uint32_t GetPartitionsPsWithAuthRequest::read(::apache::thrift::protocol::TProto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->groupNames.clear(); - uint32_t _size1754; - ::apache::thrift::protocol::TType _etype1757; - xfer += iprot->readListBegin(_etype1757, _size1754); - this->groupNames.resize(_size1754); - uint32_t _i1758; - for (_i1758 = 0; _i1758 < _size1754; ++_i1758) + uint32_t _size1766; + ::apache::thrift::protocol::TType _etype1769; + xfer += iprot->readListBegin(_etype1769, _size1766); + this->groupNames.resize(_size1766); + uint32_t _i1770; + for (_i1770 = 0; _i1770 < _size1766; ++_i1770) { - xfer += iprot->readString(this->groupNames[_i1758]); + xfer += iprot->readString(this->groupNames[_i1770]); } xfer += iprot->readListEnd(); } @@ -50025,10 +50153,10 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeFieldBegin("partVals", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partVals.size())); - std::vector ::const_iterator _iter1759; - for (_iter1759 = this->partVals.begin(); _iter1759 != this->partVals.end(); ++_iter1759) + std::vector ::const_iterator _iter1771; + for (_iter1771 = this->partVals.begin(); _iter1771 != this->partVals.end(); ++_iter1771) { - xfer += oprot->writeString((*_iter1759)); + xfer += oprot->writeString((*_iter1771)); } xfer += oprot->writeListEnd(); } @@ -50048,10 +50176,10 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeFieldBegin("groupNames", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->groupNames.size())); - std::vector ::const_iterator _iter1760; - for (_iter1760 = this->groupNames.begin(); _iter1760 != this->groupNames.end(); ++_iter1760) + std::vector ::const_iterator _iter1772; + for (_iter1772 = this->groupNames.begin(); _iter1772 != this->groupNames.end(); ++_iter1772) { - xfer += oprot->writeString((*_iter1760)); + xfer += oprot->writeString((*_iter1772)); } xfer += oprot->writeListEnd(); } @@ -50092,31 +50220,31 @@ void swap(GetPartitionsPsWithAuthRequest &a, GetPartitionsPsWithAuthRequest &b) swap(a.__isset, b.__isset); } -GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitionsPsWithAuthRequest& other1761) { - catName = other1761.catName; - dbName = other1761.dbName; - tblName = other1761.tblName; - partVals = other1761.partVals; - maxParts = other1761.maxParts; - userName = other1761.userName; - groupNames = other1761.groupNames; - validWriteIdList = other1761.validWriteIdList; - id = other1761.id; - skipColumnSchemaForPartition = other1761.skipColumnSchemaForPartition; - __isset = other1761.__isset; -} -GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1762) { - catName = other1762.catName; - dbName = other1762.dbName; - tblName = other1762.tblName; - partVals = other1762.partVals; - maxParts = other1762.maxParts; - userName = other1762.userName; - groupNames = other1762.groupNames; - validWriteIdList = other1762.validWriteIdList; - id = other1762.id; - skipColumnSchemaForPartition = other1762.skipColumnSchemaForPartition; - __isset = other1762.__isset; +GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitionsPsWithAuthRequest& other1773) { + catName = other1773.catName; + dbName = other1773.dbName; + tblName = other1773.tblName; + partVals = other1773.partVals; + maxParts = other1773.maxParts; + userName = other1773.userName; + groupNames = other1773.groupNames; + validWriteIdList = other1773.validWriteIdList; + id = other1773.id; + skipColumnSchemaForPartition = other1773.skipColumnSchemaForPartition; + __isset = other1773.__isset; +} +GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1774) { + catName = other1774.catName; + dbName = other1774.dbName; + tblName = other1774.tblName; + partVals = other1774.partVals; + maxParts = other1774.maxParts; + userName = other1774.userName; + groupNames = other1774.groupNames; + validWriteIdList = other1774.validWriteIdList; + id = other1774.id; + skipColumnSchemaForPartition = other1774.skipColumnSchemaForPartition; + __isset = other1774.__isset; return *this; } void GetPartitionsPsWithAuthRequest::printTo(std::ostream& out) const { @@ -50176,14 +50304,14 @@ uint32_t GetPartitionsPsWithAuthResponse::read(::apache::thrift::protocol::TProt if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitions.clear(); - uint32_t _size1763; - ::apache::thrift::protocol::TType _etype1766; - xfer += iprot->readListBegin(_etype1766, _size1763); - this->partitions.resize(_size1763); - uint32_t _i1767; - for (_i1767 = 0; _i1767 < _size1763; ++_i1767) + uint32_t _size1775; + ::apache::thrift::protocol::TType _etype1778; + xfer += iprot->readListBegin(_etype1778, _size1775); + this->partitions.resize(_size1775); + uint32_t _i1779; + for (_i1779 = 0; _i1779 < _size1775; ++_i1779) { - xfer += this->partitions[_i1767].read(iprot); + xfer += this->partitions[_i1779].read(iprot); } xfer += iprot->readListEnd(); } @@ -50214,10 +50342,10 @@ uint32_t GetPartitionsPsWithAuthResponse::write(::apache::thrift::protocol::TPro xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter1768; - for (_iter1768 = this->partitions.begin(); _iter1768 != this->partitions.end(); ++_iter1768) + std::vector ::const_iterator _iter1780; + for (_iter1780 = this->partitions.begin(); _iter1780 != this->partitions.end(); ++_iter1780) { - xfer += (*_iter1768).write(oprot); + xfer += (*_iter1780).write(oprot); } xfer += oprot->writeListEnd(); } @@ -50233,11 +50361,11 @@ void swap(GetPartitionsPsWithAuthResponse &a, GetPartitionsPsWithAuthResponse &b swap(a.partitions, b.partitions); } -GetPartitionsPsWithAuthResponse::GetPartitionsPsWithAuthResponse(const GetPartitionsPsWithAuthResponse& other1769) { - partitions = other1769.partitions; +GetPartitionsPsWithAuthResponse::GetPartitionsPsWithAuthResponse(const GetPartitionsPsWithAuthResponse& other1781) { + partitions = other1781.partitions; } -GetPartitionsPsWithAuthResponse& GetPartitionsPsWithAuthResponse::operator=(const GetPartitionsPsWithAuthResponse& other1770) { - partitions = other1770.partitions; +GetPartitionsPsWithAuthResponse& GetPartitionsPsWithAuthResponse::operator=(const GetPartitionsPsWithAuthResponse& other1782) { + partitions = other1782.partitions; return *this; } void GetPartitionsPsWithAuthResponse::printTo(std::ostream& out) const { @@ -50423,23 +50551,23 @@ void swap(ReplicationMetrics &a, ReplicationMetrics &b) { swap(a.__isset, b.__isset); } -ReplicationMetrics::ReplicationMetrics(const ReplicationMetrics& other1771) { - scheduledExecutionId = other1771.scheduledExecutionId; - policy = other1771.policy; - dumpExecutionId = other1771.dumpExecutionId; - metadata = other1771.metadata; - progress = other1771.progress; - messageFormat = other1771.messageFormat; - __isset = other1771.__isset; +ReplicationMetrics::ReplicationMetrics(const ReplicationMetrics& other1783) { + scheduledExecutionId = other1783.scheduledExecutionId; + policy = other1783.policy; + dumpExecutionId = other1783.dumpExecutionId; + metadata = other1783.metadata; + progress = other1783.progress; + messageFormat = other1783.messageFormat; + __isset = other1783.__isset; } -ReplicationMetrics& ReplicationMetrics::operator=(const ReplicationMetrics& other1772) { - scheduledExecutionId = other1772.scheduledExecutionId; - policy = other1772.policy; - dumpExecutionId = other1772.dumpExecutionId; - metadata = other1772.metadata; - progress = other1772.progress; - messageFormat = other1772.messageFormat; - __isset = other1772.__isset; +ReplicationMetrics& ReplicationMetrics::operator=(const ReplicationMetrics& other1784) { + scheduledExecutionId = other1784.scheduledExecutionId; + policy = other1784.policy; + dumpExecutionId = other1784.dumpExecutionId; + metadata = other1784.metadata; + progress = other1784.progress; + messageFormat = other1784.messageFormat; + __isset = other1784.__isset; return *this; } void ReplicationMetrics::printTo(std::ostream& out) const { @@ -50495,14 +50623,14 @@ uint32_t ReplicationMetricList::read(::apache::thrift::protocol::TProtocol* ipro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->replicationMetricList.clear(); - uint32_t _size1773; - ::apache::thrift::protocol::TType _etype1776; - xfer += iprot->readListBegin(_etype1776, _size1773); - this->replicationMetricList.resize(_size1773); - uint32_t _i1777; - for (_i1777 = 0; _i1777 < _size1773; ++_i1777) + uint32_t _size1785; + ::apache::thrift::protocol::TType _etype1788; + xfer += iprot->readListBegin(_etype1788, _size1785); + this->replicationMetricList.resize(_size1785); + uint32_t _i1789; + for (_i1789 = 0; _i1789 < _size1785; ++_i1789) { - xfer += this->replicationMetricList[_i1777].read(iprot); + xfer += this->replicationMetricList[_i1789].read(iprot); } xfer += iprot->readListEnd(); } @@ -50533,10 +50661,10 @@ uint32_t ReplicationMetricList::write(::apache::thrift::protocol::TProtocol* opr xfer += oprot->writeFieldBegin("replicationMetricList", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->replicationMetricList.size())); - std::vector ::const_iterator _iter1778; - for (_iter1778 = this->replicationMetricList.begin(); _iter1778 != this->replicationMetricList.end(); ++_iter1778) + std::vector ::const_iterator _iter1790; + for (_iter1790 = this->replicationMetricList.begin(); _iter1790 != this->replicationMetricList.end(); ++_iter1790) { - xfer += (*_iter1778).write(oprot); + xfer += (*_iter1790).write(oprot); } xfer += oprot->writeListEnd(); } @@ -50552,11 +50680,11 @@ void swap(ReplicationMetricList &a, ReplicationMetricList &b) { swap(a.replicationMetricList, b.replicationMetricList); } -ReplicationMetricList::ReplicationMetricList(const ReplicationMetricList& other1779) { - replicationMetricList = other1779.replicationMetricList; +ReplicationMetricList::ReplicationMetricList(const ReplicationMetricList& other1791) { + replicationMetricList = other1791.replicationMetricList; } -ReplicationMetricList& ReplicationMetricList::operator=(const ReplicationMetricList& other1780) { - replicationMetricList = other1780.replicationMetricList; +ReplicationMetricList& ReplicationMetricList::operator=(const ReplicationMetricList& other1792) { + replicationMetricList = other1792.replicationMetricList; return *this; } void ReplicationMetricList::printTo(std::ostream& out) const { @@ -50682,17 +50810,17 @@ void swap(GetReplicationMetricsRequest &a, GetReplicationMetricsRequest &b) { swap(a.__isset, b.__isset); } -GetReplicationMetricsRequest::GetReplicationMetricsRequest(const GetReplicationMetricsRequest& other1781) { - scheduledExecutionId = other1781.scheduledExecutionId; - policy = other1781.policy; - dumpExecutionId = other1781.dumpExecutionId; - __isset = other1781.__isset; +GetReplicationMetricsRequest::GetReplicationMetricsRequest(const GetReplicationMetricsRequest& other1793) { + scheduledExecutionId = other1793.scheduledExecutionId; + policy = other1793.policy; + dumpExecutionId = other1793.dumpExecutionId; + __isset = other1793.__isset; } -GetReplicationMetricsRequest& GetReplicationMetricsRequest::operator=(const GetReplicationMetricsRequest& other1782) { - scheduledExecutionId = other1782.scheduledExecutionId; - policy = other1782.policy; - dumpExecutionId = other1782.dumpExecutionId; - __isset = other1782.__isset; +GetReplicationMetricsRequest& GetReplicationMetricsRequest::operator=(const GetReplicationMetricsRequest& other1794) { + scheduledExecutionId = other1794.scheduledExecutionId; + policy = other1794.policy; + dumpExecutionId = other1794.dumpExecutionId; + __isset = other1794.__isset; return *this; } void GetReplicationMetricsRequest::printTo(std::ostream& out) const { @@ -50745,16 +50873,16 @@ uint32_t GetOpenTxnsRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->excludeTxnTypes.clear(); - uint32_t _size1783; - ::apache::thrift::protocol::TType _etype1786; - xfer += iprot->readListBegin(_etype1786, _size1783); - this->excludeTxnTypes.resize(_size1783); - uint32_t _i1787; - for (_i1787 = 0; _i1787 < _size1783; ++_i1787) + uint32_t _size1795; + ::apache::thrift::protocol::TType _etype1798; + xfer += iprot->readListBegin(_etype1798, _size1795); + this->excludeTxnTypes.resize(_size1795); + uint32_t _i1799; + for (_i1799 = 0; _i1799 < _size1795; ++_i1799) { - int32_t ecast1788; - xfer += iprot->readI32(ecast1788); - this->excludeTxnTypes[_i1787] = static_cast(ecast1788); + int32_t ecast1800; + xfer += iprot->readI32(ecast1800); + this->excludeTxnTypes[_i1799] = static_cast(ecast1800); } xfer += iprot->readListEnd(); } @@ -50784,10 +50912,10 @@ uint32_t GetOpenTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("excludeTxnTypes", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->excludeTxnTypes.size())); - std::vector ::const_iterator _iter1789; - for (_iter1789 = this->excludeTxnTypes.begin(); _iter1789 != this->excludeTxnTypes.end(); ++_iter1789) + std::vector ::const_iterator _iter1801; + for (_iter1801 = this->excludeTxnTypes.begin(); _iter1801 != this->excludeTxnTypes.end(); ++_iter1801) { - xfer += oprot->writeI32(static_cast((*_iter1789))); + xfer += oprot->writeI32(static_cast((*_iter1801))); } xfer += oprot->writeListEnd(); } @@ -50804,13 +50932,13 @@ void swap(GetOpenTxnsRequest &a, GetOpenTxnsRequest &b) { swap(a.__isset, b.__isset); } -GetOpenTxnsRequest::GetOpenTxnsRequest(const GetOpenTxnsRequest& other1790) { - excludeTxnTypes = other1790.excludeTxnTypes; - __isset = other1790.__isset; +GetOpenTxnsRequest::GetOpenTxnsRequest(const GetOpenTxnsRequest& other1802) { + excludeTxnTypes = other1802.excludeTxnTypes; + __isset = other1802.__isset; } -GetOpenTxnsRequest& GetOpenTxnsRequest::operator=(const GetOpenTxnsRequest& other1791) { - excludeTxnTypes = other1791.excludeTxnTypes; - __isset = other1791.__isset; +GetOpenTxnsRequest& GetOpenTxnsRequest::operator=(const GetOpenTxnsRequest& other1803) { + excludeTxnTypes = other1803.excludeTxnTypes; + __isset = other1803.__isset; return *this; } void GetOpenTxnsRequest::printTo(std::ostream& out) const { @@ -50938,15 +51066,15 @@ void swap(StoredProcedureRequest &a, StoredProcedureRequest &b) { swap(a.procName, b.procName); } -StoredProcedureRequest::StoredProcedureRequest(const StoredProcedureRequest& other1792) { - catName = other1792.catName; - dbName = other1792.dbName; - procName = other1792.procName; +StoredProcedureRequest::StoredProcedureRequest(const StoredProcedureRequest& other1804) { + catName = other1804.catName; + dbName = other1804.dbName; + procName = other1804.procName; } -StoredProcedureRequest& StoredProcedureRequest::operator=(const StoredProcedureRequest& other1793) { - catName = other1793.catName; - dbName = other1793.dbName; - procName = other1793.procName; +StoredProcedureRequest& StoredProcedureRequest::operator=(const StoredProcedureRequest& other1805) { + catName = other1805.catName; + dbName = other1805.dbName; + procName = other1805.procName; return *this; } void StoredProcedureRequest::printTo(std::ostream& out) const { @@ -51056,15 +51184,15 @@ void swap(ListStoredProcedureRequest &a, ListStoredProcedureRequest &b) { swap(a.__isset, b.__isset); } -ListStoredProcedureRequest::ListStoredProcedureRequest(const ListStoredProcedureRequest& other1794) { - catName = other1794.catName; - dbName = other1794.dbName; - __isset = other1794.__isset; +ListStoredProcedureRequest::ListStoredProcedureRequest(const ListStoredProcedureRequest& other1806) { + catName = other1806.catName; + dbName = other1806.dbName; + __isset = other1806.__isset; } -ListStoredProcedureRequest& ListStoredProcedureRequest::operator=(const ListStoredProcedureRequest& other1795) { - catName = other1795.catName; - dbName = other1795.dbName; - __isset = other1795.__isset; +ListStoredProcedureRequest& ListStoredProcedureRequest::operator=(const ListStoredProcedureRequest& other1807) { + catName = other1807.catName; + dbName = other1807.dbName; + __isset = other1807.__isset; return *this; } void ListStoredProcedureRequest::printTo(std::ostream& out) const { @@ -51219,21 +51347,21 @@ void swap(StoredProcedure &a, StoredProcedure &b) { swap(a.__isset, b.__isset); } -StoredProcedure::StoredProcedure(const StoredProcedure& other1796) { - name = other1796.name; - dbName = other1796.dbName; - catName = other1796.catName; - ownerName = other1796.ownerName; - source = other1796.source; - __isset = other1796.__isset; +StoredProcedure::StoredProcedure(const StoredProcedure& other1808) { + name = other1808.name; + dbName = other1808.dbName; + catName = other1808.catName; + ownerName = other1808.ownerName; + source = other1808.source; + __isset = other1808.__isset; } -StoredProcedure& StoredProcedure::operator=(const StoredProcedure& other1797) { - name = other1797.name; - dbName = other1797.dbName; - catName = other1797.catName; - ownerName = other1797.ownerName; - source = other1797.source; - __isset = other1797.__isset; +StoredProcedure& StoredProcedure::operator=(const StoredProcedure& other1809) { + name = other1809.name; + dbName = other1809.dbName; + catName = other1809.catName; + ownerName = other1809.ownerName; + source = other1809.source; + __isset = other1809.__isset; return *this; } void StoredProcedure::printTo(std::ostream& out) const { @@ -51408,23 +51536,23 @@ void swap(AddPackageRequest &a, AddPackageRequest &b) { swap(a.__isset, b.__isset); } -AddPackageRequest::AddPackageRequest(const AddPackageRequest& other1798) { - catName = other1798.catName; - dbName = other1798.dbName; - packageName = other1798.packageName; - ownerName = other1798.ownerName; - header = other1798.header; - body = other1798.body; - __isset = other1798.__isset; +AddPackageRequest::AddPackageRequest(const AddPackageRequest& other1810) { + catName = other1810.catName; + dbName = other1810.dbName; + packageName = other1810.packageName; + ownerName = other1810.ownerName; + header = other1810.header; + body = other1810.body; + __isset = other1810.__isset; } -AddPackageRequest& AddPackageRequest::operator=(const AddPackageRequest& other1799) { - catName = other1799.catName; - dbName = other1799.dbName; - packageName = other1799.packageName; - ownerName = other1799.ownerName; - header = other1799.header; - body = other1799.body; - __isset = other1799.__isset; +AddPackageRequest& AddPackageRequest::operator=(const AddPackageRequest& other1811) { + catName = other1811.catName; + dbName = other1811.dbName; + packageName = other1811.packageName; + ownerName = other1811.ownerName; + header = other1811.header; + body = other1811.body; + __isset = other1811.__isset; return *this; } void AddPackageRequest::printTo(std::ostream& out) const { @@ -51557,15 +51685,15 @@ void swap(GetPackageRequest &a, GetPackageRequest &b) { swap(a.packageName, b.packageName); } -GetPackageRequest::GetPackageRequest(const GetPackageRequest& other1800) { - catName = other1800.catName; - dbName = other1800.dbName; - packageName = other1800.packageName; +GetPackageRequest::GetPackageRequest(const GetPackageRequest& other1812) { + catName = other1812.catName; + dbName = other1812.dbName; + packageName = other1812.packageName; } -GetPackageRequest& GetPackageRequest::operator=(const GetPackageRequest& other1801) { - catName = other1801.catName; - dbName = other1801.dbName; - packageName = other1801.packageName; +GetPackageRequest& GetPackageRequest::operator=(const GetPackageRequest& other1813) { + catName = other1813.catName; + dbName = other1813.dbName; + packageName = other1813.packageName; return *this; } void GetPackageRequest::printTo(std::ostream& out) const { @@ -51695,15 +51823,15 @@ void swap(DropPackageRequest &a, DropPackageRequest &b) { swap(a.packageName, b.packageName); } -DropPackageRequest::DropPackageRequest(const DropPackageRequest& other1802) { - catName = other1802.catName; - dbName = other1802.dbName; - packageName = other1802.packageName; +DropPackageRequest::DropPackageRequest(const DropPackageRequest& other1814) { + catName = other1814.catName; + dbName = other1814.dbName; + packageName = other1814.packageName; } -DropPackageRequest& DropPackageRequest::operator=(const DropPackageRequest& other1803) { - catName = other1803.catName; - dbName = other1803.dbName; - packageName = other1803.packageName; +DropPackageRequest& DropPackageRequest::operator=(const DropPackageRequest& other1815) { + catName = other1815.catName; + dbName = other1815.dbName; + packageName = other1815.packageName; return *this; } void DropPackageRequest::printTo(std::ostream& out) const { @@ -51813,15 +51941,15 @@ void swap(ListPackageRequest &a, ListPackageRequest &b) { swap(a.__isset, b.__isset); } -ListPackageRequest::ListPackageRequest(const ListPackageRequest& other1804) { - catName = other1804.catName; - dbName = other1804.dbName; - __isset = other1804.__isset; +ListPackageRequest::ListPackageRequest(const ListPackageRequest& other1816) { + catName = other1816.catName; + dbName = other1816.dbName; + __isset = other1816.__isset; } -ListPackageRequest& ListPackageRequest::operator=(const ListPackageRequest& other1805) { - catName = other1805.catName; - dbName = other1805.dbName; - __isset = other1805.__isset; +ListPackageRequest& ListPackageRequest::operator=(const ListPackageRequest& other1817) { + catName = other1817.catName; + dbName = other1817.dbName; + __isset = other1817.__isset; return *this; } void ListPackageRequest::printTo(std::ostream& out) const { @@ -51993,23 +52121,23 @@ void swap(Package &a, Package &b) { swap(a.__isset, b.__isset); } -Package::Package(const Package& other1806) { - catName = other1806.catName; - dbName = other1806.dbName; - packageName = other1806.packageName; - ownerName = other1806.ownerName; - header = other1806.header; - body = other1806.body; - __isset = other1806.__isset; +Package::Package(const Package& other1818) { + catName = other1818.catName; + dbName = other1818.dbName; + packageName = other1818.packageName; + ownerName = other1818.ownerName; + header = other1818.header; + body = other1818.body; + __isset = other1818.__isset; } -Package& Package::operator=(const Package& other1807) { - catName = other1807.catName; - dbName = other1807.dbName; - packageName = other1807.packageName; - ownerName = other1807.ownerName; - header = other1807.header; - body = other1807.body; - __isset = other1807.__isset; +Package& Package::operator=(const Package& other1819) { + catName = other1819.catName; + dbName = other1819.dbName; + packageName = other1819.packageName; + ownerName = other1819.ownerName; + header = other1819.header; + body = other1819.body; + __isset = other1819.__isset; return *this; } void Package::printTo(std::ostream& out) const { @@ -52141,17 +52269,17 @@ void swap(GetAllWriteEventInfoRequest &a, GetAllWriteEventInfoRequest &b) { swap(a.__isset, b.__isset); } -GetAllWriteEventInfoRequest::GetAllWriteEventInfoRequest(const GetAllWriteEventInfoRequest& other1808) { - txnId = other1808.txnId; - dbName = other1808.dbName; - tableName = other1808.tableName; - __isset = other1808.__isset; +GetAllWriteEventInfoRequest::GetAllWriteEventInfoRequest(const GetAllWriteEventInfoRequest& other1820) { + txnId = other1820.txnId; + dbName = other1820.dbName; + tableName = other1820.tableName; + __isset = other1820.__isset; } -GetAllWriteEventInfoRequest& GetAllWriteEventInfoRequest::operator=(const GetAllWriteEventInfoRequest& other1809) { - txnId = other1809.txnId; - dbName = other1809.dbName; - tableName = other1809.tableName; - __isset = other1809.__isset; +GetAllWriteEventInfoRequest& GetAllWriteEventInfoRequest::operator=(const GetAllWriteEventInfoRequest& other1821) { + txnId = other1821.txnId; + dbName = other1821.dbName; + tableName = other1821.tableName; + __isset = other1821.__isset; return *this; } void GetAllWriteEventInfoRequest::printTo(std::ostream& out) const { @@ -52239,13 +52367,13 @@ void swap(MetaException &a, MetaException &b) { swap(a.__isset, b.__isset); } -MetaException::MetaException(const MetaException& other1810) : TException() { - message = other1810.message; - __isset = other1810.__isset; +MetaException::MetaException(const MetaException& other1822) : TException() { + message = other1822.message; + __isset = other1822.__isset; } -MetaException& MetaException::operator=(const MetaException& other1811) { - message = other1811.message; - __isset = other1811.__isset; +MetaException& MetaException::operator=(const MetaException& other1823) { + message = other1823.message; + __isset = other1823.__isset; return *this; } void MetaException::printTo(std::ostream& out) const { @@ -52342,13 +52470,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) { swap(a.__isset, b.__isset); } -UnknownTableException::UnknownTableException(const UnknownTableException& other1812) : TException() { - message = other1812.message; - __isset = other1812.__isset; +UnknownTableException::UnknownTableException(const UnknownTableException& other1824) : TException() { + message = other1824.message; + __isset = other1824.__isset; } -UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1813) { - message = other1813.message; - __isset = other1813.__isset; +UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1825) { + message = other1825.message; + __isset = other1825.__isset; return *this; } void UnknownTableException::printTo(std::ostream& out) const { @@ -52445,13 +52573,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) { swap(a.__isset, b.__isset); } -UnknownDBException::UnknownDBException(const UnknownDBException& other1814) : TException() { - message = other1814.message; - __isset = other1814.__isset; +UnknownDBException::UnknownDBException(const UnknownDBException& other1826) : TException() { + message = other1826.message; + __isset = other1826.__isset; } -UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1815) { - message = other1815.message; - __isset = other1815.__isset; +UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1827) { + message = other1827.message; + __isset = other1827.__isset; return *this; } void UnknownDBException::printTo(std::ostream& out) const { @@ -52548,13 +52676,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) { swap(a.__isset, b.__isset); } -AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1816) : TException() { - message = other1816.message; - __isset = other1816.__isset; +AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1828) : TException() { + message = other1828.message; + __isset = other1828.__isset; } -AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1817) { - message = other1817.message; - __isset = other1817.__isset; +AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1829) { + message = other1829.message; + __isset = other1829.__isset; return *this; } void AlreadyExistsException::printTo(std::ostream& out) const { @@ -52651,13 +52779,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) { swap(a.__isset, b.__isset); } -InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1818) : TException() { - message = other1818.message; - __isset = other1818.__isset; +InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1830) : TException() { + message = other1830.message; + __isset = other1830.__isset; } -InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1819) { - message = other1819.message; - __isset = other1819.__isset; +InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1831) { + message = other1831.message; + __isset = other1831.__isset; return *this; } void InvalidPartitionException::printTo(std::ostream& out) const { @@ -52754,13 +52882,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) { swap(a.__isset, b.__isset); } -UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1820) : TException() { - message = other1820.message; - __isset = other1820.__isset; +UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1832) : TException() { + message = other1832.message; + __isset = other1832.__isset; } -UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1821) { - message = other1821.message; - __isset = other1821.__isset; +UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1833) { + message = other1833.message; + __isset = other1833.__isset; return *this; } void UnknownPartitionException::printTo(std::ostream& out) const { @@ -52857,13 +52985,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) { swap(a.__isset, b.__isset); } -InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1822) : TException() { - message = other1822.message; - __isset = other1822.__isset; +InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1834) : TException() { + message = other1834.message; + __isset = other1834.__isset; } -InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1823) { - message = other1823.message; - __isset = other1823.__isset; +InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1835) { + message = other1835.message; + __isset = other1835.__isset; return *this; } void InvalidObjectException::printTo(std::ostream& out) const { @@ -52960,13 +53088,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) { swap(a.__isset, b.__isset); } -NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1824) : TException() { - message = other1824.message; - __isset = other1824.__isset; +NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1836) : TException() { + message = other1836.message; + __isset = other1836.__isset; } -NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1825) { - message = other1825.message; - __isset = other1825.__isset; +NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1837) { + message = other1837.message; + __isset = other1837.__isset; return *this; } void NoSuchObjectException::printTo(std::ostream& out) const { @@ -53063,13 +53191,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) { swap(a.__isset, b.__isset); } -InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1826) : TException() { - message = other1826.message; - __isset = other1826.__isset; +InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1838) : TException() { + message = other1838.message; + __isset = other1838.__isset; } -InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1827) { - message = other1827.message; - __isset = other1827.__isset; +InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1839) { + message = other1839.message; + __isset = other1839.__isset; return *this; } void InvalidOperationException::printTo(std::ostream& out) const { @@ -53166,13 +53294,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) { swap(a.__isset, b.__isset); } -ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1828) : TException() { - message = other1828.message; - __isset = other1828.__isset; +ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1840) : TException() { + message = other1840.message; + __isset = other1840.__isset; } -ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1829) { - message = other1829.message; - __isset = other1829.__isset; +ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1841) { + message = other1841.message; + __isset = other1841.__isset; return *this; } void ConfigValSecurityException::printTo(std::ostream& out) const { @@ -53269,13 +53397,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) { swap(a.__isset, b.__isset); } -InvalidInputException::InvalidInputException(const InvalidInputException& other1830) : TException() { - message = other1830.message; - __isset = other1830.__isset; +InvalidInputException::InvalidInputException(const InvalidInputException& other1842) : TException() { + message = other1842.message; + __isset = other1842.__isset; } -InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1831) { - message = other1831.message; - __isset = other1831.__isset; +InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1843) { + message = other1843.message; + __isset = other1843.__isset; return *this; } void InvalidInputException::printTo(std::ostream& out) const { @@ -53372,13 +53500,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) { swap(a.__isset, b.__isset); } -NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1832) : TException() { - message = other1832.message; - __isset = other1832.__isset; +NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1844) : TException() { + message = other1844.message; + __isset = other1844.__isset; } -NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1833) { - message = other1833.message; - __isset = other1833.__isset; +NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1845) { + message = other1845.message; + __isset = other1845.__isset; return *this; } void NoSuchTxnException::printTo(std::ostream& out) const { @@ -53475,13 +53603,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) { swap(a.__isset, b.__isset); } -TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1834) : TException() { - message = other1834.message; - __isset = other1834.__isset; +TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1846) : TException() { + message = other1846.message; + __isset = other1846.__isset; } -TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1835) { - message = other1835.message; - __isset = other1835.__isset; +TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1847) { + message = other1847.message; + __isset = other1847.__isset; return *this; } void TxnAbortedException::printTo(std::ostream& out) const { @@ -53578,13 +53706,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) { swap(a.__isset, b.__isset); } -TxnOpenException::TxnOpenException(const TxnOpenException& other1836) : TException() { - message = other1836.message; - __isset = other1836.__isset; +TxnOpenException::TxnOpenException(const TxnOpenException& other1848) : TException() { + message = other1848.message; + __isset = other1848.__isset; } -TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1837) { - message = other1837.message; - __isset = other1837.__isset; +TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1849) { + message = other1849.message; + __isset = other1849.__isset; return *this; } void TxnOpenException::printTo(std::ostream& out) const { @@ -53681,13 +53809,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) { swap(a.__isset, b.__isset); } -NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1838) : TException() { - message = other1838.message; - __isset = other1838.__isset; +NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1850) : TException() { + message = other1850.message; + __isset = other1850.__isset; } -NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1839) { - message = other1839.message; - __isset = other1839.__isset; +NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1851) { + message = other1851.message; + __isset = other1851.__isset; return *this; } void NoSuchLockException::printTo(std::ostream& out) const { @@ -53784,13 +53912,13 @@ void swap(CompactionAbortedException &a, CompactionAbortedException &b) { swap(a.__isset, b.__isset); } -CompactionAbortedException::CompactionAbortedException(const CompactionAbortedException& other1840) : TException() { - message = other1840.message; - __isset = other1840.__isset; +CompactionAbortedException::CompactionAbortedException(const CompactionAbortedException& other1852) : TException() { + message = other1852.message; + __isset = other1852.__isset; } -CompactionAbortedException& CompactionAbortedException::operator=(const CompactionAbortedException& other1841) { - message = other1841.message; - __isset = other1841.__isset; +CompactionAbortedException& CompactionAbortedException::operator=(const CompactionAbortedException& other1853) { + message = other1853.message; + __isset = other1853.__isset; return *this; } void CompactionAbortedException::printTo(std::ostream& out) const { @@ -53887,13 +54015,13 @@ void swap(NoSuchCompactionException &a, NoSuchCompactionException &b) { swap(a.__isset, b.__isset); } -NoSuchCompactionException::NoSuchCompactionException(const NoSuchCompactionException& other1842) : TException() { - message = other1842.message; - __isset = other1842.__isset; +NoSuchCompactionException::NoSuchCompactionException(const NoSuchCompactionException& other1854) : TException() { + message = other1854.message; + __isset = other1854.__isset; } -NoSuchCompactionException& NoSuchCompactionException::operator=(const NoSuchCompactionException& other1843) { - message = other1843.message; - __isset = other1843.__isset; +NoSuchCompactionException& NoSuchCompactionException::operator=(const NoSuchCompactionException& other1855) { + message = other1855.message; + __isset = other1855.__isset; return *this; } void NoSuchCompactionException::printTo(std::ostream& out) const { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h index 949c09d9a544..09238fa5e617 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -12383,9 +12383,12 @@ void swap(BasicTxnInfo &a, BasicTxnInfo &b); std::ostream& operator<<(std::ostream& out, const BasicTxnInfo& obj); typedef struct _NotificationEventRequest__isset { - _NotificationEventRequest__isset() : maxEvents(false), eventTypeSkipList(false) {} + _NotificationEventRequest__isset() : maxEvents(false), eventTypeSkipList(false), catName(false), dbName(false), tableNames(false) {} bool maxEvents :1; bool eventTypeSkipList :1; + bool catName :1; + bool dbName :1; + bool tableNames :1; } _NotificationEventRequest__isset; class NotificationEventRequest : public virtual ::apache::thrift::TBase { @@ -12395,13 +12398,18 @@ class NotificationEventRequest : public virtual ::apache::thrift::TBase { NotificationEventRequest& operator=(const NotificationEventRequest&); NotificationEventRequest() noexcept : lastEvent(0), - maxEvents(0) { + maxEvents(0), + catName(), + dbName() { } virtual ~NotificationEventRequest() noexcept; int64_t lastEvent; int32_t maxEvents; std::vector eventTypeSkipList; + std::string catName; + std::string dbName; + std::vector tableNames; _NotificationEventRequest__isset __isset; @@ -12411,6 +12419,12 @@ class NotificationEventRequest : public virtual ::apache::thrift::TBase { void __set_eventTypeSkipList(const std::vector & val); + void __set_catName(const std::string& val); + + void __set_dbName(const std::string& val); + + void __set_tableNames(const std::vector & val); + bool operator == (const NotificationEventRequest & rhs) const { if (!(lastEvent == rhs.lastEvent)) @@ -12423,6 +12437,18 @@ class NotificationEventRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.eventTypeSkipList && !(eventTypeSkipList == rhs.eventTypeSkipList)) return false; + if (__isset.catName != rhs.__isset.catName) + return false; + else if (__isset.catName && !(catName == rhs.catName)) + return false; + if (__isset.dbName != rhs.__isset.dbName) + return false; + else if (__isset.dbName && !(dbName == rhs.dbName)) + return false; + if (__isset.tableNames != rhs.__isset.tableNames) + return false; + else if (__isset.tableNames && !(tableNames == rhs.tableNames)) + return false; return true; } bool operator != (const NotificationEventRequest &rhs) const { @@ -12611,10 +12637,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b); std::ostream& operator<<(std::ostream& out, const CurrentNotificationEventId& obj); typedef struct _NotificationEventsCountRequest__isset { - _NotificationEventsCountRequest__isset() : catName(false), toEventId(false), limit(false) {} + _NotificationEventsCountRequest__isset() : catName(false), toEventId(false), limit(false), tableNames(false) {} bool catName :1; bool toEventId :1; bool limit :1; + bool tableNames :1; } _NotificationEventsCountRequest__isset; class NotificationEventsCountRequest : public virtual ::apache::thrift::TBase { @@ -12636,6 +12663,7 @@ class NotificationEventsCountRequest : public virtual ::apache::thrift::TBase { std::string catName; int64_t toEventId; int64_t limit; + std::vector tableNames; _NotificationEventsCountRequest__isset __isset; @@ -12649,6 +12677,8 @@ class NotificationEventsCountRequest : public virtual ::apache::thrift::TBase { void __set_limit(const int64_t val); + void __set_tableNames(const std::vector & val); + bool operator == (const NotificationEventsCountRequest & rhs) const { if (!(fromEventId == rhs.fromEventId)) @@ -12667,6 +12697,10 @@ class NotificationEventsCountRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.limit && !(limit == rhs.limit)) return false; + if (__isset.tableNames != rhs.__isset.tableNames) + return false; + else if (__isset.tableNames && !(tableNames == rhs.tableNames)) + return false; return true; } bool operator != (const NotificationEventsCountRequest &rhs) const { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java index a85ccdefe22a..40466c51ede7 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java @@ -1028,14 +1028,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, AlterPartitionsRequ case 4: // PARTITIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1360 = iprot.readListBegin(); - struct.partitions = new java.util.ArrayList(_list1360.size); - @org.apache.thrift.annotation.Nullable Partition _elem1361; - for (int _i1362 = 0; _i1362 < _list1360.size; ++_i1362) + org.apache.thrift.protocol.TList _list1376 = iprot.readListBegin(); + struct.partitions = new java.util.ArrayList(_list1376.size); + @org.apache.thrift.annotation.Nullable Partition _elem1377; + for (int _i1378 = 0; _i1378 < _list1376.size; ++_i1378) { - _elem1361 = new Partition(); - _elem1361.read(iprot); - struct.partitions.add(_elem1361); + _elem1377 = new Partition(); + _elem1377.read(iprot); + struct.partitions.add(_elem1377); } iprot.readListEnd(); } @@ -1080,14 +1080,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, AlterPartitionsRequ case 9: // PARTITION_COL_SCHEMA if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1363 = iprot.readListBegin(); - struct.partitionColSchema = new java.util.ArrayList(_list1363.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1364; - for (int _i1365 = 0; _i1365 < _list1363.size; ++_i1365) + org.apache.thrift.protocol.TList _list1379 = iprot.readListBegin(); + struct.partitionColSchema = new java.util.ArrayList(_list1379.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1380; + for (int _i1381 = 0; _i1381 < _list1379.size; ++_i1381) { - _elem1364 = new FieldSchema(); - _elem1364.read(iprot); - struct.partitionColSchema.add(_elem1364); + _elem1380 = new FieldSchema(); + _elem1380.read(iprot); + struct.partitionColSchema.add(_elem1380); } iprot.readListEnd(); } @@ -1130,9 +1130,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, AlterPartitionsReq oprot.writeFieldBegin(PARTITIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size())); - for (Partition _iter1366 : struct.partitions) + for (Partition _iter1382 : struct.partitions) { - _iter1366.write(oprot); + _iter1382.write(oprot); } oprot.writeListEnd(); } @@ -1167,9 +1167,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, AlterPartitionsReq oprot.writeFieldBegin(PARTITION_COL_SCHEMA_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionColSchema.size())); - for (FieldSchema _iter1367 : struct.partitionColSchema) + for (FieldSchema _iter1383 : struct.partitionColSchema) { - _iter1367.write(oprot); + _iter1383.write(oprot); } oprot.writeListEnd(); } @@ -1197,9 +1197,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, AlterPartitionsRequ oprot.writeString(struct.tableName); { oprot.writeI32(struct.partitions.size()); - for (Partition _iter1368 : struct.partitions) + for (Partition _iter1384 : struct.partitions) { - _iter1368.write(oprot); + _iter1384.write(oprot); } } java.util.BitSet optionals = new java.util.BitSet(); @@ -1240,9 +1240,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, AlterPartitionsRequ if (struct.isSetPartitionColSchema()) { { oprot.writeI32(struct.partitionColSchema.size()); - for (FieldSchema _iter1369 : struct.partitionColSchema) + for (FieldSchema _iter1385 : struct.partitionColSchema) { - _iter1369.write(oprot); + _iter1385.write(oprot); } } } @@ -1256,14 +1256,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, AlterPartitionsReque struct.tableName = iprot.readString(); struct.setTableNameIsSet(true); { - org.apache.thrift.protocol.TList _list1370 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.partitions = new java.util.ArrayList(_list1370.size); - @org.apache.thrift.annotation.Nullable Partition _elem1371; - for (int _i1372 = 0; _i1372 < _list1370.size; ++_i1372) + org.apache.thrift.protocol.TList _list1386 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitions = new java.util.ArrayList(_list1386.size); + @org.apache.thrift.annotation.Nullable Partition _elem1387; + for (int _i1388 = 0; _i1388 < _list1386.size; ++_i1388) { - _elem1371 = new Partition(); - _elem1371.read(iprot); - struct.partitions.add(_elem1371); + _elem1387 = new Partition(); + _elem1387.read(iprot); + struct.partitions.add(_elem1387); } } struct.setPartitionsIsSet(true); @@ -1291,14 +1291,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, AlterPartitionsReque } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1373 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.partitionColSchema = new java.util.ArrayList(_list1373.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1374; - for (int _i1375 = 0; _i1375 < _list1373.size; ++_i1375) + org.apache.thrift.protocol.TList _list1389 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitionColSchema = new java.util.ArrayList(_list1389.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1390; + for (int _i1391 = 0; _i1391 < _list1389.size; ++_i1391) { - _elem1374 = new FieldSchema(); - _elem1374.read(iprot); - struct.partitionColSchema.add(_elem1374); + _elem1390 = new FieldSchema(); + _elem1390.read(iprot); + struct.partitionColSchema.add(_elem1390); } } struct.setPartitionColSchemaIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java index f2c050b1bf00..f9f5aeb450f0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java @@ -1209,13 +1209,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, AlterTableRequest s case 8: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1384 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1384.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1385; - for (int _i1386 = 0; _i1386 < _list1384.size; ++_i1386) + org.apache.thrift.protocol.TList _list1400 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1400.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1401; + for (int _i1402 = 0; _i1402 < _list1400.size; ++_i1402) { - _elem1385 = iprot.readString(); - struct.processorCapabilities.add(_elem1385); + _elem1401 = iprot.readString(); + struct.processorCapabilities.add(_elem1401); } iprot.readListEnd(); } @@ -1307,9 +1307,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, AlterTableRequest oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1387 : struct.processorCapabilities) + for (java.lang.String _iter1403 : struct.processorCapabilities) { - oprot.writeString(_iter1387); + oprot.writeString(_iter1403); } oprot.writeListEnd(); } @@ -1398,9 +1398,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, AlterTableRequest s if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1388 : struct.processorCapabilities) + for (java.lang.String _iter1404 : struct.processorCapabilities) { - oprot.writeString(_iter1388); + oprot.writeString(_iter1404); } } } @@ -1445,13 +1445,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, AlterTableRequest st } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1389 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1389.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1390; - for (int _i1391 = 0; _i1391 < _list1389.size; ++_i1391) + org.apache.thrift.protocol.TList _list1405 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1405.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1406; + for (int _i1407 = 0; _i1407 < _list1405.size; ++_i1407) { - _elem1390 = iprot.readString(); - struct.processorCapabilities.add(_elem1390); + _elem1406 = iprot.readString(); + struct.processorCapabilities.add(_elem1406); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java index 09b0a12233f8..a00611f5224f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java @@ -326,13 +326,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClearFileMetadataRe case 1: // FILE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1118 = iprot.readListBegin(); - struct.fileIds = new java.util.ArrayList(_list1118.size); - long _elem1119; - for (int _i1120 = 0; _i1120 < _list1118.size; ++_i1120) + org.apache.thrift.protocol.TList _list1134 = iprot.readListBegin(); + struct.fileIds = new java.util.ArrayList(_list1134.size); + long _elem1135; + for (int _i1136 = 0; _i1136 < _list1134.size; ++_i1136) { - _elem1119 = iprot.readI64(); - struct.fileIds.add(_elem1119); + _elem1135 = iprot.readI64(); + struct.fileIds.add(_elem1135); } iprot.readListEnd(); } @@ -358,9 +358,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClearFileMetadataR oprot.writeFieldBegin(FILE_IDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size())); - for (long _iter1121 : struct.fileIds) + for (long _iter1137 : struct.fileIds) { - oprot.writeI64(_iter1121); + oprot.writeI64(_iter1137); } oprot.writeListEnd(); } @@ -385,9 +385,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRe org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.fileIds.size()); - for (long _iter1122 : struct.fileIds) + for (long _iter1138 : struct.fileIds) { - oprot.writeI64(_iter1122); + oprot.writeI64(_iter1138); } } } @@ -396,13 +396,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRe public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1123 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); - struct.fileIds = new java.util.ArrayList(_list1123.size); - long _elem1124; - for (int _i1125 = 0; _i1125 < _list1123.size; ++_i1125) + org.apache.thrift.protocol.TList _list1139 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); + struct.fileIds = new java.util.ArrayList(_list1139.size); + long _elem1140; + for (int _i1141 = 0; _i1141 < _list1139.size; ++_i1141) { - _elem1124 = iprot.readI64(); - struct.fileIds.add(_elem1124); + _elem1140 = iprot.readI64(); + struct.fileIds.add(_elem1140); } } struct.setFileIdsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java index 43d545dcfb1d..969d42ebcfab 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java @@ -329,15 +329,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClientCapabilities case 1: // VALUES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1134 = iprot.readListBegin(); - struct.values = new java.util.ArrayList(_list1134.size); - @org.apache.thrift.annotation.Nullable ClientCapability _elem1135; - for (int _i1136 = 0; _i1136 < _list1134.size; ++_i1136) + org.apache.thrift.protocol.TList _list1150 = iprot.readListBegin(); + struct.values = new java.util.ArrayList(_list1150.size); + @org.apache.thrift.annotation.Nullable ClientCapability _elem1151; + for (int _i1152 = 0; _i1152 < _list1150.size; ++_i1152) { - _elem1135 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32()); - if (_elem1135 != null) + _elem1151 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32()); + if (_elem1151 != null) { - struct.values.add(_elem1135); + struct.values.add(_elem1151); } } iprot.readListEnd(); @@ -364,9 +364,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClientCapabilities oprot.writeFieldBegin(VALUES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size())); - for (ClientCapability _iter1137 : struct.values) + for (ClientCapability _iter1153 : struct.values) { - oprot.writeI32(_iter1137.getValue()); + oprot.writeI32(_iter1153.getValue()); } oprot.writeListEnd(); } @@ -391,9 +391,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.values.size()); - for (ClientCapability _iter1138 : struct.values) + for (ClientCapability _iter1154 : struct.values) { - oprot.writeI32(_iter1138.getValue()); + oprot.writeI32(_iter1154.getValue()); } } } @@ -402,15 +402,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1139 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32); - struct.values = new java.util.ArrayList(_list1139.size); - @org.apache.thrift.annotation.Nullable ClientCapability _elem1140; - for (int _i1141 = 0; _i1141 < _list1139.size; ++_i1141) + org.apache.thrift.protocol.TList _list1155 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32); + struct.values = new java.util.ArrayList(_list1155.size); + @org.apache.thrift.annotation.Nullable ClientCapability _elem1156; + for (int _i1157 = 0; _i1157 < _list1155.size; ++_i1157) { - _elem1140 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32()); - if (_elem1140 != null) + _elem1156 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32()); + if (_elem1156 != null) { - struct.values.add(_elem1140); + struct.values.add(_elem1156); } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateDatabaseRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateDatabaseRequest.java index 1fc11e50fce9..547def9882ff 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateDatabaseRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateDatabaseRequest.java @@ -1251,15 +1251,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateDatabaseReque case 4: // PARAMETERS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1350 = iprot.readMapBegin(); - struct.parameters = new java.util.HashMap(2*_map1350.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1351; - @org.apache.thrift.annotation.Nullable java.lang.String _val1352; - for (int _i1353 = 0; _i1353 < _map1350.size; ++_i1353) + org.apache.thrift.protocol.TMap _map1366 = iprot.readMapBegin(); + struct.parameters = new java.util.HashMap(2*_map1366.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1367; + @org.apache.thrift.annotation.Nullable java.lang.String _val1368; + for (int _i1369 = 0; _i1369 < _map1366.size; ++_i1369) { - _key1351 = iprot.readString(); - _val1352 = iprot.readString(); - struct.parameters.put(_key1351, _val1352); + _key1367 = iprot.readString(); + _val1368 = iprot.readString(); + struct.parameters.put(_key1367, _val1368); } iprot.readMapEnd(); } @@ -1370,10 +1370,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateDatabaseRequ oprot.writeFieldBegin(PARAMETERS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size())); - for (java.util.Map.Entry _iter1354 : struct.parameters.entrySet()) + for (java.util.Map.Entry _iter1370 : struct.parameters.entrySet()) { - oprot.writeString(_iter1354.getKey()); - oprot.writeString(_iter1354.getValue()); + oprot.writeString(_iter1370.getKey()); + oprot.writeString(_iter1370.getValue()); } oprot.writeMapEnd(); } @@ -1496,10 +1496,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, CreateDatabaseReque if (struct.isSetParameters()) { { oprot.writeI32(struct.parameters.size()); - for (java.util.Map.Entry _iter1355 : struct.parameters.entrySet()) + for (java.util.Map.Entry _iter1371 : struct.parameters.entrySet()) { - oprot.writeString(_iter1355.getKey()); - oprot.writeString(_iter1355.getValue()); + oprot.writeString(_iter1371.getKey()); + oprot.writeString(_iter1371.getValue()); } } } @@ -1545,15 +1545,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, CreateDatabaseReques } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map1356 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.parameters = new java.util.HashMap(2*_map1356.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1357; - @org.apache.thrift.annotation.Nullable java.lang.String _val1358; - for (int _i1359 = 0; _i1359 < _map1356.size; ++_i1359) + org.apache.thrift.protocol.TMap _map1372 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.parameters = new java.util.HashMap(2*_map1372.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1373; + @org.apache.thrift.annotation.Nullable java.lang.String _val1374; + for (int _i1375 = 0; _i1375 < _map1372.size; ++_i1375) { - _key1357 = iprot.readString(); - _val1358 = iprot.readString(); - struct.parameters.put(_key1357, _val1358); + _key1373 = iprot.readString(); + _val1374 = iprot.readString(); + struct.parameters.put(_key1373, _val1374); } } struct.setParametersIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateTableRequest.java index e85606916ef8..aa504e058e5a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateTableRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateTableRequest.java @@ -1206,14 +1206,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 3: // PRIMARY_KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1294 = iprot.readListBegin(); - struct.primaryKeys = new java.util.ArrayList(_list1294.size); - @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1295; - for (int _i1296 = 0; _i1296 < _list1294.size; ++_i1296) + org.apache.thrift.protocol.TList _list1310 = iprot.readListBegin(); + struct.primaryKeys = new java.util.ArrayList(_list1310.size); + @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1311; + for (int _i1312 = 0; _i1312 < _list1310.size; ++_i1312) { - _elem1295 = new SQLPrimaryKey(); - _elem1295.read(iprot); - struct.primaryKeys.add(_elem1295); + _elem1311 = new SQLPrimaryKey(); + _elem1311.read(iprot); + struct.primaryKeys.add(_elem1311); } iprot.readListEnd(); } @@ -1225,14 +1225,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 4: // FOREIGN_KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1297 = iprot.readListBegin(); - struct.foreignKeys = new java.util.ArrayList(_list1297.size); - @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1298; - for (int _i1299 = 0; _i1299 < _list1297.size; ++_i1299) + org.apache.thrift.protocol.TList _list1313 = iprot.readListBegin(); + struct.foreignKeys = new java.util.ArrayList(_list1313.size); + @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1314; + for (int _i1315 = 0; _i1315 < _list1313.size; ++_i1315) { - _elem1298 = new SQLForeignKey(); - _elem1298.read(iprot); - struct.foreignKeys.add(_elem1298); + _elem1314 = new SQLForeignKey(); + _elem1314.read(iprot); + struct.foreignKeys.add(_elem1314); } iprot.readListEnd(); } @@ -1244,14 +1244,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 5: // UNIQUE_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1300 = iprot.readListBegin(); - struct.uniqueConstraints = new java.util.ArrayList(_list1300.size); - @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1301; - for (int _i1302 = 0; _i1302 < _list1300.size; ++_i1302) + org.apache.thrift.protocol.TList _list1316 = iprot.readListBegin(); + struct.uniqueConstraints = new java.util.ArrayList(_list1316.size); + @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1317; + for (int _i1318 = 0; _i1318 < _list1316.size; ++_i1318) { - _elem1301 = new SQLUniqueConstraint(); - _elem1301.read(iprot); - struct.uniqueConstraints.add(_elem1301); + _elem1317 = new SQLUniqueConstraint(); + _elem1317.read(iprot); + struct.uniqueConstraints.add(_elem1317); } iprot.readListEnd(); } @@ -1263,14 +1263,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 6: // NOT_NULL_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1303 = iprot.readListBegin(); - struct.notNullConstraints = new java.util.ArrayList(_list1303.size); - @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1304; - for (int _i1305 = 0; _i1305 < _list1303.size; ++_i1305) + org.apache.thrift.protocol.TList _list1319 = iprot.readListBegin(); + struct.notNullConstraints = new java.util.ArrayList(_list1319.size); + @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1320; + for (int _i1321 = 0; _i1321 < _list1319.size; ++_i1321) { - _elem1304 = new SQLNotNullConstraint(); - _elem1304.read(iprot); - struct.notNullConstraints.add(_elem1304); + _elem1320 = new SQLNotNullConstraint(); + _elem1320.read(iprot); + struct.notNullConstraints.add(_elem1320); } iprot.readListEnd(); } @@ -1282,14 +1282,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 7: // DEFAULT_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1306 = iprot.readListBegin(); - struct.defaultConstraints = new java.util.ArrayList(_list1306.size); - @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1307; - for (int _i1308 = 0; _i1308 < _list1306.size; ++_i1308) + org.apache.thrift.protocol.TList _list1322 = iprot.readListBegin(); + struct.defaultConstraints = new java.util.ArrayList(_list1322.size); + @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1323; + for (int _i1324 = 0; _i1324 < _list1322.size; ++_i1324) { - _elem1307 = new SQLDefaultConstraint(); - _elem1307.read(iprot); - struct.defaultConstraints.add(_elem1307); + _elem1323 = new SQLDefaultConstraint(); + _elem1323.read(iprot); + struct.defaultConstraints.add(_elem1323); } iprot.readListEnd(); } @@ -1301,14 +1301,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 8: // CHECK_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1309 = iprot.readListBegin(); - struct.checkConstraints = new java.util.ArrayList(_list1309.size); - @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1310; - for (int _i1311 = 0; _i1311 < _list1309.size; ++_i1311) + org.apache.thrift.protocol.TList _list1325 = iprot.readListBegin(); + struct.checkConstraints = new java.util.ArrayList(_list1325.size); + @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1326; + for (int _i1327 = 0; _i1327 < _list1325.size; ++_i1327) { - _elem1310 = new SQLCheckConstraint(); - _elem1310.read(iprot); - struct.checkConstraints.add(_elem1310); + _elem1326 = new SQLCheckConstraint(); + _elem1326.read(iprot); + struct.checkConstraints.add(_elem1326); } iprot.readListEnd(); } @@ -1320,13 +1320,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, CreateTableRequest case 9: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1312 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1312.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1313; - for (int _i1314 = 0; _i1314 < _list1312.size; ++_i1314) + org.apache.thrift.protocol.TList _list1328 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1328.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1329; + for (int _i1330 = 0; _i1330 < _list1328.size; ++_i1330) { - _elem1313 = iprot.readString(); - struct.processorCapabilities.add(_elem1313); + _elem1329 = iprot.readString(); + struct.processorCapabilities.add(_elem1329); } iprot.readListEnd(); } @@ -1373,9 +1373,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size())); - for (SQLPrimaryKey _iter1315 : struct.primaryKeys) + for (SQLPrimaryKey _iter1331 : struct.primaryKeys) { - _iter1315.write(oprot); + _iter1331.write(oprot); } oprot.writeListEnd(); } @@ -1387,9 +1387,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size())); - for (SQLForeignKey _iter1316 : struct.foreignKeys) + for (SQLForeignKey _iter1332 : struct.foreignKeys) { - _iter1316.write(oprot); + _iter1332.write(oprot); } oprot.writeListEnd(); } @@ -1401,9 +1401,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size())); - for (SQLUniqueConstraint _iter1317 : struct.uniqueConstraints) + for (SQLUniqueConstraint _iter1333 : struct.uniqueConstraints) { - _iter1317.write(oprot); + _iter1333.write(oprot); } oprot.writeListEnd(); } @@ -1415,9 +1415,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size())); - for (SQLNotNullConstraint _iter1318 : struct.notNullConstraints) + for (SQLNotNullConstraint _iter1334 : struct.notNullConstraints) { - _iter1318.write(oprot); + _iter1334.write(oprot); } oprot.writeListEnd(); } @@ -1429,9 +1429,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size())); - for (SQLDefaultConstraint _iter1319 : struct.defaultConstraints) + for (SQLDefaultConstraint _iter1335 : struct.defaultConstraints) { - _iter1319.write(oprot); + _iter1335.write(oprot); } oprot.writeListEnd(); } @@ -1443,9 +1443,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size())); - for (SQLCheckConstraint _iter1320 : struct.checkConstraints) + for (SQLCheckConstraint _iter1336 : struct.checkConstraints) { - _iter1320.write(oprot); + _iter1336.write(oprot); } oprot.writeListEnd(); } @@ -1457,9 +1457,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, CreateTableRequest oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1321 : struct.processorCapabilities) + for (java.lang.String _iter1337 : struct.processorCapabilities) { - oprot.writeString(_iter1321); + oprot.writeString(_iter1337); } oprot.writeListEnd(); } @@ -1526,63 +1526,63 @@ public void write(org.apache.thrift.protocol.TProtocol prot, CreateTableRequest if (struct.isSetPrimaryKeys()) { { oprot.writeI32(struct.primaryKeys.size()); - for (SQLPrimaryKey _iter1322 : struct.primaryKeys) + for (SQLPrimaryKey _iter1338 : struct.primaryKeys) { - _iter1322.write(oprot); + _iter1338.write(oprot); } } } if (struct.isSetForeignKeys()) { { oprot.writeI32(struct.foreignKeys.size()); - for (SQLForeignKey _iter1323 : struct.foreignKeys) + for (SQLForeignKey _iter1339 : struct.foreignKeys) { - _iter1323.write(oprot); + _iter1339.write(oprot); } } } if (struct.isSetUniqueConstraints()) { { oprot.writeI32(struct.uniqueConstraints.size()); - for (SQLUniqueConstraint _iter1324 : struct.uniqueConstraints) + for (SQLUniqueConstraint _iter1340 : struct.uniqueConstraints) { - _iter1324.write(oprot); + _iter1340.write(oprot); } } } if (struct.isSetNotNullConstraints()) { { oprot.writeI32(struct.notNullConstraints.size()); - for (SQLNotNullConstraint _iter1325 : struct.notNullConstraints) + for (SQLNotNullConstraint _iter1341 : struct.notNullConstraints) { - _iter1325.write(oprot); + _iter1341.write(oprot); } } } if (struct.isSetDefaultConstraints()) { { oprot.writeI32(struct.defaultConstraints.size()); - for (SQLDefaultConstraint _iter1326 : struct.defaultConstraints) + for (SQLDefaultConstraint _iter1342 : struct.defaultConstraints) { - _iter1326.write(oprot); + _iter1342.write(oprot); } } } if (struct.isSetCheckConstraints()) { { oprot.writeI32(struct.checkConstraints.size()); - for (SQLCheckConstraint _iter1327 : struct.checkConstraints) + for (SQLCheckConstraint _iter1343 : struct.checkConstraints) { - _iter1327.write(oprot); + _iter1343.write(oprot); } } } if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1328 : struct.processorCapabilities) + for (java.lang.String _iter1344 : struct.processorCapabilities) { - oprot.writeString(_iter1328); + oprot.writeString(_iter1344); } } } @@ -1605,97 +1605,97 @@ public void read(org.apache.thrift.protocol.TProtocol prot, CreateTableRequest s } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1329 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.primaryKeys = new java.util.ArrayList(_list1329.size); - @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1330; - for (int _i1331 = 0; _i1331 < _list1329.size; ++_i1331) + org.apache.thrift.protocol.TList _list1345 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.primaryKeys = new java.util.ArrayList(_list1345.size); + @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1346; + for (int _i1347 = 0; _i1347 < _list1345.size; ++_i1347) { - _elem1330 = new SQLPrimaryKey(); - _elem1330.read(iprot); - struct.primaryKeys.add(_elem1330); + _elem1346 = new SQLPrimaryKey(); + _elem1346.read(iprot); + struct.primaryKeys.add(_elem1346); } } struct.setPrimaryKeysIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1332 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.foreignKeys = new java.util.ArrayList(_list1332.size); - @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1333; - for (int _i1334 = 0; _i1334 < _list1332.size; ++_i1334) + org.apache.thrift.protocol.TList _list1348 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.foreignKeys = new java.util.ArrayList(_list1348.size); + @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1349; + for (int _i1350 = 0; _i1350 < _list1348.size; ++_i1350) { - _elem1333 = new SQLForeignKey(); - _elem1333.read(iprot); - struct.foreignKeys.add(_elem1333); + _elem1349 = new SQLForeignKey(); + _elem1349.read(iprot); + struct.foreignKeys.add(_elem1349); } } struct.setForeignKeysIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list1335 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.uniqueConstraints = new java.util.ArrayList(_list1335.size); - @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1336; - for (int _i1337 = 0; _i1337 < _list1335.size; ++_i1337) + org.apache.thrift.protocol.TList _list1351 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.uniqueConstraints = new java.util.ArrayList(_list1351.size); + @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1352; + for (int _i1353 = 0; _i1353 < _list1351.size; ++_i1353) { - _elem1336 = new SQLUniqueConstraint(); - _elem1336.read(iprot); - struct.uniqueConstraints.add(_elem1336); + _elem1352 = new SQLUniqueConstraint(); + _elem1352.read(iprot); + struct.uniqueConstraints.add(_elem1352); } } struct.setUniqueConstraintsIsSet(true); } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1338 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.notNullConstraints = new java.util.ArrayList(_list1338.size); - @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1339; - for (int _i1340 = 0; _i1340 < _list1338.size; ++_i1340) + org.apache.thrift.protocol.TList _list1354 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.notNullConstraints = new java.util.ArrayList(_list1354.size); + @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1355; + for (int _i1356 = 0; _i1356 < _list1354.size; ++_i1356) { - _elem1339 = new SQLNotNullConstraint(); - _elem1339.read(iprot); - struct.notNullConstraints.add(_elem1339); + _elem1355 = new SQLNotNullConstraint(); + _elem1355.read(iprot); + struct.notNullConstraints.add(_elem1355); } } struct.setNotNullConstraintsIsSet(true); } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1341 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.defaultConstraints = new java.util.ArrayList(_list1341.size); - @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1342; - for (int _i1343 = 0; _i1343 < _list1341.size; ++_i1343) + org.apache.thrift.protocol.TList _list1357 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.defaultConstraints = new java.util.ArrayList(_list1357.size); + @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1358; + for (int _i1359 = 0; _i1359 < _list1357.size; ++_i1359) { - _elem1342 = new SQLDefaultConstraint(); - _elem1342.read(iprot); - struct.defaultConstraints.add(_elem1342); + _elem1358 = new SQLDefaultConstraint(); + _elem1358.read(iprot); + struct.defaultConstraints.add(_elem1358); } } struct.setDefaultConstraintsIsSet(true); } if (incoming.get(6)) { { - org.apache.thrift.protocol.TList _list1344 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.checkConstraints = new java.util.ArrayList(_list1344.size); - @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1345; - for (int _i1346 = 0; _i1346 < _list1344.size; ++_i1346) + org.apache.thrift.protocol.TList _list1360 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.checkConstraints = new java.util.ArrayList(_list1360.size); + @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1361; + for (int _i1362 = 0; _i1362 < _list1360.size; ++_i1362) { - _elem1345 = new SQLCheckConstraint(); - _elem1345.read(iprot); - struct.checkConstraints.add(_elem1345); + _elem1361 = new SQLCheckConstraint(); + _elem1361.read(iprot); + struct.checkConstraints.add(_elem1361); } } struct.setCheckConstraintsIsSet(true); } if (incoming.get(7)) { { - org.apache.thrift.protocol.TList _list1347 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1347.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1348; - for (int _i1349 = 0; _i1349 < _list1347.size; ++_i1349) + org.apache.thrift.protocol.TList _list1363 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1363.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1364; + for (int _i1365 = 0; _i1365 < _list1363.size; ++_i1365) { - _elem1348 = iprot.readString(); - struct.processorCapabilities.add(_elem1348); + _elem1364 = iprot.readString(); + struct.processorCapabilities.add(_elem1364); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ExtendedTableInfo.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ExtendedTableInfo.java index c2e387ae304a..7baa5515f975 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ExtendedTableInfo.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ExtendedTableInfo.java @@ -602,13 +602,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ExtendedTableInfo s case 3: // REQUIRED_READ_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1190 = iprot.readListBegin(); - struct.requiredReadCapabilities = new java.util.ArrayList(_list1190.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1191; - for (int _i1192 = 0; _i1192 < _list1190.size; ++_i1192) + org.apache.thrift.protocol.TList _list1206 = iprot.readListBegin(); + struct.requiredReadCapabilities = new java.util.ArrayList(_list1206.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1207; + for (int _i1208 = 0; _i1208 < _list1206.size; ++_i1208) { - _elem1191 = iprot.readString(); - struct.requiredReadCapabilities.add(_elem1191); + _elem1207 = iprot.readString(); + struct.requiredReadCapabilities.add(_elem1207); } iprot.readListEnd(); } @@ -620,13 +620,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ExtendedTableInfo s case 4: // REQUIRED_WRITE_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1193 = iprot.readListBegin(); - struct.requiredWriteCapabilities = new java.util.ArrayList(_list1193.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1194; - for (int _i1195 = 0; _i1195 < _list1193.size; ++_i1195) + org.apache.thrift.protocol.TList _list1209 = iprot.readListBegin(); + struct.requiredWriteCapabilities = new java.util.ArrayList(_list1209.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1210; + for (int _i1211 = 0; _i1211 < _list1209.size; ++_i1211) { - _elem1194 = iprot.readString(); - struct.requiredWriteCapabilities.add(_elem1194); + _elem1210 = iprot.readString(); + struct.requiredWriteCapabilities.add(_elem1210); } iprot.readListEnd(); } @@ -663,9 +663,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ExtendedTableInfo oprot.writeFieldBegin(REQUIRED_READ_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.requiredReadCapabilities.size())); - for (java.lang.String _iter1196 : struct.requiredReadCapabilities) + for (java.lang.String _iter1212 : struct.requiredReadCapabilities) { - oprot.writeString(_iter1196); + oprot.writeString(_iter1212); } oprot.writeListEnd(); } @@ -677,9 +677,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ExtendedTableInfo oprot.writeFieldBegin(REQUIRED_WRITE_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.requiredWriteCapabilities.size())); - for (java.lang.String _iter1197 : struct.requiredWriteCapabilities) + for (java.lang.String _iter1213 : struct.requiredWriteCapabilities) { - oprot.writeString(_iter1197); + oprot.writeString(_iter1213); } oprot.writeListEnd(); } @@ -721,18 +721,18 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ExtendedTableInfo s if (struct.isSetRequiredReadCapabilities()) { { oprot.writeI32(struct.requiredReadCapabilities.size()); - for (java.lang.String _iter1198 : struct.requiredReadCapabilities) + for (java.lang.String _iter1214 : struct.requiredReadCapabilities) { - oprot.writeString(_iter1198); + oprot.writeString(_iter1214); } } } if (struct.isSetRequiredWriteCapabilities()) { { oprot.writeI32(struct.requiredWriteCapabilities.size()); - for (java.lang.String _iter1199 : struct.requiredWriteCapabilities) + for (java.lang.String _iter1215 : struct.requiredWriteCapabilities) { - oprot.writeString(_iter1199); + oprot.writeString(_iter1215); } } } @@ -750,26 +750,26 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ExtendedTableInfo st } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1200 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.requiredReadCapabilities = new java.util.ArrayList(_list1200.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1201; - for (int _i1202 = 0; _i1202 < _list1200.size; ++_i1202) + org.apache.thrift.protocol.TList _list1216 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.requiredReadCapabilities = new java.util.ArrayList(_list1216.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1217; + for (int _i1218 = 0; _i1218 < _list1216.size; ++_i1218) { - _elem1201 = iprot.readString(); - struct.requiredReadCapabilities.add(_elem1201); + _elem1217 = iprot.readString(); + struct.requiredReadCapabilities.add(_elem1217); } } struct.setRequiredReadCapabilitiesIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1203 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.requiredWriteCapabilities = new java.util.ArrayList(_list1203.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1204; - for (int _i1205 = 0; _i1205 < _list1203.size; ++_i1205) + org.apache.thrift.protocol.TList _list1219 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.requiredWriteCapabilities = new java.util.ArrayList(_list1219.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1220; + for (int _i1221 = 0; _i1221 < _list1219.size; ++_i1221) { - _elem1204 = iprot.readString(); - struct.requiredWriteCapabilities.add(_elem1204); + _elem1220 = iprot.readString(); + struct.requiredWriteCapabilities.add(_elem1220); } } struct.setRequiredWriteCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java index 8ca2de700c13..fb635b12deed 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java @@ -325,14 +325,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, FindSchemasByColsRe case 1: // SCHEMA_VERSIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1286 = iprot.readListBegin(); - struct.schemaVersions = new java.util.ArrayList(_list1286.size); - @org.apache.thrift.annotation.Nullable SchemaVersionDescriptor _elem1287; - for (int _i1288 = 0; _i1288 < _list1286.size; ++_i1288) + org.apache.thrift.protocol.TList _list1302 = iprot.readListBegin(); + struct.schemaVersions = new java.util.ArrayList(_list1302.size); + @org.apache.thrift.annotation.Nullable SchemaVersionDescriptor _elem1303; + for (int _i1304 = 0; _i1304 < _list1302.size; ++_i1304) { - _elem1287 = new SchemaVersionDescriptor(); - _elem1287.read(iprot); - struct.schemaVersions.add(_elem1287); + _elem1303 = new SchemaVersionDescriptor(); + _elem1303.read(iprot); + struct.schemaVersions.add(_elem1303); } iprot.readListEnd(); } @@ -358,9 +358,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, FindSchemasByColsR oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size())); - for (SchemaVersionDescriptor _iter1289 : struct.schemaVersions) + for (SchemaVersionDescriptor _iter1305 : struct.schemaVersions) { - _iter1289.write(oprot); + _iter1305.write(oprot); } oprot.writeListEnd(); } @@ -391,9 +391,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, FindSchemasByColsRe if (struct.isSetSchemaVersions()) { { oprot.writeI32(struct.schemaVersions.size()); - for (SchemaVersionDescriptor _iter1290 : struct.schemaVersions) + for (SchemaVersionDescriptor _iter1306 : struct.schemaVersions) { - _iter1290.write(oprot); + _iter1306.write(oprot); } } } @@ -405,14 +405,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, FindSchemasByColsRes java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1291 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.schemaVersions = new java.util.ArrayList(_list1291.size); - @org.apache.thrift.annotation.Nullable SchemaVersionDescriptor _elem1292; - for (int _i1293 = 0; _i1293 < _list1291.size; ++_i1293) + org.apache.thrift.protocol.TList _list1307 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.schemaVersions = new java.util.ArrayList(_list1307.size); + @org.apache.thrift.annotation.Nullable SchemaVersionDescriptor _elem1308; + for (int _i1309 = 0; _i1309 < _list1307.size; ++_i1309) { - _elem1292 = new SchemaVersionDescriptor(); - _elem1292.read(iprot); - struct.schemaVersions.add(_elem1292); + _elem1308 = new SchemaVersionDescriptor(); + _elem1308.read(iprot); + struct.schemaVersions.add(_elem1308); } } struct.setSchemaVersionsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java index 266245d46bfe..11c6b73f337a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java @@ -861,13 +861,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, FireEventRequest st case 5: // PARTITION_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1024 = iprot.readListBegin(); - struct.partitionVals = new java.util.ArrayList(_list1024.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1025; - for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026) + org.apache.thrift.protocol.TList _list1040 = iprot.readListBegin(); + struct.partitionVals = new java.util.ArrayList(_list1040.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1041; + for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042) { - _elem1025 = iprot.readString(); - struct.partitionVals.add(_elem1025); + _elem1041 = iprot.readString(); + struct.partitionVals.add(_elem1041); } iprot.readListEnd(); } @@ -887,15 +887,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, FireEventRequest st case 7: // TBL_PARAMS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1027 = iprot.readMapBegin(); - struct.tblParams = new java.util.HashMap(2*_map1027.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1028; - @org.apache.thrift.annotation.Nullable java.lang.String _val1029; - for (int _i1030 = 0; _i1030 < _map1027.size; ++_i1030) + org.apache.thrift.protocol.TMap _map1043 = iprot.readMapBegin(); + struct.tblParams = new java.util.HashMap(2*_map1043.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1044; + @org.apache.thrift.annotation.Nullable java.lang.String _val1045; + for (int _i1046 = 0; _i1046 < _map1043.size; ++_i1046) { - _key1028 = iprot.readString(); - _val1029 = iprot.readString(); - struct.tblParams.put(_key1028, _val1029); + _key1044 = iprot.readString(); + _val1045 = iprot.readString(); + struct.tblParams.put(_key1044, _val1045); } iprot.readMapEnd(); } @@ -944,9 +944,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, FireEventRequest s oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size())); - for (java.lang.String _iter1031 : struct.partitionVals) + for (java.lang.String _iter1047 : struct.partitionVals) { - oprot.writeString(_iter1031); + oprot.writeString(_iter1047); } oprot.writeListEnd(); } @@ -965,10 +965,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, FireEventRequest s oprot.writeFieldBegin(TBL_PARAMS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.tblParams.size())); - for (java.util.Map.Entry _iter1032 : struct.tblParams.entrySet()) + for (java.util.Map.Entry _iter1048 : struct.tblParams.entrySet()) { - oprot.writeString(_iter1032.getKey()); - oprot.writeString(_iter1032.getValue()); + oprot.writeString(_iter1048.getKey()); + oprot.writeString(_iter1048.getValue()); } oprot.writeMapEnd(); } @@ -1020,9 +1020,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, FireEventRequest st if (struct.isSetPartitionVals()) { { oprot.writeI32(struct.partitionVals.size()); - for (java.lang.String _iter1033 : struct.partitionVals) + for (java.lang.String _iter1049 : struct.partitionVals) { - oprot.writeString(_iter1033); + oprot.writeString(_iter1049); } } } @@ -1032,10 +1032,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, FireEventRequest st if (struct.isSetTblParams()) { { oprot.writeI32(struct.tblParams.size()); - for (java.util.Map.Entry _iter1034 : struct.tblParams.entrySet()) + for (java.util.Map.Entry _iter1050 : struct.tblParams.entrySet()) { - oprot.writeString(_iter1034.getKey()); - oprot.writeString(_iter1034.getValue()); + oprot.writeString(_iter1050.getKey()); + oprot.writeString(_iter1050.getValue()); } } } @@ -1060,13 +1060,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, FireEventRequest str } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1035 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partitionVals = new java.util.ArrayList(_list1035.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1036; - for (int _i1037 = 0; _i1037 < _list1035.size; ++_i1037) + org.apache.thrift.protocol.TList _list1051 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partitionVals = new java.util.ArrayList(_list1051.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1052; + for (int _i1053 = 0; _i1053 < _list1051.size; ++_i1053) { - _elem1036 = iprot.readString(); - struct.partitionVals.add(_elem1036); + _elem1052 = iprot.readString(); + struct.partitionVals.add(_elem1052); } } struct.setPartitionValsIsSet(true); @@ -1077,15 +1077,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, FireEventRequest str } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map1038 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.tblParams = new java.util.HashMap(2*_map1038.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1039; - @org.apache.thrift.annotation.Nullable java.lang.String _val1040; - for (int _i1041 = 0; _i1041 < _map1038.size; ++_i1041) + org.apache.thrift.protocol.TMap _map1054 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.tblParams = new java.util.HashMap(2*_map1054.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1055; + @org.apache.thrift.annotation.Nullable java.lang.String _val1056; + for (int _i1057 = 0; _i1057 < _map1054.size; ++_i1057) { - _key1039 = iprot.readString(); - _val1040 = iprot.readString(); - struct.tblParams.put(_key1039, _val1040); + _key1055 = iprot.readString(); + _val1056 = iprot.readString(); + struct.tblParams.put(_key1055, _val1056); } } struct.setTblParamsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequestData.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequestData.java index 78b577936502..cf1c04767190 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequestData.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequestData.java @@ -170,14 +170,14 @@ protected java.lang.Object standardSchemeReadValue(org.apache.thrift.protocol.TP if (field.type == INSERT_DATAS_FIELD_DESC.type) { java.util.List insertDatas; { - org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin(); - insertDatas = new java.util.ArrayList(_list1016.size); - @org.apache.thrift.annotation.Nullable InsertEventRequestData _elem1017; - for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018) + org.apache.thrift.protocol.TList _list1032 = iprot.readListBegin(); + insertDatas = new java.util.ArrayList(_list1032.size); + @org.apache.thrift.annotation.Nullable InsertEventRequestData _elem1033; + for (int _i1034 = 0; _i1034 < _list1032.size; ++_i1034) { - _elem1017 = new InsertEventRequestData(); - _elem1017.read(iprot); - insertDatas.add(_elem1017); + _elem1033 = new InsertEventRequestData(); + _elem1033.read(iprot); + insertDatas.add(_elem1033); } iprot.readListEnd(); } @@ -215,9 +215,9 @@ protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol opr java.util.List insertDatas = (java.util.List)value_; { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, insertDatas.size())); - for (InsertEventRequestData _iter1019 : insertDatas) + for (InsertEventRequestData _iter1035 : insertDatas) { - _iter1019.write(oprot); + _iter1035.write(oprot); } oprot.writeListEnd(); } @@ -244,14 +244,14 @@ protected java.lang.Object tupleSchemeReadValue(org.apache.thrift.protocol.TProt case INSERT_DATAS: java.util.List insertDatas; { - org.apache.thrift.protocol.TList _list1020 = iprot.readListBegin(); - insertDatas = new java.util.ArrayList(_list1020.size); - @org.apache.thrift.annotation.Nullable InsertEventRequestData _elem1021; - for (int _i1022 = 0; _i1022 < _list1020.size; ++_i1022) + org.apache.thrift.protocol.TList _list1036 = iprot.readListBegin(); + insertDatas = new java.util.ArrayList(_list1036.size); + @org.apache.thrift.annotation.Nullable InsertEventRequestData _elem1037; + for (int _i1038 = 0; _i1038 < _list1036.size; ++_i1038) { - _elem1021 = new InsertEventRequestData(); - _elem1021.read(iprot); - insertDatas.add(_elem1021); + _elem1037 = new InsertEventRequestData(); + _elem1037.read(iprot); + insertDatas.add(_elem1037); } iprot.readListEnd(); } @@ -279,9 +279,9 @@ protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) java.util.List insertDatas = (java.util.List)value_; { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, insertDatas.size())); - for (InsertEventRequestData _iter1023 : insertDatas) + for (InsertEventRequestData _iter1039 : insertDatas) { - _iter1023.write(oprot); + _iter1039.write(oprot); } oprot.writeListEnd(); } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java index 7e4efe3d7218..ad91d4c362be 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventResponse.java @@ -322,13 +322,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, FireEventResponse s case 1: // EVENT_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin(); - struct.eventIds = new java.util.ArrayList(_list1042.size); - long _elem1043; - for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044) + org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin(); + struct.eventIds = new java.util.ArrayList(_list1058.size); + long _elem1059; + for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060) { - _elem1043 = iprot.readI64(); - struct.eventIds.add(_elem1043); + _elem1059 = iprot.readI64(); + struct.eventIds.add(_elem1059); } iprot.readListEnd(); } @@ -354,9 +354,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, FireEventResponse oprot.writeFieldBegin(EVENT_IDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.eventIds.size())); - for (long _iter1045 : struct.eventIds) + for (long _iter1061 : struct.eventIds) { - oprot.writeI64(_iter1045); + oprot.writeI64(_iter1061); } oprot.writeListEnd(); } @@ -387,9 +387,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, FireEventResponse s if (struct.isSetEventIds()) { { oprot.writeI32(struct.eventIds.size()); - for (long _iter1046 : struct.eventIds) + for (long _iter1062 : struct.eventIds) { - oprot.writeI64(_iter1046); + oprot.writeI64(_iter1062); } } } @@ -401,13 +401,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, FireEventResponse st java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1047 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); - struct.eventIds = new java.util.ArrayList(_list1047.size); - long _elem1048; - for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049) + org.apache.thrift.protocol.TList _list1063 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); + struct.eventIds = new java.util.ArrayList(_list1063.size); + long _elem1064; + for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065) { - _elem1048 = iprot.readI64(); - struct.eventIds.add(_elem1048); + _elem1064 = iprot.readI64(); + struct.eventIds.add(_elem1064); } } struct.setEventIdsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java index 04930c13288d..c6c8ada34b7d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java @@ -321,14 +321,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetAllFunctionsResp case 1: // FUNCTIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1126 = iprot.readListBegin(); - struct.functions = new java.util.ArrayList(_list1126.size); - @org.apache.thrift.annotation.Nullable Function _elem1127; - for (int _i1128 = 0; _i1128 < _list1126.size; ++_i1128) + org.apache.thrift.protocol.TList _list1142 = iprot.readListBegin(); + struct.functions = new java.util.ArrayList(_list1142.size); + @org.apache.thrift.annotation.Nullable Function _elem1143; + for (int _i1144 = 0; _i1144 < _list1142.size; ++_i1144) { - _elem1127 = new Function(); - _elem1127.read(iprot); - struct.functions.add(_elem1127); + _elem1143 = new Function(); + _elem1143.read(iprot); + struct.functions.add(_elem1143); } iprot.readListEnd(); } @@ -355,9 +355,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetAllFunctionsRes oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size())); - for (Function _iter1129 : struct.functions) + for (Function _iter1145 : struct.functions) { - _iter1129.write(oprot); + _iter1145.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetAllFunctionsResp if (struct.isSetFunctions()) { { oprot.writeI32(struct.functions.size()); - for (Function _iter1130 : struct.functions) + for (Function _iter1146 : struct.functions) { - _iter1130.write(oprot); + _iter1146.write(oprot); } } } @@ -403,14 +403,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetAllFunctionsRespo java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1131 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.functions = new java.util.ArrayList(_list1131.size); - @org.apache.thrift.annotation.Nullable Function _elem1132; - for (int _i1133 = 0; _i1133 < _list1131.size; ++_i1133) + org.apache.thrift.protocol.TList _list1147 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.functions = new java.util.ArrayList(_list1147.size); + @org.apache.thrift.annotation.Nullable Function _elem1148; + for (int _i1149 = 0; _i1149 < _list1147.size; ++_i1149) { - _elem1132 = new Function(); - _elem1132.read(iprot); - struct.functions.add(_elem1132); + _elem1148 = new Function(); + _elem1148.read(iprot); + struct.functions.add(_elem1148); } } struct.setFunctionsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetDatabaseRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetDatabaseRequest.java index 8191f69d6487..fd44969a5de0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetDatabaseRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetDatabaseRequest.java @@ -577,13 +577,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetDatabaseRequest case 3: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1206 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1206.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1207; - for (int _i1208 = 0; _i1208 < _list1206.size; ++_i1208) + org.apache.thrift.protocol.TList _list1222 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1222.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1223; + for (int _i1224 = 0; _i1224 < _list1222.size; ++_i1224) { - _elem1207 = iprot.readString(); - struct.processorCapabilities.add(_elem1207); + _elem1223 = iprot.readString(); + struct.processorCapabilities.add(_elem1223); } iprot.readListEnd(); } @@ -632,9 +632,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetDatabaseRequest oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1209 : struct.processorCapabilities) + for (java.lang.String _iter1225 : struct.processorCapabilities) { - oprot.writeString(_iter1209); + oprot.writeString(_iter1225); } oprot.writeListEnd(); } @@ -688,9 +688,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetDatabaseRequest if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1210 : struct.processorCapabilities) + for (java.lang.String _iter1226 : struct.processorCapabilities) { - oprot.writeString(_iter1210); + oprot.writeString(_iter1226); } } } @@ -713,13 +713,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetDatabaseRequest s } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1211 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1211.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1212; - for (int _i1213 = 0; _i1213 < _list1211.size; ++_i1213) + org.apache.thrift.protocol.TList _list1227 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1227.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1228; + for (int _i1229 = 0; _i1229 < _list1227.size; ++_i1229) { - _elem1212 = iprot.readString(); - struct.processorCapabilities.add(_elem1212); + _elem1228 = iprot.readString(); + struct.processorCapabilities.add(_elem1228); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFieldsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFieldsResponse.java index 91ed7f8f0123..d1a3bfa5614b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFieldsResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFieldsResponse.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetFieldsResponse s case 1: // FIELDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1424 = iprot.readListBegin(); - struct.fields = new java.util.ArrayList(_list1424.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1425; - for (int _i1426 = 0; _i1426 < _list1424.size; ++_i1426) + org.apache.thrift.protocol.TList _list1440 = iprot.readListBegin(); + struct.fields = new java.util.ArrayList(_list1440.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1441; + for (int _i1442 = 0; _i1442 < _list1440.size; ++_i1442) { - _elem1425 = new FieldSchema(); - _elem1425.read(iprot); - struct.fields.add(_elem1425); + _elem1441 = new FieldSchema(); + _elem1441.read(iprot); + struct.fields.add(_elem1441); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetFieldsResponse oprot.writeFieldBegin(FIELDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.fields.size())); - for (FieldSchema _iter1427 : struct.fields) + for (FieldSchema _iter1443 : struct.fields) { - _iter1427.write(oprot); + _iter1443.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFieldsResponse s org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.fields.size()); - for (FieldSchema _iter1428 : struct.fields) + for (FieldSchema _iter1444 : struct.fields) { - _iter1428.write(oprot); + _iter1444.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFieldsResponse s public void read(org.apache.thrift.protocol.TProtocol prot, GetFieldsResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1429 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.fields = new java.util.ArrayList(_list1429.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1430; - for (int _i1431 = 0; _i1431 < _list1429.size; ++_i1431) + org.apache.thrift.protocol.TList _list1445 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.fields = new java.util.ArrayList(_list1445.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1446; + for (int _i1447 = 0; _i1447 < _list1445.size; ++_i1447) { - _elem1430 = new FieldSchema(); - _elem1430.read(iprot); - struct.fields.add(_elem1430); + _elem1446 = new FieldSchema(); + _elem1446.read(iprot); + struct.fields.add(_elem1446); } } struct.setFieldsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java index 688ff30d8343..0d2afc82449b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java @@ -596,13 +596,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataByEx case 1: // FILE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1076 = iprot.readListBegin(); - struct.fileIds = new java.util.ArrayList(_list1076.size); - long _elem1077; - for (int _i1078 = 0; _i1078 < _list1076.size; ++_i1078) + org.apache.thrift.protocol.TList _list1092 = iprot.readListBegin(); + struct.fileIds = new java.util.ArrayList(_list1092.size); + long _elem1093; + for (int _i1094 = 0; _i1094 < _list1092.size; ++_i1094) { - _elem1077 = iprot.readI64(); - struct.fileIds.add(_elem1077); + _elem1093 = iprot.readI64(); + struct.fileIds.add(_elem1093); } iprot.readListEnd(); } @@ -652,9 +652,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataByE oprot.writeFieldBegin(FILE_IDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size())); - for (long _iter1079 : struct.fileIds) + for (long _iter1095 : struct.fileIds) { - oprot.writeI64(_iter1079); + oprot.writeI64(_iter1095); } oprot.writeListEnd(); } @@ -696,9 +696,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByEx org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.fileIds.size()); - for (long _iter1080 : struct.fileIds) + for (long _iter1096 : struct.fileIds) { - oprot.writeI64(_iter1080); + oprot.writeI64(_iter1096); } } oprot.writeBinary(struct.expr); @@ -722,13 +722,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByEx public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1081 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); - struct.fileIds = new java.util.ArrayList(_list1081.size); - long _elem1082; - for (int _i1083 = 0; _i1083 < _list1081.size; ++_i1083) + org.apache.thrift.protocol.TList _list1097 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); + struct.fileIds = new java.util.ArrayList(_list1097.size); + long _elem1098; + for (int _i1099 = 0; _i1099 < _list1097.size; ++_i1099) { - _elem1082 = iprot.readI64(); - struct.fileIds.add(_elem1082); + _elem1098 = iprot.readI64(); + struct.fileIds.add(_elem1098); } } struct.setFileIdsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java index 4fbb948361d7..92dc0dce4274 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java @@ -415,16 +415,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataByEx case 1: // METADATA if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1066 = iprot.readMapBegin(); - struct.metadata = new java.util.HashMap(2*_map1066.size); - long _key1067; - @org.apache.thrift.annotation.Nullable MetadataPpdResult _val1068; - for (int _i1069 = 0; _i1069 < _map1066.size; ++_i1069) + org.apache.thrift.protocol.TMap _map1082 = iprot.readMapBegin(); + struct.metadata = new java.util.HashMap(2*_map1082.size); + long _key1083; + @org.apache.thrift.annotation.Nullable MetadataPpdResult _val1084; + for (int _i1085 = 0; _i1085 < _map1082.size; ++_i1085) { - _key1067 = iprot.readI64(); - _val1068 = new MetadataPpdResult(); - _val1068.read(iprot); - struct.metadata.put(_key1067, _val1068); + _key1083 = iprot.readI64(); + _val1084 = new MetadataPpdResult(); + _val1084.read(iprot); + struct.metadata.put(_key1083, _val1084); } iprot.readMapEnd(); } @@ -458,10 +458,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataByE oprot.writeFieldBegin(METADATA_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size())); - for (java.util.Map.Entry _iter1070 : struct.metadata.entrySet()) + for (java.util.Map.Entry _iter1086 : struct.metadata.entrySet()) { - oprot.writeI64(_iter1070.getKey()); - _iter1070.getValue().write(oprot); + oprot.writeI64(_iter1086.getKey()); + _iter1086.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -489,10 +489,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByEx org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.metadata.size()); - for (java.util.Map.Entry _iter1071 : struct.metadata.entrySet()) + for (java.util.Map.Entry _iter1087 : struct.metadata.entrySet()) { - oprot.writeI64(_iter1071.getKey()); - _iter1071.getValue().write(oprot); + oprot.writeI64(_iter1087.getKey()); + _iter1087.getValue().write(oprot); } } oprot.writeBool(struct.isSupported); @@ -502,16 +502,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByEx public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map1072 = iprot.readMapBegin(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT); - struct.metadata = new java.util.HashMap(2*_map1072.size); - long _key1073; - @org.apache.thrift.annotation.Nullable MetadataPpdResult _val1074; - for (int _i1075 = 0; _i1075 < _map1072.size; ++_i1075) + org.apache.thrift.protocol.TMap _map1088 = iprot.readMapBegin(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT); + struct.metadata = new java.util.HashMap(2*_map1088.size); + long _key1089; + @org.apache.thrift.annotation.Nullable MetadataPpdResult _val1090; + for (int _i1091 = 0; _i1091 < _map1088.size; ++_i1091) { - _key1073 = iprot.readI64(); - _val1074 = new MetadataPpdResult(); - _val1074.read(iprot); - struct.metadata.put(_key1073, _val1074); + _key1089 = iprot.readI64(); + _val1090 = new MetadataPpdResult(); + _val1090.read(iprot); + struct.metadata.put(_key1089, _val1090); } } struct.setMetadataIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java index 28248f1e6f83..261e1449e9bc 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java @@ -326,13 +326,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataRequ case 1: // FILE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1094 = iprot.readListBegin(); - struct.fileIds = new java.util.ArrayList(_list1094.size); - long _elem1095; - for (int _i1096 = 0; _i1096 < _list1094.size; ++_i1096) + org.apache.thrift.protocol.TList _list1110 = iprot.readListBegin(); + struct.fileIds = new java.util.ArrayList(_list1110.size); + long _elem1111; + for (int _i1112 = 0; _i1112 < _list1110.size; ++_i1112) { - _elem1095 = iprot.readI64(); - struct.fileIds.add(_elem1095); + _elem1111 = iprot.readI64(); + struct.fileIds.add(_elem1111); } iprot.readListEnd(); } @@ -358,9 +358,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataReq oprot.writeFieldBegin(FILE_IDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size())); - for (long _iter1097 : struct.fileIds) + for (long _iter1113 : struct.fileIds) { - oprot.writeI64(_iter1097); + oprot.writeI64(_iter1113); } oprot.writeListEnd(); } @@ -385,9 +385,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.fileIds.size()); - for (long _iter1098 : struct.fileIds) + for (long _iter1114 : struct.fileIds) { - oprot.writeI64(_iter1098); + oprot.writeI64(_iter1114); } } } @@ -396,13 +396,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequ public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1099 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); - struct.fileIds = new java.util.ArrayList(_list1099.size); - long _elem1100; - for (int _i1101 = 0; _i1101 < _list1099.size; ++_i1101) + org.apache.thrift.protocol.TList _list1115 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); + struct.fileIds = new java.util.ArrayList(_list1115.size); + long _elem1116; + for (int _i1117 = 0; _i1117 < _list1115.size; ++_i1117) { - _elem1100 = iprot.readI64(); - struct.fileIds.add(_elem1100); + _elem1116 = iprot.readI64(); + struct.fileIds.add(_elem1116); } } struct.setFileIdsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java index 14eb9603ab4b..0480c4caee6f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java @@ -404,15 +404,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetFileMetadataResu case 1: // METADATA if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1084 = iprot.readMapBegin(); - struct.metadata = new java.util.HashMap(2*_map1084.size); - long _key1085; - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _val1086; - for (int _i1087 = 0; _i1087 < _map1084.size; ++_i1087) + org.apache.thrift.protocol.TMap _map1100 = iprot.readMapBegin(); + struct.metadata = new java.util.HashMap(2*_map1100.size); + long _key1101; + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _val1102; + for (int _i1103 = 0; _i1103 < _map1100.size; ++_i1103) { - _key1085 = iprot.readI64(); - _val1086 = iprot.readBinary(); - struct.metadata.put(_key1085, _val1086); + _key1101 = iprot.readI64(); + _val1102 = iprot.readBinary(); + struct.metadata.put(_key1101, _val1102); } iprot.readMapEnd(); } @@ -446,10 +446,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetFileMetadataRes oprot.writeFieldBegin(METADATA_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size())); - for (java.util.Map.Entry _iter1088 : struct.metadata.entrySet()) + for (java.util.Map.Entry _iter1104 : struct.metadata.entrySet()) { - oprot.writeI64(_iter1088.getKey()); - oprot.writeBinary(_iter1088.getValue()); + oprot.writeI64(_iter1104.getKey()); + oprot.writeBinary(_iter1104.getValue()); } oprot.writeMapEnd(); } @@ -477,10 +477,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResu org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.metadata.size()); - for (java.util.Map.Entry _iter1089 : struct.metadata.entrySet()) + for (java.util.Map.Entry _iter1105 : struct.metadata.entrySet()) { - oprot.writeI64(_iter1089.getKey()); - oprot.writeBinary(_iter1089.getValue()); + oprot.writeI64(_iter1105.getKey()); + oprot.writeBinary(_iter1105.getValue()); } } oprot.writeBool(struct.isSupported); @@ -490,15 +490,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResu public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map1090 = iprot.readMapBegin(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING); - struct.metadata = new java.util.HashMap(2*_map1090.size); - long _key1091; - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _val1092; - for (int _i1093 = 0; _i1093 < _map1090.size; ++_i1093) + org.apache.thrift.protocol.TMap _map1106 = iprot.readMapBegin(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING); + struct.metadata = new java.util.HashMap(2*_map1106.size); + long _key1107; + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _val1108; + for (int _i1109 = 0; _i1109 < _map1106.size; ++_i1109) { - _key1091 = iprot.readI64(); - _val1092 = iprot.readBinary(); - struct.metadata.put(_key1091, _val1092); + _key1107 = iprot.readI64(); + _val1108 = iprot.readBinary(); + struct.metadata.put(_key1107, _val1108); } } struct.setMetadataIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java index 267f983400ab..9e3e03c5f308 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java @@ -321,15 +321,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetOpenTxnsRequest case 1: // EXCLUDE_TXN_TYPES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1504 = iprot.readListBegin(); - struct.excludeTxnTypes = new java.util.ArrayList(_list1504.size); - @org.apache.thrift.annotation.Nullable TxnType _elem1505; - for (int _i1506 = 0; _i1506 < _list1504.size; ++_i1506) + org.apache.thrift.protocol.TList _list1520 = iprot.readListBegin(); + struct.excludeTxnTypes = new java.util.ArrayList(_list1520.size); + @org.apache.thrift.annotation.Nullable TxnType _elem1521; + for (int _i1522 = 0; _i1522 < _list1520.size; ++_i1522) { - _elem1505 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); - if (_elem1505 != null) + _elem1521 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); + if (_elem1521 != null) { - struct.excludeTxnTypes.add(_elem1505); + struct.excludeTxnTypes.add(_elem1521); } } iprot.readListEnd(); @@ -357,9 +357,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetOpenTxnsRequest oprot.writeFieldBegin(EXCLUDE_TXN_TYPES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.excludeTxnTypes.size())); - for (TxnType _iter1507 : struct.excludeTxnTypes) + for (TxnType _iter1523 : struct.excludeTxnTypes) { - oprot.writeI32(_iter1507.getValue()); + oprot.writeI32(_iter1523.getValue()); } oprot.writeListEnd(); } @@ -391,9 +391,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsRequest if (struct.isSetExcludeTxnTypes()) { { oprot.writeI32(struct.excludeTxnTypes.size()); - for (TxnType _iter1508 : struct.excludeTxnTypes) + for (TxnType _iter1524 : struct.excludeTxnTypes) { - oprot.writeI32(_iter1508.getValue()); + oprot.writeI32(_iter1524.getValue()); } } } @@ -405,15 +405,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsRequest s java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1509 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32); - struct.excludeTxnTypes = new java.util.ArrayList(_list1509.size); - @org.apache.thrift.annotation.Nullable TxnType _elem1510; - for (int _i1511 = 0; _i1511 < _list1509.size; ++_i1511) + org.apache.thrift.protocol.TList _list1525 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32); + struct.excludeTxnTypes = new java.util.ArrayList(_list1525.size); + @org.apache.thrift.annotation.Nullable TxnType _elem1526; + for (int _i1527 = 0; _i1527 < _list1525.size; ++_i1527) { - _elem1510 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); - if (_elem1510 != null) + _elem1526 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); + if (_elem1526 != null) { - struct.excludeTxnTypes.add(_elem1510); + struct.excludeTxnTypes.add(_elem1526); } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsRequest.java index dd0ea44e27b8..d59ffc0d8a6b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsRequest.java @@ -837,13 +837,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionNamesPs case 4: // PART_VALUES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1456 = iprot.readListBegin(); - struct.partValues = new java.util.ArrayList(_list1456.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1457; - for (int _i1458 = 0; _i1458 < _list1456.size; ++_i1458) + org.apache.thrift.protocol.TList _list1472 = iprot.readListBegin(); + struct.partValues = new java.util.ArrayList(_list1472.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1473; + for (int _i1474 = 0; _i1474 < _list1472.size; ++_i1474) { - _elem1457 = iprot.readString(); - struct.partValues.add(_elem1457); + _elem1473 = iprot.readString(); + struct.partValues.add(_elem1473); } iprot.readListEnd(); } @@ -911,9 +911,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionNamesP oprot.writeFieldBegin(PART_VALUES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partValues.size())); - for (java.lang.String _iter1459 : struct.partValues) + for (java.lang.String _iter1475 : struct.partValues) { - oprot.writeString(_iter1459); + oprot.writeString(_iter1475); } oprot.writeListEnd(); } @@ -979,9 +979,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionNamesPs if (struct.isSetPartValues()) { { oprot.writeI32(struct.partValues.size()); - for (java.lang.String _iter1460 : struct.partValues) + for (java.lang.String _iter1476 : struct.partValues) { - oprot.writeString(_iter1460); + oprot.writeString(_iter1476); } } } @@ -1010,13 +1010,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionNamesPsR } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1461 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partValues = new java.util.ArrayList(_list1461.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1462; - for (int _i1463 = 0; _i1463 < _list1461.size; ++_i1463) + org.apache.thrift.protocol.TList _list1477 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partValues = new java.util.ArrayList(_list1477.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1478; + for (int _i1479 = 0; _i1479 < _list1477.size; ++_i1479) { - _elem1462 = iprot.readString(); - struct.partValues.add(_elem1462); + _elem1478 = iprot.readString(); + struct.partValues.add(_elem1478); } } struct.setPartValuesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsResponse.java index 819992eab989..5e74d024ba47 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionNamesPsResponse.java @@ -326,13 +326,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionNamesPs case 1: // NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1464 = iprot.readListBegin(); - struct.names = new java.util.ArrayList(_list1464.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1465; - for (int _i1466 = 0; _i1466 < _list1464.size; ++_i1466) + org.apache.thrift.protocol.TList _list1480 = iprot.readListBegin(); + struct.names = new java.util.ArrayList(_list1480.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1481; + for (int _i1482 = 0; _i1482 < _list1480.size; ++_i1482) { - _elem1465 = iprot.readString(); - struct.names.add(_elem1465); + _elem1481 = iprot.readString(); + struct.names.add(_elem1481); } iprot.readListEnd(); } @@ -358,9 +358,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionNamesP oprot.writeFieldBegin(NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size())); - for (java.lang.String _iter1467 : struct.names) + for (java.lang.String _iter1483 : struct.names) { - oprot.writeString(_iter1467); + oprot.writeString(_iter1483); } oprot.writeListEnd(); } @@ -385,9 +385,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionNamesPs org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.names.size()); - for (java.lang.String _iter1468 : struct.names) + for (java.lang.String _iter1484 : struct.names) { - oprot.writeString(_iter1468); + oprot.writeString(_iter1484); } } } @@ -396,13 +396,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionNamesPs public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionNamesPsResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1469 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.names = new java.util.ArrayList(_list1469.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1470; - for (int _i1471 = 0; _i1471 < _list1469.size; ++_i1471) + org.apache.thrift.protocol.TList _list1485 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.names = new java.util.ArrayList(_list1485.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1486; + for (int _i1487 = 0; _i1487 < _list1485.size; ++_i1487) { - _elem1470 = iprot.readString(); - struct.names.add(_elem1470); + _elem1486 = iprot.readString(); + struct.names.add(_elem1486); } } struct.setNamesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionRequest.java index d26326134eab..af97699162c5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionRequest.java @@ -764,13 +764,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionRequest case 4: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1440 = iprot.readListBegin(); - struct.partVals = new java.util.ArrayList(_list1440.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1441; - for (int _i1442 = 0; _i1442 < _list1440.size; ++_i1442) + org.apache.thrift.protocol.TList _list1456 = iprot.readListBegin(); + struct.partVals = new java.util.ArrayList(_list1456.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1457; + for (int _i1458 = 0; _i1458 < _list1456.size; ++_i1458) { - _elem1441 = iprot.readString(); - struct.partVals.add(_elem1441); + _elem1457 = iprot.readString(); + struct.partVals.add(_elem1457); } iprot.readListEnd(); } @@ -829,9 +829,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionReques oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partVals.size())); - for (java.lang.String _iter1443 : struct.partVals) + for (java.lang.String _iter1459 : struct.partVals) { - oprot.writeString(_iter1443); + oprot.writeString(_iter1459); } oprot.writeListEnd(); } @@ -870,9 +870,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionRequest oprot.writeString(struct.tblName); { oprot.writeI32(struct.partVals.size()); - for (java.lang.String _iter1444 : struct.partVals) + for (java.lang.String _iter1460 : struct.partVals) { - oprot.writeString(_iter1444); + oprot.writeString(_iter1460); } } java.util.BitSet optionals = new java.util.BitSet(); @@ -905,13 +905,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionRequest struct.tblName = iprot.readString(); struct.setTblNameIsSet(true); { - org.apache.thrift.protocol.TList _list1445 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partVals = new java.util.ArrayList(_list1445.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1446; - for (int _i1447 = 0; _i1447 < _list1445.size; ++_i1447) + org.apache.thrift.protocol.TList _list1461 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partVals = new java.util.ArrayList(_list1461.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1462; + for (int _i1463 = 0; _i1463 < _list1461.size; ++_i1463) { - _elem1446 = iprot.readString(); - struct.partVals.add(_elem1446); + _elem1462 = iprot.readString(); + struct.partVals.add(_elem1462); } } struct.setPartValsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java index 764b3a78f295..6587d8eb42b5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java @@ -419,13 +419,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsFilter case 8: // FILTERS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1392 = iprot.readListBegin(); - struct.filters = new java.util.ArrayList(_list1392.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1393; - for (int _i1394 = 0; _i1394 < _list1392.size; ++_i1394) + org.apache.thrift.protocol.TList _list1408 = iprot.readListBegin(); + struct.filters = new java.util.ArrayList(_list1408.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1409; + for (int _i1410 = 0; _i1410 < _list1408.size; ++_i1410) { - _elem1393 = iprot.readString(); - struct.filters.add(_elem1393); + _elem1409 = iprot.readString(); + struct.filters.add(_elem1409); } iprot.readListEnd(); } @@ -459,9 +459,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsFilte oprot.writeFieldBegin(FILTERS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filters.size())); - for (java.lang.String _iter1395 : struct.filters) + for (java.lang.String _iter1411 : struct.filters) { - oprot.writeString(_iter1395); + oprot.writeString(_iter1411); } oprot.writeListEnd(); } @@ -499,9 +499,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsFilter if (struct.isSetFilters()) { { oprot.writeI32(struct.filters.size()); - for (java.lang.String _iter1396 : struct.filters) + for (java.lang.String _iter1412 : struct.filters) { - oprot.writeString(_iter1396); + oprot.writeString(_iter1412); } } } @@ -517,13 +517,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsFilterS } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1397 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.filters = new java.util.ArrayList(_list1397.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1398; - for (int _i1399 = 0; _i1399 < _list1397.size; ++_i1399) + org.apache.thrift.protocol.TList _list1413 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.filters = new java.util.ArrayList(_list1413.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1414; + for (int _i1415 = 0; _i1415 < _list1413.size; ++_i1415) { - _elem1398 = iprot.readString(); - struct.filters.add(_elem1398); + _elem1414 = iprot.readString(); + struct.filters.add(_elem1414); } } struct.setFiltersIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java index d23de0137882..865f3fe36646 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java @@ -1092,13 +1092,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsPsWith case 4: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1472 = iprot.readListBegin(); - struct.partVals = new java.util.ArrayList(_list1472.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1473; - for (int _i1474 = 0; _i1474 < _list1472.size; ++_i1474) + org.apache.thrift.protocol.TList _list1488 = iprot.readListBegin(); + struct.partVals = new java.util.ArrayList(_list1488.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1489; + for (int _i1490 = 0; _i1490 < _list1488.size; ++_i1490) { - _elem1473 = iprot.readString(); - struct.partVals.add(_elem1473); + _elem1489 = iprot.readString(); + struct.partVals.add(_elem1489); } iprot.readListEnd(); } @@ -1126,13 +1126,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsPsWith case 7: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1475 = iprot.readListBegin(); - struct.groupNames = new java.util.ArrayList(_list1475.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1476; - for (int _i1477 = 0; _i1477 < _list1475.size; ++_i1477) + org.apache.thrift.protocol.TList _list1491 = iprot.readListBegin(); + struct.groupNames = new java.util.ArrayList(_list1491.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1492; + for (int _i1493 = 0; _i1493 < _list1491.size; ++_i1493) { - _elem1476 = iprot.readString(); - struct.groupNames.add(_elem1476); + _elem1492 = iprot.readString(); + struct.groupNames.add(_elem1492); } iprot.readListEnd(); } @@ -1200,9 +1200,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partVals.size())); - for (java.lang.String _iter1478 : struct.partVals) + for (java.lang.String _iter1494 : struct.partVals) { - oprot.writeString(_iter1478); + oprot.writeString(_iter1494); } oprot.writeListEnd(); } @@ -1226,9 +1226,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groupNames.size())); - for (java.lang.String _iter1479 : struct.groupNames) + for (java.lang.String _iter1495 : struct.groupNames) { - oprot.writeString(_iter1479); + oprot.writeString(_iter1495); } oprot.writeListEnd(); } @@ -1303,9 +1303,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetPartVals()) { { oprot.writeI32(struct.partVals.size()); - for (java.lang.String _iter1480 : struct.partVals) + for (java.lang.String _iter1496 : struct.partVals) { - oprot.writeString(_iter1480); + oprot.writeString(_iter1496); } } } @@ -1318,9 +1318,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetGroupNames()) { { oprot.writeI32(struct.groupNames.size()); - for (java.lang.String _iter1481 : struct.groupNames) + for (java.lang.String _iter1497 : struct.groupNames) { - oprot.writeString(_iter1481); + oprot.writeString(_iter1497); } } } @@ -1349,13 +1349,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1482 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partVals = new java.util.ArrayList(_list1482.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1483; - for (int _i1484 = 0; _i1484 < _list1482.size; ++_i1484) + org.apache.thrift.protocol.TList _list1498 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partVals = new java.util.ArrayList(_list1498.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1499; + for (int _i1500 = 0; _i1500 < _list1498.size; ++_i1500) { - _elem1483 = iprot.readString(); - struct.partVals.add(_elem1483); + _elem1499 = iprot.readString(); + struct.partVals.add(_elem1499); } } struct.setPartValsIsSet(true); @@ -1370,13 +1370,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1485 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.groupNames = new java.util.ArrayList(_list1485.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1486; - for (int _i1487 = 0; _i1487 < _list1485.size; ++_i1487) + org.apache.thrift.protocol.TList _list1501 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.groupNames = new java.util.ArrayList(_list1501.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1502; + for (int _i1503 = 0; _i1503 < _list1501.size; ++_i1503) { - _elem1486 = iprot.readString(); - struct.groupNames.add(_elem1486); + _elem1502 = iprot.readString(); + struct.groupNames.add(_elem1502); } } struct.setGroupNamesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java index 59a3a65b2c85..fa839ad0470d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsPsWith case 1: // PARTITIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1488 = iprot.readListBegin(); - struct.partitions = new java.util.ArrayList(_list1488.size); - @org.apache.thrift.annotation.Nullable Partition _elem1489; - for (int _i1490 = 0; _i1490 < _list1488.size; ++_i1490) + org.apache.thrift.protocol.TList _list1504 = iprot.readListBegin(); + struct.partitions = new java.util.ArrayList(_list1504.size); + @org.apache.thrift.annotation.Nullable Partition _elem1505; + for (int _i1506 = 0; _i1506 < _list1504.size; ++_i1506) { - _elem1489 = new Partition(); - _elem1489.read(iprot); - struct.partitions.add(_elem1489); + _elem1505 = new Partition(); + _elem1505.read(iprot); + struct.partitions.add(_elem1505); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldBegin(PARTITIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size())); - for (Partition _iter1491 : struct.partitions) + for (Partition _iter1507 : struct.partitions) { - _iter1491.write(oprot); + _iter1507.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.partitions.size()); - for (Partition _iter1492 : struct.partitions) + for (Partition _iter1508 : struct.partitions) { - _iter1492.write(oprot); + _iter1508.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithAuthResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1493 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.partitions = new java.util.ArrayList(_list1493.size); - @org.apache.thrift.annotation.Nullable Partition _elem1494; - for (int _i1495 = 0; _i1495 < _list1493.size; ++_i1495) + org.apache.thrift.protocol.TList _list1509 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitions = new java.util.ArrayList(_list1509.size); + @org.apache.thrift.annotation.Nullable Partition _elem1510; + for (int _i1511 = 0; _i1511 < _list1509.size; ++_i1511) { - _elem1494 = new Partition(); - _elem1494.read(iprot); - struct.partitions.add(_elem1494); + _elem1510 = new Partition(); + _elem1510.read(iprot); + struct.partitions.add(_elem1510); } } struct.setPartitionsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java index 095fe315ec4b..3aa6606ddc59 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java @@ -1195,13 +1195,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsReques case 6: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1408 = iprot.readListBegin(); - struct.groupNames = new java.util.ArrayList(_list1408.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1409; - for (int _i1410 = 0; _i1410 < _list1408.size; ++_i1410) + org.apache.thrift.protocol.TList _list1424 = iprot.readListBegin(); + struct.groupNames = new java.util.ArrayList(_list1424.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1425; + for (int _i1426 = 0; _i1426 < _list1424.size; ++_i1426) { - _elem1409 = iprot.readString(); - struct.groupNames.add(_elem1409); + _elem1425 = iprot.readString(); + struct.groupNames.add(_elem1425); } iprot.readListEnd(); } @@ -1231,13 +1231,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsReques case 9: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1411 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1411.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1412; - for (int _i1413 = 0; _i1413 < _list1411.size; ++_i1413) + org.apache.thrift.protocol.TList _list1427 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1427.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1428; + for (int _i1429 = 0; _i1429 < _list1427.size; ++_i1429) { - _elem1412 = iprot.readString(); - struct.processorCapabilities.add(_elem1412); + _elem1428 = iprot.readString(); + struct.processorCapabilities.add(_elem1428); } iprot.readListEnd(); } @@ -1309,9 +1309,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsReque oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groupNames.size())); - for (java.lang.String _iter1414 : struct.groupNames) + for (java.lang.String _iter1430 : struct.groupNames) { - oprot.writeString(_iter1414); + oprot.writeString(_iter1430); } oprot.writeListEnd(); } @@ -1333,9 +1333,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsReque oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1415 : struct.processorCapabilities) + for (java.lang.String _iter1431 : struct.processorCapabilities) { - oprot.writeString(_iter1415); + oprot.writeString(_iter1431); } oprot.writeListEnd(); } @@ -1426,9 +1426,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsReques if (struct.isSetGroupNames()) { { oprot.writeI32(struct.groupNames.size()); - for (java.lang.String _iter1416 : struct.groupNames) + for (java.lang.String _iter1432 : struct.groupNames) { - oprot.writeString(_iter1416); + oprot.writeString(_iter1432); } } } @@ -1441,9 +1441,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsReques if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1417 : struct.processorCapabilities) + for (java.lang.String _iter1433 : struct.processorCapabilities) { - oprot.writeString(_iter1417); + oprot.writeString(_iter1433); } } } @@ -1481,13 +1481,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsRequest } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1418 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.groupNames = new java.util.ArrayList(_list1418.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1419; - for (int _i1420 = 0; _i1420 < _list1418.size; ++_i1420) + org.apache.thrift.protocol.TList _list1434 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.groupNames = new java.util.ArrayList(_list1434.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1435; + for (int _i1436 = 0; _i1436 < _list1434.size; ++_i1436) { - _elem1419 = iprot.readString(); - struct.groupNames.add(_elem1419); + _elem1435 = iprot.readString(); + struct.groupNames.add(_elem1435); } } struct.setGroupNamesIsSet(true); @@ -1504,13 +1504,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsRequest } if (incoming.get(8)) { { - org.apache.thrift.protocol.TList _list1421 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1421.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1422; - for (int _i1423 = 0; _i1423 < _list1421.size; ++_i1423) + org.apache.thrift.protocol.TList _list1437 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1437.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1438; + for (int _i1439 = 0; _i1439 < _list1437.size; ++_i1439) { - _elem1422 = iprot.readString(); - struct.processorCapabilities.add(_elem1422); + _elem1438 = iprot.readString(); + struct.processorCapabilities.add(_elem1438); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java index a841687ec4fc..4be2e90b2089 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java @@ -325,14 +325,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsRespon case 1: // PARTITION_SPEC if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1400 = iprot.readListBegin(); - struct.partitionSpec = new java.util.ArrayList(_list1400.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1401; - for (int _i1402 = 0; _i1402 < _list1400.size; ++_i1402) + org.apache.thrift.protocol.TList _list1416 = iprot.readListBegin(); + struct.partitionSpec = new java.util.ArrayList(_list1416.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1417; + for (int _i1418 = 0; _i1418 < _list1416.size; ++_i1418) { - _elem1401 = new PartitionSpec(); - _elem1401.read(iprot); - struct.partitionSpec.add(_elem1401); + _elem1417 = new PartitionSpec(); + _elem1417.read(iprot); + struct.partitionSpec.add(_elem1417); } iprot.readListEnd(); } @@ -358,9 +358,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsRespo oprot.writeFieldBegin(PARTITION_SPEC_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionSpec.size())); - for (PartitionSpec _iter1403 : struct.partitionSpec) + for (PartitionSpec _iter1419 : struct.partitionSpec) { - _iter1403.write(oprot); + _iter1419.write(oprot); } oprot.writeListEnd(); } @@ -391,9 +391,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsRespon if (struct.isSetPartitionSpec()) { { oprot.writeI32(struct.partitionSpec.size()); - for (PartitionSpec _iter1404 : struct.partitionSpec) + for (PartitionSpec _iter1420 : struct.partitionSpec) { - _iter1404.write(oprot); + _iter1420.write(oprot); } } } @@ -405,14 +405,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsRespons java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1405 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.partitionSpec = new java.util.ArrayList(_list1405.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1406; - for (int _i1407 = 0; _i1407 < _list1405.size; ++_i1407) + org.apache.thrift.protocol.TList _list1421 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitionSpec = new java.util.ArrayList(_list1421.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1422; + for (int _i1423 = 0; _i1423 < _list1421.size; ++_i1423) { - _elem1406 = new PartitionSpec(); - _elem1406.read(iprot); - struct.partitionSpec.add(_elem1406); + _elem1422 = new PartitionSpec(); + _elem1422.read(iprot); + struct.partitionSpec.add(_elem1422); } } struct.setPartitionSpecIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetProjectionsSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetProjectionsSpec.java index fe77ca13596e..9a0042f43a54 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetProjectionsSpec.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetProjectionsSpec.java @@ -484,13 +484,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetProjectionsSpec case 1: // FIELD_LIST if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1142 = iprot.readListBegin(); - struct.fieldList = new java.util.ArrayList(_list1142.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1143; - for (int _i1144 = 0; _i1144 < _list1142.size; ++_i1144) + org.apache.thrift.protocol.TList _list1158 = iprot.readListBegin(); + struct.fieldList = new java.util.ArrayList(_list1158.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1159; + for (int _i1160 = 0; _i1160 < _list1158.size; ++_i1160) { - _elem1143 = iprot.readString(); - struct.fieldList.add(_elem1143); + _elem1159 = iprot.readString(); + struct.fieldList.add(_elem1159); } iprot.readListEnd(); } @@ -532,9 +532,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetProjectionsSpec oprot.writeFieldBegin(FIELD_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fieldList.size())); - for (java.lang.String _iter1145 : struct.fieldList) + for (java.lang.String _iter1161 : struct.fieldList) { - oprot.writeString(_iter1145); + oprot.writeString(_iter1161); } oprot.writeListEnd(); } @@ -581,9 +581,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetProjectionsSpec if (struct.isSetFieldList()) { { oprot.writeI32(struct.fieldList.size()); - for (java.lang.String _iter1146 : struct.fieldList) + for (java.lang.String _iter1162 : struct.fieldList) { - oprot.writeString(_iter1146); + oprot.writeString(_iter1162); } } } @@ -601,13 +601,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetProjectionsSpec s java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1147 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.fieldList = new java.util.ArrayList(_list1147.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1148; - for (int _i1149 = 0; _i1149 < _list1147.size; ++_i1149) + org.apache.thrift.protocol.TList _list1163 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.fieldList = new java.util.ArrayList(_list1163.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1164; + for (int _i1165 = 0; _i1165 < _list1163.size; ++_i1165) { - _elem1148 = iprot.readString(); - struct.fieldList.add(_elem1148); + _elem1164 = iprot.readString(); + struct.fieldList.add(_elem1164); } } struct.setFieldListIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSchemaResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSchemaResponse.java index 0247c0b44ac9..745e8f2b544e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSchemaResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSchemaResponse.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemaResponse s case 1: // FIELDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1432 = iprot.readListBegin(); - struct.fields = new java.util.ArrayList(_list1432.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1433; - for (int _i1434 = 0; _i1434 < _list1432.size; ++_i1434) + org.apache.thrift.protocol.TList _list1448 = iprot.readListBegin(); + struct.fields = new java.util.ArrayList(_list1448.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1449; + for (int _i1450 = 0; _i1450 < _list1448.size; ++_i1450) { - _elem1433 = new FieldSchema(); - _elem1433.read(iprot); - struct.fields.add(_elem1433); + _elem1449 = new FieldSchema(); + _elem1449.read(iprot); + struct.fields.add(_elem1449); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemaResponse oprot.writeFieldBegin(FIELDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.fields.size())); - for (FieldSchema _iter1435 : struct.fields) + for (FieldSchema _iter1451 : struct.fields) { - _iter1435.write(oprot); + _iter1451.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemaResponse s org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.fields.size()); - for (FieldSchema _iter1436 : struct.fields) + for (FieldSchema _iter1452 : struct.fields) { - _iter1436.write(oprot); + _iter1452.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemaResponse s public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemaResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1437 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.fields = new java.util.ArrayList(_list1437.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1438; - for (int _i1439 = 0; _i1439 < _list1437.size; ++_i1439) + org.apache.thrift.protocol.TList _list1453 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.fields = new java.util.ArrayList(_list1453.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1454; + for (int _i1455 = 0; _i1455 < _list1453.size; ++_i1455) { - _elem1438 = new FieldSchema(); - _elem1438.read(iprot); - struct.fields.add(_elem1438); + _elem1454 = new FieldSchema(); + _elem1454.read(iprot); + struct.fields.add(_elem1454); } } struct.setFieldsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java index d50fe1e256c2..50d04e78653f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java @@ -1106,13 +1106,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableRequest str case 8: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1150 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1150.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1151; - for (int _i1152 = 0; _i1152 < _list1150.size; ++_i1152) + org.apache.thrift.protocol.TList _list1166 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1166.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1167; + for (int _i1168 = 0; _i1168 < _list1166.size; ++_i1168) { - _elem1151 = iprot.readString(); - struct.processorCapabilities.add(_elem1151); + _elem1167 = iprot.readString(); + struct.processorCapabilities.add(_elem1167); } iprot.readListEnd(); } @@ -1199,9 +1199,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableRequest st oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1153 : struct.processorCapabilities) + for (java.lang.String _iter1169 : struct.processorCapabilities) { - oprot.writeString(_iter1153); + oprot.writeString(_iter1169); } oprot.writeListEnd(); } @@ -1287,9 +1287,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetTableRequest str if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1154 : struct.processorCapabilities) + for (java.lang.String _iter1170 : struct.processorCapabilities) { - oprot.writeString(_iter1154); + oprot.writeString(_iter1170); } } } @@ -1331,13 +1331,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetTableRequest stru } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1155 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1155.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1156; - for (int _i1157 = 0; _i1157 < _list1155.size; ++_i1157) + org.apache.thrift.protocol.TList _list1171 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1171.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1172; + for (int _i1173 = 0; _i1173 < _list1171.size; ++_i1173) { - _elem1156 = iprot.readString(); - struct.processorCapabilities.add(_elem1156); + _elem1172 = iprot.readString(); + struct.processorCapabilities.add(_elem1172); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesExtRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesExtRequest.java index 76b79955d351..96f9f8166449 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesExtRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesExtRequest.java @@ -856,13 +856,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetTablesExtRequest case 6: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1182 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1182.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1183; - for (int _i1184 = 0; _i1184 < _list1182.size; ++_i1184) + org.apache.thrift.protocol.TList _list1198 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1198.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1199; + for (int _i1200 = 0; _i1200 < _list1198.size; ++_i1200) { - _elem1183 = iprot.readString(); - struct.processorCapabilities.add(_elem1183); + _elem1199 = iprot.readString(); + struct.processorCapabilities.add(_elem1199); } iprot.readListEnd(); } @@ -920,9 +920,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetTablesExtReques oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1185 : struct.processorCapabilities) + for (java.lang.String _iter1201 : struct.processorCapabilities) { - oprot.writeString(_iter1185); + oprot.writeString(_iter1201); } oprot.writeListEnd(); } @@ -974,9 +974,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetTablesExtRequest if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1186 : struct.processorCapabilities) + for (java.lang.String _iter1202 : struct.processorCapabilities) { - oprot.writeString(_iter1186); + oprot.writeString(_iter1202); } } } @@ -1003,13 +1003,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesExtRequest } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1187 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1187.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1188; - for (int _i1189 = 0; _i1189 < _list1187.size; ++_i1189) + org.apache.thrift.protocol.TList _list1203 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1203.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1204; + for (int _i1205 = 0; _i1205 < _list1203.size; ++_i1205) { - _elem1188 = iprot.readString(); - struct.processorCapabilities.add(_elem1188); + _elem1204 = iprot.readString(); + struct.processorCapabilities.add(_elem1204); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java index 69d63349bd44..8b67a0b311aa 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java @@ -926,13 +926,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetTablesRequest st case 2: // TBL_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1158 = iprot.readListBegin(); - struct.tblNames = new java.util.ArrayList(_list1158.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1159; - for (int _i1160 = 0; _i1160 < _list1158.size; ++_i1160) + org.apache.thrift.protocol.TList _list1174 = iprot.readListBegin(); + struct.tblNames = new java.util.ArrayList(_list1174.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1175; + for (int _i1176 = 0; _i1176 < _list1174.size; ++_i1176) { - _elem1159 = iprot.readString(); - struct.tblNames.add(_elem1159); + _elem1175 = iprot.readString(); + struct.tblNames.add(_elem1175); } iprot.readListEnd(); } @@ -961,13 +961,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetTablesRequest st case 5: // PROCESSOR_CAPABILITIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1161 = iprot.readListBegin(); - struct.processorCapabilities = new java.util.ArrayList(_list1161.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1162; - for (int _i1163 = 0; _i1163 < _list1161.size; ++_i1163) + org.apache.thrift.protocol.TList _list1177 = iprot.readListBegin(); + struct.processorCapabilities = new java.util.ArrayList(_list1177.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1178; + for (int _i1179 = 0; _i1179 < _list1177.size; ++_i1179) { - _elem1162 = iprot.readString(); - struct.processorCapabilities.add(_elem1162); + _elem1178 = iprot.readString(); + struct.processorCapabilities.add(_elem1178); } iprot.readListEnd(); } @@ -1024,9 +1024,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetTablesRequest s oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size())); - for (java.lang.String _iter1164 : struct.tblNames) + for (java.lang.String _iter1180 : struct.tblNames) { - oprot.writeString(_iter1164); + oprot.writeString(_iter1180); } oprot.writeListEnd(); } @@ -1052,9 +1052,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetTablesRequest s oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size())); - for (java.lang.String _iter1165 : struct.processorCapabilities) + for (java.lang.String _iter1181 : struct.processorCapabilities) { - oprot.writeString(_iter1165); + oprot.writeString(_iter1181); } oprot.writeListEnd(); } @@ -1126,9 +1126,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetTablesRequest st if (struct.isSetTblNames()) { { oprot.writeI32(struct.tblNames.size()); - for (java.lang.String _iter1166 : struct.tblNames) + for (java.lang.String _iter1182 : struct.tblNames) { - oprot.writeString(_iter1166); + oprot.writeString(_iter1182); } } } @@ -1141,9 +1141,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetTablesRequest st if (struct.isSetProcessorCapabilities()) { { oprot.writeI32(struct.processorCapabilities.size()); - for (java.lang.String _iter1167 : struct.processorCapabilities) + for (java.lang.String _iter1183 : struct.processorCapabilities) { - oprot.writeString(_iter1167); + oprot.writeString(_iter1183); } } } @@ -1166,13 +1166,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesRequest str java.util.BitSet incoming = iprot.readBitSet(7); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1168 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.tblNames = new java.util.ArrayList(_list1168.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1169; - for (int _i1170 = 0; _i1170 < _list1168.size; ++_i1170) + org.apache.thrift.protocol.TList _list1184 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tblNames = new java.util.ArrayList(_list1184.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1185; + for (int _i1186 = 0; _i1186 < _list1184.size; ++_i1186) { - _elem1169 = iprot.readString(); - struct.tblNames.add(_elem1169); + _elem1185 = iprot.readString(); + struct.tblNames.add(_elem1185); } } struct.setTblNamesIsSet(true); @@ -1188,13 +1188,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesRequest str } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list1171 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.processorCapabilities = new java.util.ArrayList(_list1171.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1172; - for (int _i1173 = 0; _i1173 < _list1171.size; ++_i1173) + org.apache.thrift.protocol.TList _list1187 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.processorCapabilities = new java.util.ArrayList(_list1187.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1188; + for (int _i1189 = 0; _i1189 < _list1187.size; ++_i1189) { - _elem1172 = iprot.readString(); - struct.processorCapabilities.add(_elem1172); + _elem1188 = iprot.readString(); + struct.processorCapabilities.add(_elem1188); } } struct.setProcessorCapabilitiesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java index e618de88e2a1..08900bdfa6b2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetTablesResult str case 1: // TABLES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1174 = iprot.readListBegin(); - struct.tables = new java.util.ArrayList
(_list1174.size); - @org.apache.thrift.annotation.Nullable Table _elem1175; - for (int _i1176 = 0; _i1176 < _list1174.size; ++_i1176) + org.apache.thrift.protocol.TList _list1190 = iprot.readListBegin(); + struct.tables = new java.util.ArrayList
(_list1190.size); + @org.apache.thrift.annotation.Nullable Table _elem1191; + for (int _i1192 = 0; _i1192 < _list1190.size; ++_i1192) { - _elem1175 = new Table(); - _elem1175.read(iprot); - struct.tables.add(_elem1175); + _elem1191 = new Table(); + _elem1191.read(iprot); + struct.tables.add(_elem1191); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetTablesResult st oprot.writeFieldBegin(TABLES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size())); - for (Table _iter1177 : struct.tables) + for (Table _iter1193 : struct.tables) { - _iter1177.write(oprot); + _iter1193.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetTablesResult str org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.tables.size()); - for (Table _iter1178 : struct.tables) + for (Table _iter1194 : struct.tables) { - _iter1178.write(oprot); + _iter1194.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetTablesResult str public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1179 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.tables = new java.util.ArrayList
(_list1179.size); - @org.apache.thrift.annotation.Nullable Table _elem1180; - for (int _i1181 = 0; _i1181 < _list1179.size; ++_i1181) + org.apache.thrift.protocol.TList _list1195 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.tables = new java.util.ArrayList
(_list1195.size); + @org.apache.thrift.annotation.Nullable Table _elem1196; + for (int _i1197 = 0; _i1197 < _list1195.size; ++_i1197) { - _elem1180 = new Table(); - _elem1180.read(iprot); - struct.tables.add(_elem1180); + _elem1196 = new Table(); + _elem1196.read(iprot); + struct.tables.add(_elem1196); } } struct.setTablesIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java index 4e0f49df0c7e..c6fd3443e033 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java @@ -711,13 +711,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, InsertEventRequestD case 2: // FILES_ADDED if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list984 = iprot.readListBegin(); - struct.filesAdded = new java.util.ArrayList(_list984.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem985; - for (int _i986 = 0; _i986 < _list984.size; ++_i986) + org.apache.thrift.protocol.TList _list1000 = iprot.readListBegin(); + struct.filesAdded = new java.util.ArrayList(_list1000.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1001; + for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002) { - _elem985 = iprot.readString(); - struct.filesAdded.add(_elem985); + _elem1001 = iprot.readString(); + struct.filesAdded.add(_elem1001); } iprot.readListEnd(); } @@ -729,13 +729,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, InsertEventRequestD case 3: // FILES_ADDED_CHECKSUM if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list987 = iprot.readListBegin(); - struct.filesAddedChecksum = new java.util.ArrayList(_list987.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem988; - for (int _i989 = 0; _i989 < _list987.size; ++_i989) + org.apache.thrift.protocol.TList _list1003 = iprot.readListBegin(); + struct.filesAddedChecksum = new java.util.ArrayList(_list1003.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1004; + for (int _i1005 = 0; _i1005 < _list1003.size; ++_i1005) { - _elem988 = iprot.readString(); - struct.filesAddedChecksum.add(_elem988); + _elem1004 = iprot.readString(); + struct.filesAddedChecksum.add(_elem1004); } iprot.readListEnd(); } @@ -747,13 +747,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, InsertEventRequestD case 4: // SUB_DIRECTORY_LIST if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list990 = iprot.readListBegin(); - struct.subDirectoryList = new java.util.ArrayList(_list990.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem991; - for (int _i992 = 0; _i992 < _list990.size; ++_i992) + org.apache.thrift.protocol.TList _list1006 = iprot.readListBegin(); + struct.subDirectoryList = new java.util.ArrayList(_list1006.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1007; + for (int _i1008 = 0; _i1008 < _list1006.size; ++_i1008) { - _elem991 = iprot.readString(); - struct.subDirectoryList.add(_elem991); + _elem1007 = iprot.readString(); + struct.subDirectoryList.add(_elem1007); } iprot.readListEnd(); } @@ -765,13 +765,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, InsertEventRequestD case 5: // PARTITION_VAL if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list993 = iprot.readListBegin(); - struct.partitionVal = new java.util.ArrayList(_list993.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem994; - for (int _i995 = 0; _i995 < _list993.size; ++_i995) + org.apache.thrift.protocol.TList _list1009 = iprot.readListBegin(); + struct.partitionVal = new java.util.ArrayList(_list1009.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1010; + for (int _i1011 = 0; _i1011 < _list1009.size; ++_i1011) { - _elem994 = iprot.readString(); - struct.partitionVal.add(_elem994); + _elem1010 = iprot.readString(); + struct.partitionVal.add(_elem1010); } iprot.readListEnd(); } @@ -802,9 +802,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, InsertEventRequest oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size())); - for (java.lang.String _iter996 : struct.filesAdded) + for (java.lang.String _iter1012 : struct.filesAdded) { - oprot.writeString(_iter996); + oprot.writeString(_iter1012); } oprot.writeListEnd(); } @@ -815,9 +815,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, InsertEventRequest oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size())); - for (java.lang.String _iter997 : struct.filesAddedChecksum) + for (java.lang.String _iter1013 : struct.filesAddedChecksum) { - oprot.writeString(_iter997); + oprot.writeString(_iter1013); } oprot.writeListEnd(); } @@ -829,9 +829,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, InsertEventRequest oprot.writeFieldBegin(SUB_DIRECTORY_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.subDirectoryList.size())); - for (java.lang.String _iter998 : struct.subDirectoryList) + for (java.lang.String _iter1014 : struct.subDirectoryList) { - oprot.writeString(_iter998); + oprot.writeString(_iter1014); } oprot.writeListEnd(); } @@ -843,9 +843,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, InsertEventRequest oprot.writeFieldBegin(PARTITION_VAL_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVal.size())); - for (java.lang.String _iter999 : struct.partitionVal) + for (java.lang.String _iter1015 : struct.partitionVal) { - oprot.writeString(_iter999); + oprot.writeString(_iter1015); } oprot.writeListEnd(); } @@ -871,9 +871,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestD org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.filesAdded.size()); - for (java.lang.String _iter1000 : struct.filesAdded) + for (java.lang.String _iter1016 : struct.filesAdded) { - oprot.writeString(_iter1000); + oprot.writeString(_iter1016); } } java.util.BitSet optionals = new java.util.BitSet(); @@ -896,27 +896,27 @@ public void write(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestD if (struct.isSetFilesAddedChecksum()) { { oprot.writeI32(struct.filesAddedChecksum.size()); - for (java.lang.String _iter1001 : struct.filesAddedChecksum) + for (java.lang.String _iter1017 : struct.filesAddedChecksum) { - oprot.writeString(_iter1001); + oprot.writeString(_iter1017); } } } if (struct.isSetSubDirectoryList()) { { oprot.writeI32(struct.subDirectoryList.size()); - for (java.lang.String _iter1002 : struct.subDirectoryList) + for (java.lang.String _iter1018 : struct.subDirectoryList) { - oprot.writeString(_iter1002); + oprot.writeString(_iter1018); } } } if (struct.isSetPartitionVal()) { { oprot.writeI32(struct.partitionVal.size()); - for (java.lang.String _iter1003 : struct.partitionVal) + for (java.lang.String _iter1019 : struct.partitionVal) { - oprot.writeString(_iter1003); + oprot.writeString(_iter1019); } } } @@ -926,13 +926,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestD public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1004 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.filesAdded = new java.util.ArrayList(_list1004.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1005; - for (int _i1006 = 0; _i1006 < _list1004.size; ++_i1006) + org.apache.thrift.protocol.TList _list1020 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.filesAdded = new java.util.ArrayList(_list1020.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1021; + for (int _i1022 = 0; _i1022 < _list1020.size; ++_i1022) { - _elem1005 = iprot.readString(); - struct.filesAdded.add(_elem1005); + _elem1021 = iprot.readString(); + struct.filesAdded.add(_elem1021); } } struct.setFilesAddedIsSet(true); @@ -943,39 +943,39 @@ public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestDa } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1007 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.filesAddedChecksum = new java.util.ArrayList(_list1007.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1008; - for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009) + org.apache.thrift.protocol.TList _list1023 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.filesAddedChecksum = new java.util.ArrayList(_list1023.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1024; + for (int _i1025 = 0; _i1025 < _list1023.size; ++_i1025) { - _elem1008 = iprot.readString(); - struct.filesAddedChecksum.add(_elem1008); + _elem1024 = iprot.readString(); + struct.filesAddedChecksum.add(_elem1024); } } struct.setFilesAddedChecksumIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.subDirectoryList = new java.util.ArrayList(_list1010.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1011; - for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012) + org.apache.thrift.protocol.TList _list1026 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.subDirectoryList = new java.util.ArrayList(_list1026.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1027; + for (int _i1028 = 0; _i1028 < _list1026.size; ++_i1028) { - _elem1011 = iprot.readString(); - struct.subDirectoryList.add(_elem1011); + _elem1027 = iprot.readString(); + struct.subDirectoryList.add(_elem1027); } } struct.setSubDirectoryListIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list1013 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partitionVal = new java.util.ArrayList(_list1013.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1014; - for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015) + org.apache.thrift.protocol.TList _list1029 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partitionVal = new java.util.ArrayList(_list1029.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1030; + for (int _i1031 = 0; _i1031 < _list1029.size; ++_i1031) { - _elem1014 = iprot.readString(); - struct.partitionVal.add(_elem1014); + _elem1030 = iprot.readString(); + struct.partitionVal.add(_elem1030); } } struct.setPartitionValIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java index 563786aa30b4..303b91524b31 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java @@ -14,6 +14,9 @@ private static final org.apache.thrift.protocol.TField LAST_EVENT_FIELD_DESC = new org.apache.thrift.protocol.TField("lastEvent", org.apache.thrift.protocol.TType.I64, (short)1); private static final org.apache.thrift.protocol.TField MAX_EVENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxEvents", org.apache.thrift.protocol.TType.I32, (short)2); private static final org.apache.thrift.protocol.TField EVENT_TYPE_SKIP_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("eventTypeSkipList", org.apache.thrift.protocol.TType.LIST, (short)3); + private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField TABLE_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNames", org.apache.thrift.protocol.TType.LIST, (short)6); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new NotificationEventRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new NotificationEventRequestTupleSchemeFactory(); @@ -21,12 +24,18 @@ private long lastEvent; // required private int maxEvents; // optional private @org.apache.thrift.annotation.Nullable java.util.List eventTypeSkipList; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String catName; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String dbName; // optional + private @org.apache.thrift.annotation.Nullable java.util.List tableNames; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { LAST_EVENT((short)1, "lastEvent"), MAX_EVENTS((short)2, "maxEvents"), - EVENT_TYPE_SKIP_LIST((short)3, "eventTypeSkipList"); + EVENT_TYPE_SKIP_LIST((short)3, "eventTypeSkipList"), + CAT_NAME((short)4, "catName"), + DB_NAME((short)5, "dbName"), + TABLE_NAMES((short)6, "tableNames"); private static final java.util.Map byName = new java.util.HashMap(); @@ -48,6 +57,12 @@ public static _Fields findByThriftId(int fieldId) { return MAX_EVENTS; case 3: // EVENT_TYPE_SKIP_LIST return EVENT_TYPE_SKIP_LIST; + case 4: // CAT_NAME + return CAT_NAME; + case 5: // DB_NAME + return DB_NAME; + case 6: // TABLE_NAMES + return TABLE_NAMES; default: return null; } @@ -92,7 +107,7 @@ public java.lang.String getFieldName() { private static final int __LASTEVENT_ISSET_ID = 0; private static final int __MAXEVENTS_ISSET_ID = 1; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.MAX_EVENTS,_Fields.EVENT_TYPE_SKIP_LIST}; + private static final _Fields optionals[] = {_Fields.MAX_EVENTS,_Fields.EVENT_TYPE_SKIP_LIST,_Fields.CAT_NAME,_Fields.DB_NAME,_Fields.TABLE_NAMES}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -103,6 +118,13 @@ public java.lang.String getFieldName() { tmpMap.put(_Fields.EVENT_TYPE_SKIP_LIST, new org.apache.thrift.meta_data.FieldMetaData("eventTypeSkipList", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLE_NAMES, new org.apache.thrift.meta_data.FieldMetaData("tableNames", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotificationEventRequest.class, metaDataMap); } @@ -129,6 +151,16 @@ public NotificationEventRequest(NotificationEventRequest other) { java.util.List __this__eventTypeSkipList = new java.util.ArrayList(other.eventTypeSkipList); this.eventTypeSkipList = __this__eventTypeSkipList; } + if (other.isSetCatName()) { + this.catName = other.catName; + } + if (other.isSetDbName()) { + this.dbName = other.dbName; + } + if (other.isSetTableNames()) { + java.util.List __this__tableNames = new java.util.ArrayList(other.tableNames); + this.tableNames = __this__tableNames; + } } public NotificationEventRequest deepCopy() { @@ -142,6 +174,9 @@ public void clear() { setMaxEventsIsSet(false); this.maxEvents = 0; this.eventTypeSkipList = null; + this.catName = null; + this.dbName = null; + this.tableNames = null; } public long getLastEvent() { @@ -228,6 +263,94 @@ public void setEventTypeSkipListIsSet(boolean value) { } } + @org.apache.thrift.annotation.Nullable + public java.lang.String getCatName() { + return this.catName; + } + + public void setCatName(@org.apache.thrift.annotation.Nullable java.lang.String catName) { + this.catName = catName; + } + + public void unsetCatName() { + this.catName = null; + } + + /** Returns true if field catName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatName() { + return this.catName != null; + } + + public void setCatNameIsSet(boolean value) { + if (!value) { + this.catName = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getDbName() { + return this.dbName; + } + + public void setDbName(@org.apache.thrift.annotation.Nullable java.lang.String dbName) { + this.dbName = dbName; + } + + public void unsetDbName() { + this.dbName = null; + } + + /** Returns true if field dbName is set (has been assigned a value) and false otherwise */ + public boolean isSetDbName() { + return this.dbName != null; + } + + public void setDbNameIsSet(boolean value) { + if (!value) { + this.dbName = null; + } + } + + public int getTableNamesSize() { + return (this.tableNames == null) ? 0 : this.tableNames.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getTableNamesIterator() { + return (this.tableNames == null) ? null : this.tableNames.iterator(); + } + + public void addToTableNames(java.lang.String elem) { + if (this.tableNames == null) { + this.tableNames = new java.util.ArrayList(); + } + this.tableNames.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getTableNames() { + return this.tableNames; + } + + public void setTableNames(@org.apache.thrift.annotation.Nullable java.util.List tableNames) { + this.tableNames = tableNames; + } + + public void unsetTableNames() { + this.tableNames = null; + } + + /** Returns true if field tableNames is set (has been assigned a value) and false otherwise */ + public boolean isSetTableNames() { + return this.tableNames != null; + } + + public void setTableNamesIsSet(boolean value) { + if (!value) { + this.tableNames = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case LAST_EVENT: @@ -254,6 +377,30 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case CAT_NAME: + if (value == null) { + unsetCatName(); + } else { + setCatName((java.lang.String)value); + } + break; + + case DB_NAME: + if (value == null) { + unsetDbName(); + } else { + setDbName((java.lang.String)value); + } + break; + + case TABLE_NAMES: + if (value == null) { + unsetTableNames(); + } else { + setTableNames((java.util.List)value); + } + break; + } } @@ -269,6 +416,15 @@ public java.lang.Object getFieldValue(_Fields field) { case EVENT_TYPE_SKIP_LIST: return getEventTypeSkipList(); + case CAT_NAME: + return getCatName(); + + case DB_NAME: + return getDbName(); + + case TABLE_NAMES: + return getTableNames(); + } throw new java.lang.IllegalStateException(); } @@ -286,6 +442,12 @@ public boolean isSet(_Fields field) { return isSetMaxEvents(); case EVENT_TYPE_SKIP_LIST: return isSetEventTypeSkipList(); + case CAT_NAME: + return isSetCatName(); + case DB_NAME: + return isSetDbName(); + case TABLE_NAMES: + return isSetTableNames(); } throw new java.lang.IllegalStateException(); } @@ -330,6 +492,33 @@ public boolean equals(NotificationEventRequest that) { return false; } + boolean this_present_catName = true && this.isSetCatName(); + boolean that_present_catName = true && that.isSetCatName(); + if (this_present_catName || that_present_catName) { + if (!(this_present_catName && that_present_catName)) + return false; + if (!this.catName.equals(that.catName)) + return false; + } + + boolean this_present_dbName = true && this.isSetDbName(); + boolean that_present_dbName = true && that.isSetDbName(); + if (this_present_dbName || that_present_dbName) { + if (!(this_present_dbName && that_present_dbName)) + return false; + if (!this.dbName.equals(that.dbName)) + return false; + } + + boolean this_present_tableNames = true && this.isSetTableNames(); + boolean that_present_tableNames = true && that.isSetTableNames(); + if (this_present_tableNames || that_present_tableNames) { + if (!(this_present_tableNames && that_present_tableNames)) + return false; + if (!this.tableNames.equals(that.tableNames)) + return false; + } + return true; } @@ -347,6 +536,18 @@ public int hashCode() { if (isSetEventTypeSkipList()) hashCode = hashCode * 8191 + eventTypeSkipList.hashCode(); + hashCode = hashCode * 8191 + ((isSetCatName()) ? 131071 : 524287); + if (isSetCatName()) + hashCode = hashCode * 8191 + catName.hashCode(); + + hashCode = hashCode * 8191 + ((isSetDbName()) ? 131071 : 524287); + if (isSetDbName()) + hashCode = hashCode * 8191 + dbName.hashCode(); + + hashCode = hashCode * 8191 + ((isSetTableNames()) ? 131071 : 524287); + if (isSetTableNames()) + hashCode = hashCode * 8191 + tableNames.hashCode(); + return hashCode; } @@ -388,6 +589,36 @@ public int compareTo(NotificationEventRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetCatName(), other.isSetCatName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetDbName(), other.isSetDbName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetTableNames(), other.isSetTableNames()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableNames()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNames, other.tableNames); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -428,6 +659,36 @@ public java.lang.String toString() { } first = false; } + if (isSetCatName()) { + if (!first) sb.append(", "); + sb.append("catName:"); + if (this.catName == null) { + sb.append("null"); + } else { + sb.append(this.catName); + } + first = false; + } + if (isSetDbName()) { + if (!first) sb.append(", "); + sb.append("dbName:"); + if (this.dbName == null) { + sb.append("null"); + } else { + sb.append(this.dbName); + } + first = false; + } + if (isSetTableNames()) { + if (!first) sb.append(", "); + sb.append("tableNames:"); + if (this.tableNames == null) { + sb.append("null"); + } else { + sb.append(this.tableNames); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -511,6 +772,40 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, NotificationEventRe org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 4: // CAT_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catName = iprot.readString(); + struct.setCatNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // DB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // TABLE_NAMES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list971 = iprot.readListBegin(); + struct.tableNames = new java.util.ArrayList(_list971.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem972; + for (int _i973 = 0; _i973 < _list971.size; ++_i973) + { + _elem972 = iprot.readString(); + struct.tableNames.add(_elem972); + } + iprot.readListEnd(); + } + struct.setTableNamesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -537,9 +832,37 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, NotificationEventR oprot.writeFieldBegin(EVENT_TYPE_SKIP_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.eventTypeSkipList.size())); - for (java.lang.String _iter971 : struct.eventTypeSkipList) + for (java.lang.String _iter974 : struct.eventTypeSkipList) { - oprot.writeString(_iter971); + oprot.writeString(_iter974); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.catName != null) { + if (struct.isSetCatName()) { + oprot.writeFieldBegin(CAT_NAME_FIELD_DESC); + oprot.writeString(struct.catName); + oprot.writeFieldEnd(); + } + } + if (struct.dbName != null) { + if (struct.isSetDbName()) { + oprot.writeFieldBegin(DB_NAME_FIELD_DESC); + oprot.writeString(struct.dbName); + oprot.writeFieldEnd(); + } + } + if (struct.tableNames != null) { + if (struct.isSetTableNames()) { + oprot.writeFieldBegin(TABLE_NAMES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableNames.size())); + for (java.lang.String _iter975 : struct.tableNames) + { + oprot.writeString(_iter975); } oprot.writeListEnd(); } @@ -571,16 +894,40 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NotificationEventRe if (struct.isSetEventTypeSkipList()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetCatName()) { + optionals.set(2); + } + if (struct.isSetDbName()) { + optionals.set(3); + } + if (struct.isSetTableNames()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); if (struct.isSetMaxEvents()) { oprot.writeI32(struct.maxEvents); } if (struct.isSetEventTypeSkipList()) { { oprot.writeI32(struct.eventTypeSkipList.size()); - for (java.lang.String _iter972 : struct.eventTypeSkipList) + for (java.lang.String _iter976 : struct.eventTypeSkipList) { - oprot.writeString(_iter972); + oprot.writeString(_iter976); + } + } + } + if (struct.isSetCatName()) { + oprot.writeString(struct.catName); + } + if (struct.isSetDbName()) { + oprot.writeString(struct.dbName); + } + if (struct.isSetTableNames()) { + { + oprot.writeI32(struct.tableNames.size()); + for (java.lang.String _iter977 : struct.tableNames) + { + oprot.writeString(_iter977); } } } @@ -591,24 +938,45 @@ public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventReq org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; struct.lastEvent = iprot.readI64(); struct.setLastEventIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(2); + java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { struct.maxEvents = iprot.readI32(); struct.setMaxEventsIsSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list973 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.eventTypeSkipList = new java.util.ArrayList(_list973.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem974; - for (int _i975 = 0; _i975 < _list973.size; ++_i975) + org.apache.thrift.protocol.TList _list978 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.eventTypeSkipList = new java.util.ArrayList(_list978.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem979; + for (int _i980 = 0; _i980 < _list978.size; ++_i980) { - _elem974 = iprot.readString(); - struct.eventTypeSkipList.add(_elem974); + _elem979 = iprot.readString(); + struct.eventTypeSkipList.add(_elem979); } } struct.setEventTypeSkipListIsSet(true); } + if (incoming.get(2)) { + struct.catName = iprot.readString(); + struct.setCatNameIsSet(true); + } + if (incoming.get(3)) { + struct.dbName = iprot.readString(); + struct.setDbNameIsSet(true); + } + if (incoming.get(4)) { + { + org.apache.thrift.protocol.TList _list981 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tableNames = new java.util.ArrayList(_list981.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem982; + for (int _i983 = 0; _i983 < _list981.size; ++_i983) + { + _elem982 = iprot.readString(); + struct.tableNames.add(_elem982); + } + } + struct.setTableNamesIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java index d72fad442696..d47dcffb4098 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, NotificationEventRe case 1: // EVENTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list976 = iprot.readListBegin(); - struct.events = new java.util.ArrayList(_list976.size); - @org.apache.thrift.annotation.Nullable NotificationEvent _elem977; - for (int _i978 = 0; _i978 < _list976.size; ++_i978) + org.apache.thrift.protocol.TList _list984 = iprot.readListBegin(); + struct.events = new java.util.ArrayList(_list984.size); + @org.apache.thrift.annotation.Nullable NotificationEvent _elem985; + for (int _i986 = 0; _i986 < _list984.size; ++_i986) { - _elem977 = new NotificationEvent(); - _elem977.read(iprot); - struct.events.add(_elem977); + _elem985 = new NotificationEvent(); + _elem985.read(iprot); + struct.events.add(_elem985); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, NotificationEventR oprot.writeFieldBegin(EVENTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size())); - for (NotificationEvent _iter979 : struct.events) + for (NotificationEvent _iter987 : struct.events) { - _iter979.write(oprot); + _iter987.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NotificationEventRe org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.events.size()); - for (NotificationEvent _iter980 : struct.events) + for (NotificationEvent _iter988 : struct.events) { - _iter980.write(oprot); + _iter988.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NotificationEventRe public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list981 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.events = new java.util.ArrayList(_list981.size); - @org.apache.thrift.annotation.Nullable NotificationEvent _elem982; - for (int _i983 = 0; _i983 < _list981.size; ++_i983) + org.apache.thrift.protocol.TList _list989 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.events = new java.util.ArrayList(_list989.size); + @org.apache.thrift.annotation.Nullable NotificationEvent _elem990; + for (int _i991 = 0; _i991 < _list989.size; ++_i991) { - _elem982 = new NotificationEvent(); - _elem982.read(iprot); - struct.events.add(_elem982); + _elem990 = new NotificationEvent(); + _elem990.read(iprot); + struct.events.add(_elem990); } } struct.setEventsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java index 4db39ac7b711..2dbd7f2ce205 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java @@ -16,6 +16,7 @@ private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)3); private static final org.apache.thrift.protocol.TField TO_EVENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("toEventId", org.apache.thrift.protocol.TType.I64, (short)4); private static final org.apache.thrift.protocol.TField LIMIT_FIELD_DESC = new org.apache.thrift.protocol.TField("limit", org.apache.thrift.protocol.TType.I64, (short)5); + private static final org.apache.thrift.protocol.TField TABLE_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNames", org.apache.thrift.protocol.TType.LIST, (short)6); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new NotificationEventsCountRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new NotificationEventsCountRequestTupleSchemeFactory(); @@ -25,6 +26,7 @@ private @org.apache.thrift.annotation.Nullable java.lang.String catName; // optional private long toEventId; // optional private long limit; // optional + private @org.apache.thrift.annotation.Nullable java.util.List tableNames; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -32,7 +34,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { DB_NAME((short)2, "dbName"), CAT_NAME((short)3, "catName"), TO_EVENT_ID((short)4, "toEventId"), - LIMIT((short)5, "limit"); + LIMIT((short)5, "limit"), + TABLE_NAMES((short)6, "tableNames"); private static final java.util.Map byName = new java.util.HashMap(); @@ -58,6 +61,8 @@ public static _Fields findByThriftId(int fieldId) { return TO_EVENT_ID; case 5: // LIMIT return LIMIT; + case 6: // TABLE_NAMES + return TABLE_NAMES; default: return null; } @@ -103,7 +108,7 @@ public java.lang.String getFieldName() { private static final int __TOEVENTID_ISSET_ID = 1; private static final int __LIMIT_ISSET_ID = 2; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.TO_EVENT_ID,_Fields.LIMIT}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.TO_EVENT_ID,_Fields.LIMIT,_Fields.TABLE_NAMES}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -117,6 +122,9 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.LIMIT, new org.apache.thrift.meta_data.FieldMetaData("limit", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.TABLE_NAMES, new org.apache.thrift.meta_data.FieldMetaData("tableNames", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotificationEventsCountRequest.class, metaDataMap); } @@ -148,6 +156,10 @@ public NotificationEventsCountRequest(NotificationEventsCountRequest other) { } this.toEventId = other.toEventId; this.limit = other.limit; + if (other.isSetTableNames()) { + java.util.List __this__tableNames = new java.util.ArrayList(other.tableNames); + this.tableNames = __this__tableNames; + } } public NotificationEventsCountRequest deepCopy() { @@ -164,6 +176,7 @@ public void clear() { this.toEventId = 0; setLimitIsSet(false); this.limit = 0; + this.tableNames = null; } public long getFromEventId() { @@ -280,6 +293,46 @@ public void setLimitIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __LIMIT_ISSET_ID, value); } + public int getTableNamesSize() { + return (this.tableNames == null) ? 0 : this.tableNames.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getTableNamesIterator() { + return (this.tableNames == null) ? null : this.tableNames.iterator(); + } + + public void addToTableNames(java.lang.String elem) { + if (this.tableNames == null) { + this.tableNames = new java.util.ArrayList(); + } + this.tableNames.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getTableNames() { + return this.tableNames; + } + + public void setTableNames(@org.apache.thrift.annotation.Nullable java.util.List tableNames) { + this.tableNames = tableNames; + } + + public void unsetTableNames() { + this.tableNames = null; + } + + /** Returns true if field tableNames is set (has been assigned a value) and false otherwise */ + public boolean isSetTableNames() { + return this.tableNames != null; + } + + public void setTableNamesIsSet(boolean value) { + if (!value) { + this.tableNames = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case FROM_EVENT_ID: @@ -322,6 +375,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case TABLE_NAMES: + if (value == null) { + unsetTableNames(); + } else { + setTableNames((java.util.List)value); + } + break; + } } @@ -343,6 +404,9 @@ public java.lang.Object getFieldValue(_Fields field) { case LIMIT: return getLimit(); + case TABLE_NAMES: + return getTableNames(); + } throw new java.lang.IllegalStateException(); } @@ -364,6 +428,8 @@ public boolean isSet(_Fields field) { return isSetToEventId(); case LIMIT: return isSetLimit(); + case TABLE_NAMES: + return isSetTableNames(); } throw new java.lang.IllegalStateException(); } @@ -426,6 +492,15 @@ public boolean equals(NotificationEventsCountRequest that) { return false; } + boolean this_present_tableNames = true && this.isSetTableNames(); + boolean that_present_tableNames = true && that.isSetTableNames(); + if (this_present_tableNames || that_present_tableNames) { + if (!(this_present_tableNames && that_present_tableNames)) + return false; + if (!this.tableNames.equals(that.tableNames)) + return false; + } + return true; } @@ -451,6 +526,10 @@ public int hashCode() { if (isSetLimit()) hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(limit); + hashCode = hashCode * 8191 + ((isSetTableNames()) ? 131071 : 524287); + if (isSetTableNames()) + hashCode = hashCode * 8191 + tableNames.hashCode(); + return hashCode; } @@ -512,6 +591,16 @@ public int compareTo(NotificationEventsCountRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetTableNames(), other.isSetTableNames()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableNames()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNames, other.tableNames); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -566,6 +655,16 @@ public java.lang.String toString() { sb.append(this.limit); first = false; } + if (isSetTableNames()) { + if (!first) sb.append(", "); + sb.append("tableNames:"); + if (this.tableNames == null) { + sb.append("null"); + } else { + sb.append(this.tableNames); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -659,6 +758,24 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, NotificationEventsC org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 6: // TABLE_NAMES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list992 = iprot.readListBegin(); + struct.tableNames = new java.util.ArrayList(_list992.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem993; + for (int _i994 = 0; _i994 < _list992.size; ++_i994) + { + _elem993 = iprot.readString(); + struct.tableNames.add(_elem993); + } + iprot.readListEnd(); + } + struct.setTableNamesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -697,6 +814,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, NotificationEvents oprot.writeI64(struct.limit); oprot.writeFieldEnd(); } + if (struct.tableNames != null) { + if (struct.isSetTableNames()) { + oprot.writeFieldBegin(TABLE_NAMES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableNames.size())); + for (java.lang.String _iter995 : struct.tableNames) + { + oprot.writeString(_iter995); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -726,7 +857,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NotificationEventsC if (struct.isSetLimit()) { optionals.set(2); } - oprot.writeBitSet(optionals, 3); + if (struct.isSetTableNames()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } @@ -736,6 +870,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NotificationEventsC if (struct.isSetLimit()) { oprot.writeI64(struct.limit); } + if (struct.isSetTableNames()) { + { + oprot.writeI32(struct.tableNames.size()); + for (java.lang.String _iter996 : struct.tableNames) + { + oprot.writeString(_iter996); + } + } + } } @Override @@ -745,7 +888,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventsCo struct.setFromEventIdIsSet(true); struct.dbName = iprot.readString(); struct.setDbNameIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(3); + java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); @@ -758,6 +901,19 @@ public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventsCo struct.limit = iprot.readI64(); struct.setLimitIsSet(true); } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TList _list997 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tableNames = new java.util.ArrayList(_list997.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem998; + for (int _i999 = 0; _i999 < _list997.size; ++_i999) + { + _elem998 = iprot.readString(); + struct.tableNames.add(_elem998); + } + } + struct.setTableNamesIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsResponse.java index 01163b76a517..f09f19810c94 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsResponse.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsResponse case 1: // PARTITIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1448 = iprot.readListBegin(); - struct.partitions = new java.util.ArrayList(_list1448.size); - @org.apache.thrift.annotation.Nullable Partition _elem1449; - for (int _i1450 = 0; _i1450 < _list1448.size; ++_i1450) + org.apache.thrift.protocol.TList _list1464 = iprot.readListBegin(); + struct.partitions = new java.util.ArrayList(_list1464.size); + @org.apache.thrift.annotation.Nullable Partition _elem1465; + for (int _i1466 = 0; _i1466 < _list1464.size; ++_i1466) { - _elem1449 = new Partition(); - _elem1449.read(iprot); - struct.partitions.add(_elem1449); + _elem1465 = new Partition(); + _elem1465.read(iprot); + struct.partitions.add(_elem1465); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, PartitionsResponse oprot.writeFieldBegin(PARTITIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size())); - for (Partition _iter1451 : struct.partitions) + for (Partition _iter1467 : struct.partitions) { - _iter1451.write(oprot); + _iter1467.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsResponse org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.partitions.size()); - for (Partition _iter1452 : struct.partitions) + for (Partition _iter1468 : struct.partitions) { - _iter1452.write(oprot); + _iter1468.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsResponse public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1453 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.partitions = new java.util.ArrayList(_list1453.size); - @org.apache.thrift.annotation.Nullable Partition _elem1454; - for (int _i1455 = 0; _i1455 < _list1453.size; ++_i1455) + org.apache.thrift.protocol.TList _list1469 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitions = new java.util.ArrayList(_list1469.size); + @org.apache.thrift.annotation.Nullable Partition _elem1470; + for (int _i1471 = 0; _i1471 < _list1469.size; ++_i1471) { - _elem1454 = new Partition(); - _elem1454.read(iprot); - struct.partitions.add(_elem1454); + _elem1470 = new Partition(); + _elem1470.read(iprot); + struct.partitions.add(_elem1470); } } struct.setPartitionsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java index b1cdedd67a33..a439a6738b26 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java @@ -523,13 +523,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PutFileMetadataRequ case 1: // FILE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1102 = iprot.readListBegin(); - struct.fileIds = new java.util.ArrayList(_list1102.size); - long _elem1103; - for (int _i1104 = 0; _i1104 < _list1102.size; ++_i1104) + org.apache.thrift.protocol.TList _list1118 = iprot.readListBegin(); + struct.fileIds = new java.util.ArrayList(_list1118.size); + long _elem1119; + for (int _i1120 = 0; _i1120 < _list1118.size; ++_i1120) { - _elem1103 = iprot.readI64(); - struct.fileIds.add(_elem1103); + _elem1119 = iprot.readI64(); + struct.fileIds.add(_elem1119); } iprot.readListEnd(); } @@ -541,13 +541,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PutFileMetadataRequ case 2: // METADATA if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1105 = iprot.readListBegin(); - struct.metadata = new java.util.ArrayList(_list1105.size); - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem1106; - for (int _i1107 = 0; _i1107 < _list1105.size; ++_i1107) + org.apache.thrift.protocol.TList _list1121 = iprot.readListBegin(); + struct.metadata = new java.util.ArrayList(_list1121.size); + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem1122; + for (int _i1123 = 0; _i1123 < _list1121.size; ++_i1123) { - _elem1106 = iprot.readBinary(); - struct.metadata.add(_elem1106); + _elem1122 = iprot.readBinary(); + struct.metadata.add(_elem1122); } iprot.readListEnd(); } @@ -581,9 +581,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, PutFileMetadataReq oprot.writeFieldBegin(FILE_IDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size())); - for (long _iter1108 : struct.fileIds) + for (long _iter1124 : struct.fileIds) { - oprot.writeI64(_iter1108); + oprot.writeI64(_iter1124); } oprot.writeListEnd(); } @@ -593,9 +593,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, PutFileMetadataReq oprot.writeFieldBegin(METADATA_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size())); - for (java.nio.ByteBuffer _iter1109 : struct.metadata) + for (java.nio.ByteBuffer _iter1125 : struct.metadata) { - oprot.writeBinary(_iter1109); + oprot.writeBinary(_iter1125); } oprot.writeListEnd(); } @@ -627,16 +627,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequ org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.fileIds.size()); - for (long _iter1110 : struct.fileIds) + for (long _iter1126 : struct.fileIds) { - oprot.writeI64(_iter1110); + oprot.writeI64(_iter1126); } } { oprot.writeI32(struct.metadata.size()); - for (java.nio.ByteBuffer _iter1111 : struct.metadata) + for (java.nio.ByteBuffer _iter1127 : struct.metadata) { - oprot.writeBinary(_iter1111); + oprot.writeBinary(_iter1127); } } java.util.BitSet optionals = new java.util.BitSet(); @@ -653,24 +653,24 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequ public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1112 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); - struct.fileIds = new java.util.ArrayList(_list1112.size); - long _elem1113; - for (int _i1114 = 0; _i1114 < _list1112.size; ++_i1114) + org.apache.thrift.protocol.TList _list1128 = iprot.readListBegin(org.apache.thrift.protocol.TType.I64); + struct.fileIds = new java.util.ArrayList(_list1128.size); + long _elem1129; + for (int _i1130 = 0; _i1130 < _list1128.size; ++_i1130) { - _elem1113 = iprot.readI64(); - struct.fileIds.add(_elem1113); + _elem1129 = iprot.readI64(); + struct.fileIds.add(_elem1129); } } struct.setFileIdsIsSet(true); { - org.apache.thrift.protocol.TList _list1115 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.metadata = new java.util.ArrayList(_list1115.size); - @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem1116; - for (int _i1117 = 0; _i1117 < _list1115.size; ++_i1117) + org.apache.thrift.protocol.TList _list1131 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.metadata = new java.util.ArrayList(_list1131.size); + @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem1132; + for (int _i1133 = 0; _i1133 < _list1131.size; ++_i1133) { - _elem1116 = iprot.readBinary(); - struct.metadata.add(_elem1116); + _elem1132 = iprot.readBinary(); + struct.metadata.add(_elem1132); } } struct.setMetadataIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java index 3f91c10cb157..36a111ece4df 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java @@ -925,13 +925,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, RenamePartitionRequ case 4: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1376 = iprot.readListBegin(); - struct.partVals = new java.util.ArrayList(_list1376.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1377; - for (int _i1378 = 0; _i1378 < _list1376.size; ++_i1378) + org.apache.thrift.protocol.TList _list1392 = iprot.readListBegin(); + struct.partVals = new java.util.ArrayList(_list1392.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1393; + for (int _i1394 = 0; _i1394 < _list1392.size; ++_i1394) { - _elem1377 = iprot.readString(); - struct.partVals.add(_elem1377); + _elem1393 = iprot.readString(); + struct.partVals.add(_elem1393); } iprot.readListEnd(); } @@ -1007,9 +1007,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, RenamePartitionReq oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partVals.size())); - for (java.lang.String _iter1379 : struct.partVals) + for (java.lang.String _iter1395 : struct.partVals) { - oprot.writeString(_iter1379); + oprot.writeString(_iter1395); } oprot.writeListEnd(); } @@ -1058,9 +1058,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, RenamePartitionRequ oprot.writeString(struct.tableName); { oprot.writeI32(struct.partVals.size()); - for (java.lang.String _iter1380 : struct.partVals) + for (java.lang.String _iter1396 : struct.partVals) { - oprot.writeString(_iter1380); + oprot.writeString(_iter1396); } } struct.newPart.write(oprot); @@ -1100,13 +1100,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, RenamePartitionReque struct.tableName = iprot.readString(); struct.setTableNameIsSet(true); { - org.apache.thrift.protocol.TList _list1381 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partVals = new java.util.ArrayList(_list1381.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1382; - for (int _i1383 = 0; _i1383 < _list1381.size; ++_i1383) + org.apache.thrift.protocol.TList _list1397 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partVals = new java.util.ArrayList(_list1397.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1398; + for (int _i1399 = 0; _i1399 < _list1397.size; ++_i1399) { - _elem1382 = iprot.readString(); - struct.partVals.add(_elem1382); + _elem1398 = iprot.readString(); + struct.partVals.add(_elem1398); } } struct.setPartValsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java index e7df32a58f06..891fa2969a98 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ReplicationMetricLi case 1: // REPLICATION_METRIC_LIST if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1496 = iprot.readListBegin(); - struct.replicationMetricList = new java.util.ArrayList(_list1496.size); - @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1497; - for (int _i1498 = 0; _i1498 < _list1496.size; ++_i1498) + org.apache.thrift.protocol.TList _list1512 = iprot.readListBegin(); + struct.replicationMetricList = new java.util.ArrayList(_list1512.size); + @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1513; + for (int _i1514 = 0; _i1514 < _list1512.size; ++_i1514) { - _elem1497 = new ReplicationMetrics(); - _elem1497.read(iprot); - struct.replicationMetricList.add(_elem1497); + _elem1513 = new ReplicationMetrics(); + _elem1513.read(iprot); + struct.replicationMetricList.add(_elem1513); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ReplicationMetricL oprot.writeFieldBegin(REPLICATION_METRIC_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.replicationMetricList.size())); - for (ReplicationMetrics _iter1499 : struct.replicationMetricList) + for (ReplicationMetrics _iter1515 : struct.replicationMetricList) { - _iter1499.write(oprot); + _iter1515.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricLi org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.replicationMetricList.size()); - for (ReplicationMetrics _iter1500 : struct.replicationMetricList) + for (ReplicationMetrics _iter1516 : struct.replicationMetricList) { - _iter1500.write(oprot); + _iter1516.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricLi public void read(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricList struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1501 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.replicationMetricList = new java.util.ArrayList(_list1501.size); - @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1502; - for (int _i1503 = 0; _i1503 < _list1501.size; ++_i1503) + org.apache.thrift.protocol.TList _list1517 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.replicationMetricList = new java.util.ArrayList(_list1517.size); + @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1518; + for (int _i1519 = 0; _i1519 < _list1517.size; ++_i1519) { - _elem1502 = new ReplicationMetrics(); - _elem1502.read(iprot); - struct.replicationMetricList.add(_elem1502); + _elem1518 = new ReplicationMetrics(); + _elem1518.read(iprot); + struct.replicationMetricList.add(_elem1518); } } struct.setReplicationMetricListIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java index 83b4821e848f..760b1058d76a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java @@ -1088,14 +1088,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SchemaVersion struc case 4: // COLS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1278 = iprot.readListBegin(); - struct.cols = new java.util.ArrayList(_list1278.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1279; - for (int _i1280 = 0; _i1280 < _list1278.size; ++_i1280) + org.apache.thrift.protocol.TList _list1294 = iprot.readListBegin(); + struct.cols = new java.util.ArrayList(_list1294.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1295; + for (int _i1296 = 0; _i1296 < _list1294.size; ++_i1296) { - _elem1279 = new FieldSchema(); - _elem1279.read(iprot); - struct.cols.add(_elem1279); + _elem1295 = new FieldSchema(); + _elem1295.read(iprot); + struct.cols.add(_elem1295); } iprot.readListEnd(); } @@ -1181,9 +1181,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SchemaVersion stru oprot.writeFieldBegin(COLS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size())); - for (FieldSchema _iter1281 : struct.cols) + for (FieldSchema _iter1297 : struct.cols) { - _iter1281.write(oprot); + _iter1297.write(oprot); } oprot.writeListEnd(); } @@ -1292,9 +1292,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SchemaVersion struc if (struct.isSetCols()) { { oprot.writeI32(struct.cols.size()); - for (FieldSchema _iter1282 : struct.cols) + for (FieldSchema _iter1298 : struct.cols) { - _iter1282.write(oprot); + _iter1298.write(oprot); } } } @@ -1337,14 +1337,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SchemaVersion struct } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list1283 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.cols = new java.util.ArrayList(_list1283.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1284; - for (int _i1285 = 0; _i1285 < _list1283.size; ++_i1285) + org.apache.thrift.protocol.TList _list1299 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.cols = new java.util.ArrayList(_list1299.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1300; + for (int _i1301 = 0; _i1301 < _list1299.size; ++_i1301) { - _elem1284 = new FieldSchema(); - _elem1284.read(iprot); - struct.cols.add(_elem1284); + _elem1300 = new FieldSchema(); + _elem1300.read(iprot); + struct.cols.add(_elem1300); } } struct.setColsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java index f4b313fbdeba..eb9b6dc553de 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java @@ -58665,13 +58665,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_databases_resul case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1512 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1512.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1513; - for (int _i1514 = 0; _i1514 < _list1512.size; ++_i1514) + org.apache.thrift.protocol.TList _list1528 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1528.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1529; + for (int _i1530 = 0; _i1530 < _list1528.size; ++_i1530) { - _elem1513 = iprot.readString(); - struct.success.add(_elem1513); + _elem1529 = iprot.readString(); + struct.success.add(_elem1529); } iprot.readListEnd(); } @@ -58706,9 +58706,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_databases_resu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1515 : struct.success) + for (java.lang.String _iter1531 : struct.success) { - oprot.writeString(_iter1515); + oprot.writeString(_iter1531); } oprot.writeListEnd(); } @@ -58747,9 +58747,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_databases_resul if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1516 : struct.success) + for (java.lang.String _iter1532 : struct.success) { - oprot.writeString(_iter1516); + oprot.writeString(_iter1532); } } } @@ -58764,13 +58764,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_databases_result java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1517 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1517.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1518; - for (int _i1519 = 0; _i1519 < _list1517.size; ++_i1519) + org.apache.thrift.protocol.TList _list1533 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1533.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1534; + for (int _i1535 = 0; _i1535 < _list1533.size; ++_i1535) { - _elem1518 = iprot.readString(); - struct.success.add(_elem1518); + _elem1534 = iprot.readString(); + struct.success.add(_elem1534); } } struct.setSuccessIsSet(true); @@ -59433,13 +59433,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_databases_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1520 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1520.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1521; - for (int _i1522 = 0; _i1522 < _list1520.size; ++_i1522) + org.apache.thrift.protocol.TList _list1536 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1536.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1537; + for (int _i1538 = 0; _i1538 < _list1536.size; ++_i1538) { - _elem1521 = iprot.readString(); - struct.success.add(_elem1521); + _elem1537 = iprot.readString(); + struct.success.add(_elem1537); } iprot.readListEnd(); } @@ -59474,9 +59474,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_databases_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1523 : struct.success) + for (java.lang.String _iter1539 : struct.success) { - oprot.writeString(_iter1523); + oprot.writeString(_iter1539); } oprot.writeListEnd(); } @@ -59515,9 +59515,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_databases_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1524 : struct.success) + for (java.lang.String _iter1540 : struct.success) { - oprot.writeString(_iter1524); + oprot.writeString(_iter1540); } } } @@ -59532,13 +59532,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_databases_re java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1525 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1525.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1526; - for (int _i1527 = 0; _i1527 < _list1525.size; ++_i1527) + org.apache.thrift.protocol.TList _list1541 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1541.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1542; + for (int _i1543 = 0; _i1543 < _list1541.size; ++_i1543) { - _elem1526 = iprot.readString(); - struct.success.add(_elem1526); + _elem1542 = iprot.readString(); + struct.success.add(_elem1542); } } struct.setSuccessIsSet(true); @@ -64163,13 +64163,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_dataconnectors_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1528 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1528.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1529; - for (int _i1530 = 0; _i1530 < _list1528.size; ++_i1530) + org.apache.thrift.protocol.TList _list1544 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1544.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1545; + for (int _i1546 = 0; _i1546 < _list1544.size; ++_i1546) { - _elem1529 = iprot.readString(); - struct.success.add(_elem1529); + _elem1545 = iprot.readString(); + struct.success.add(_elem1545); } iprot.readListEnd(); } @@ -64204,9 +64204,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_dataconnectors oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1531 : struct.success) + for (java.lang.String _iter1547 : struct.success) { - oprot.writeString(_iter1531); + oprot.writeString(_iter1547); } oprot.writeListEnd(); } @@ -64245,9 +64245,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_dataconnectors_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1532 : struct.success) + for (java.lang.String _iter1548 : struct.success) { - oprot.writeString(_iter1532); + oprot.writeString(_iter1548); } } } @@ -64262,13 +64262,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_dataconnectors_r java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1533 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1533.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1534; - for (int _i1535 = 0; _i1535 < _list1533.size; ++_i1535) + org.apache.thrift.protocol.TList _list1549 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1549.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1550; + for (int _i1551 = 0; _i1551 < _list1549.size; ++_i1551) { - _elem1534 = iprot.readString(); - struct.success.add(_elem1534); + _elem1550 = iprot.readString(); + struct.success.add(_elem1550); } } struct.setSuccessIsSet(true); @@ -68909,16 +68909,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_type_all_result case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1536 = iprot.readMapBegin(); - struct.success = new java.util.HashMap(2*_map1536.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1537; - @org.apache.thrift.annotation.Nullable Type _val1538; - for (int _i1539 = 0; _i1539 < _map1536.size; ++_i1539) + org.apache.thrift.protocol.TMap _map1552 = iprot.readMapBegin(); + struct.success = new java.util.HashMap(2*_map1552.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1553; + @org.apache.thrift.annotation.Nullable Type _val1554; + for (int _i1555 = 0; _i1555 < _map1552.size; ++_i1555) { - _key1537 = iprot.readString(); - _val1538 = new Type(); - _val1538.read(iprot); - struct.success.put(_key1537, _val1538); + _key1553 = iprot.readString(); + _val1554 = new Type(); + _val1554.read(iprot); + struct.success.put(_key1553, _val1554); } iprot.readMapEnd(); } @@ -68953,10 +68953,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_type_all_resul oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (java.util.Map.Entry _iter1540 : struct.success.entrySet()) + for (java.util.Map.Entry _iter1556 : struct.success.entrySet()) { - oprot.writeString(_iter1540.getKey()); - _iter1540.getValue().write(oprot); + oprot.writeString(_iter1556.getKey()); + _iter1556.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -68995,10 +68995,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_type_all_result if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.util.Map.Entry _iter1541 : struct.success.entrySet()) + for (java.util.Map.Entry _iter1557 : struct.success.entrySet()) { - oprot.writeString(_iter1541.getKey()); - _iter1541.getValue().write(oprot); + oprot.writeString(_iter1557.getKey()); + _iter1557.getValue().write(oprot); } } } @@ -69013,16 +69013,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_type_all_result java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map1542 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.HashMap(2*_map1542.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1543; - @org.apache.thrift.annotation.Nullable Type _val1544; - for (int _i1545 = 0; _i1545 < _map1542.size; ++_i1545) + org.apache.thrift.protocol.TMap _map1558 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.HashMap(2*_map1558.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1559; + @org.apache.thrift.annotation.Nullable Type _val1560; + for (int _i1561 = 0; _i1561 < _map1558.size; ++_i1561) { - _key1543 = iprot.readString(); - _val1544 = new Type(); - _val1544.read(iprot); - struct.success.put(_key1543, _val1544); + _key1559 = iprot.readString(); + _val1560 = new Type(); + _val1560.read(iprot); + struct.success.put(_key1559, _val1560); } } struct.setSuccessIsSet(true); @@ -70066,14 +70066,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_fields_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1546 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1546.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1547; - for (int _i1548 = 0; _i1548 < _list1546.size; ++_i1548) + org.apache.thrift.protocol.TList _list1562 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1562.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1563; + for (int _i1564 = 0; _i1564 < _list1562.size; ++_i1564) { - _elem1547 = new FieldSchema(); - _elem1547.read(iprot); - struct.success.add(_elem1547); + _elem1563 = new FieldSchema(); + _elem1563.read(iprot); + struct.success.add(_elem1563); } iprot.readListEnd(); } @@ -70126,9 +70126,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_fields_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1549 : struct.success) + for (FieldSchema _iter1565 : struct.success) { - _iter1549.write(oprot); + _iter1565.write(oprot); } oprot.writeListEnd(); } @@ -70183,9 +70183,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_fields_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1550 : struct.success) + for (FieldSchema _iter1566 : struct.success) { - _iter1550.write(oprot); + _iter1566.write(oprot); } } } @@ -70206,14 +70206,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_fields_result st java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1551 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1551.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1552; - for (int _i1553 = 0; _i1553 < _list1551.size; ++_i1553) + org.apache.thrift.protocol.TList _list1567 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1567.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1568; + for (int _i1569 = 0; _i1569 < _list1567.size; ++_i1569) { - _elem1552 = new FieldSchema(); - _elem1552.read(iprot); - struct.success.add(_elem1552); + _elem1568 = new FieldSchema(); + _elem1568.read(iprot); + struct.success.add(_elem1568); } } struct.setSuccessIsSet(true); @@ -71376,14 +71376,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_fields_with_env case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1554 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1554.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1555; - for (int _i1556 = 0; _i1556 < _list1554.size; ++_i1556) + org.apache.thrift.protocol.TList _list1570 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1570.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1571; + for (int _i1572 = 0; _i1572 < _list1570.size; ++_i1572) { - _elem1555 = new FieldSchema(); - _elem1555.read(iprot); - struct.success.add(_elem1555); + _elem1571 = new FieldSchema(); + _elem1571.read(iprot); + struct.success.add(_elem1571); } iprot.readListEnd(); } @@ -71436,9 +71436,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_fields_with_en oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1557 : struct.success) + for (FieldSchema _iter1573 : struct.success) { - _iter1557.write(oprot); + _iter1573.write(oprot); } oprot.writeListEnd(); } @@ -71493,9 +71493,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_fields_with_env if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1558 : struct.success) + for (FieldSchema _iter1574 : struct.success) { - _iter1558.write(oprot); + _iter1574.write(oprot); } } } @@ -71516,14 +71516,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_fields_with_envi java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1559 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1559.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1560; - for (int _i1561 = 0; _i1561 < _list1559.size; ++_i1561) + org.apache.thrift.protocol.TList _list1575 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1575.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1576; + for (int _i1577 = 0; _i1577 < _list1575.size; ++_i1577) { - _elem1560 = new FieldSchema(); - _elem1560.read(iprot); - struct.success.add(_elem1560); + _elem1576 = new FieldSchema(); + _elem1576.read(iprot); + struct.success.add(_elem1576); } } struct.setSuccessIsSet(true); @@ -73629,14 +73629,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1562 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1562.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1563; - for (int _i1564 = 0; _i1564 < _list1562.size; ++_i1564) + org.apache.thrift.protocol.TList _list1578 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1578.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1579; + for (int _i1580 = 0; _i1580 < _list1578.size; ++_i1580) { - _elem1563 = new FieldSchema(); - _elem1563.read(iprot); - struct.success.add(_elem1563); + _elem1579 = new FieldSchema(); + _elem1579.read(iprot); + struct.success.add(_elem1579); } iprot.readListEnd(); } @@ -73689,9 +73689,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1565 : struct.success) + for (FieldSchema _iter1581 : struct.success) { - _iter1565.write(oprot); + _iter1581.write(oprot); } oprot.writeListEnd(); } @@ -73746,9 +73746,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1566 : struct.success) + for (FieldSchema _iter1582 : struct.success) { - _iter1566.write(oprot); + _iter1582.write(oprot); } } } @@ -73769,14 +73769,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_result st java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1567 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1567.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1568; - for (int _i1569 = 0; _i1569 < _list1567.size; ++_i1569) + org.apache.thrift.protocol.TList _list1583 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1583.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1584; + for (int _i1585 = 0; _i1585 < _list1583.size; ++_i1585) { - _elem1568 = new FieldSchema(); - _elem1568.read(iprot); - struct.success.add(_elem1568); + _elem1584 = new FieldSchema(); + _elem1584.read(iprot); + struct.success.add(_elem1584); } } struct.setSuccessIsSet(true); @@ -74939,14 +74939,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_with_env case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1570 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1570.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1571; - for (int _i1572 = 0; _i1572 < _list1570.size; ++_i1572) + org.apache.thrift.protocol.TList _list1586 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1586.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1587; + for (int _i1588 = 0; _i1588 < _list1586.size; ++_i1588) { - _elem1571 = new FieldSchema(); - _elem1571.read(iprot); - struct.success.add(_elem1571); + _elem1587 = new FieldSchema(); + _elem1587.read(iprot); + struct.success.add(_elem1587); } iprot.readListEnd(); } @@ -74999,9 +74999,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_with_en oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1573 : struct.success) + for (FieldSchema _iter1589 : struct.success) { - _iter1573.write(oprot); + _iter1589.write(oprot); } oprot.writeListEnd(); } @@ -75056,9 +75056,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_with_env if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1574 : struct.success) + for (FieldSchema _iter1590 : struct.success) { - _iter1574.write(oprot); + _iter1590.write(oprot); } } } @@ -75079,14 +75079,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_with_envi java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1575 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1575.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1576; - for (int _i1577 = 0; _i1577 < _list1575.size; ++_i1577) + org.apache.thrift.protocol.TList _list1591 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1591.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1592; + for (int _i1593 = 0; _i1593 < _list1591.size; ++_i1593) { - _elem1576 = new FieldSchema(); - _elem1576.read(iprot); - struct.success.add(_elem1576); + _elem1592 = new FieldSchema(); + _elem1592.read(iprot); + struct.success.add(_elem1592); } } struct.setSuccessIsSet(true); @@ -79293,14 +79293,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 2: // PRIMARY_KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1578 = iprot.readListBegin(); - struct.primaryKeys = new java.util.ArrayList(_list1578.size); - @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1579; - for (int _i1580 = 0; _i1580 < _list1578.size; ++_i1580) + org.apache.thrift.protocol.TList _list1594 = iprot.readListBegin(); + struct.primaryKeys = new java.util.ArrayList(_list1594.size); + @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1595; + for (int _i1596 = 0; _i1596 < _list1594.size; ++_i1596) { - _elem1579 = new SQLPrimaryKey(); - _elem1579.read(iprot); - struct.primaryKeys.add(_elem1579); + _elem1595 = new SQLPrimaryKey(); + _elem1595.read(iprot); + struct.primaryKeys.add(_elem1595); } iprot.readListEnd(); } @@ -79312,14 +79312,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 3: // FOREIGN_KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1581 = iprot.readListBegin(); - struct.foreignKeys = new java.util.ArrayList(_list1581.size); - @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1582; - for (int _i1583 = 0; _i1583 < _list1581.size; ++_i1583) + org.apache.thrift.protocol.TList _list1597 = iprot.readListBegin(); + struct.foreignKeys = new java.util.ArrayList(_list1597.size); + @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1598; + for (int _i1599 = 0; _i1599 < _list1597.size; ++_i1599) { - _elem1582 = new SQLForeignKey(); - _elem1582.read(iprot); - struct.foreignKeys.add(_elem1582); + _elem1598 = new SQLForeignKey(); + _elem1598.read(iprot); + struct.foreignKeys.add(_elem1598); } iprot.readListEnd(); } @@ -79331,14 +79331,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 4: // UNIQUE_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1584 = iprot.readListBegin(); - struct.uniqueConstraints = new java.util.ArrayList(_list1584.size); - @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1585; - for (int _i1586 = 0; _i1586 < _list1584.size; ++_i1586) + org.apache.thrift.protocol.TList _list1600 = iprot.readListBegin(); + struct.uniqueConstraints = new java.util.ArrayList(_list1600.size); + @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1601; + for (int _i1602 = 0; _i1602 < _list1600.size; ++_i1602) { - _elem1585 = new SQLUniqueConstraint(); - _elem1585.read(iprot); - struct.uniqueConstraints.add(_elem1585); + _elem1601 = new SQLUniqueConstraint(); + _elem1601.read(iprot); + struct.uniqueConstraints.add(_elem1601); } iprot.readListEnd(); } @@ -79350,14 +79350,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 5: // NOT_NULL_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1587 = iprot.readListBegin(); - struct.notNullConstraints = new java.util.ArrayList(_list1587.size); - @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1588; - for (int _i1589 = 0; _i1589 < _list1587.size; ++_i1589) + org.apache.thrift.protocol.TList _list1603 = iprot.readListBegin(); + struct.notNullConstraints = new java.util.ArrayList(_list1603.size); + @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1604; + for (int _i1605 = 0; _i1605 < _list1603.size; ++_i1605) { - _elem1588 = new SQLNotNullConstraint(); - _elem1588.read(iprot); - struct.notNullConstraints.add(_elem1588); + _elem1604 = new SQLNotNullConstraint(); + _elem1604.read(iprot); + struct.notNullConstraints.add(_elem1604); } iprot.readListEnd(); } @@ -79369,14 +79369,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 6: // DEFAULT_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1590 = iprot.readListBegin(); - struct.defaultConstraints = new java.util.ArrayList(_list1590.size); - @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1591; - for (int _i1592 = 0; _i1592 < _list1590.size; ++_i1592) + org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin(); + struct.defaultConstraints = new java.util.ArrayList(_list1606.size); + @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1607; + for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608) { - _elem1591 = new SQLDefaultConstraint(); - _elem1591.read(iprot); - struct.defaultConstraints.add(_elem1591); + _elem1607 = new SQLDefaultConstraint(); + _elem1607.read(iprot); + struct.defaultConstraints.add(_elem1607); } iprot.readListEnd(); } @@ -79388,14 +79388,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 7: // CHECK_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1593 = iprot.readListBegin(); - struct.checkConstraints = new java.util.ArrayList(_list1593.size); - @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1594; - for (int _i1595 = 0; _i1595 < _list1593.size; ++_i1595) + org.apache.thrift.protocol.TList _list1609 = iprot.readListBegin(); + struct.checkConstraints = new java.util.ArrayList(_list1609.size); + @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1610; + for (int _i1611 = 0; _i1611 < _list1609.size; ++_i1611) { - _elem1594 = new SQLCheckConstraint(); - _elem1594.read(iprot); - struct.checkConstraints.add(_elem1594); + _elem1610 = new SQLCheckConstraint(); + _elem1610.read(iprot); + struct.checkConstraints.add(_elem1610); } iprot.readListEnd(); } @@ -79426,9 +79426,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size())); - for (SQLPrimaryKey _iter1596 : struct.primaryKeys) + for (SQLPrimaryKey _iter1612 : struct.primaryKeys) { - _iter1596.write(oprot); + _iter1612.write(oprot); } oprot.writeListEnd(); } @@ -79438,9 +79438,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size())); - for (SQLForeignKey _iter1597 : struct.foreignKeys) + for (SQLForeignKey _iter1613 : struct.foreignKeys) { - _iter1597.write(oprot); + _iter1613.write(oprot); } oprot.writeListEnd(); } @@ -79450,9 +79450,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size())); - for (SQLUniqueConstraint _iter1598 : struct.uniqueConstraints) + for (SQLUniqueConstraint _iter1614 : struct.uniqueConstraints) { - _iter1598.write(oprot); + _iter1614.write(oprot); } oprot.writeListEnd(); } @@ -79462,9 +79462,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size())); - for (SQLNotNullConstraint _iter1599 : struct.notNullConstraints) + for (SQLNotNullConstraint _iter1615 : struct.notNullConstraints) { - _iter1599.write(oprot); + _iter1615.write(oprot); } oprot.writeListEnd(); } @@ -79474,9 +79474,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size())); - for (SQLDefaultConstraint _iter1600 : struct.defaultConstraints) + for (SQLDefaultConstraint _iter1616 : struct.defaultConstraints) { - _iter1600.write(oprot); + _iter1616.write(oprot); } oprot.writeListEnd(); } @@ -79486,9 +79486,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size())); - for (SQLCheckConstraint _iter1601 : struct.checkConstraints) + for (SQLCheckConstraint _iter1617 : struct.checkConstraints) { - _iter1601.write(oprot); + _iter1617.write(oprot); } oprot.writeListEnd(); } @@ -79540,54 +79540,54 @@ public void write(org.apache.thrift.protocol.TProtocol prot, create_table_with_c if (struct.isSetPrimaryKeys()) { { oprot.writeI32(struct.primaryKeys.size()); - for (SQLPrimaryKey _iter1602 : struct.primaryKeys) + for (SQLPrimaryKey _iter1618 : struct.primaryKeys) { - _iter1602.write(oprot); + _iter1618.write(oprot); } } } if (struct.isSetForeignKeys()) { { oprot.writeI32(struct.foreignKeys.size()); - for (SQLForeignKey _iter1603 : struct.foreignKeys) + for (SQLForeignKey _iter1619 : struct.foreignKeys) { - _iter1603.write(oprot); + _iter1619.write(oprot); } } } if (struct.isSetUniqueConstraints()) { { oprot.writeI32(struct.uniqueConstraints.size()); - for (SQLUniqueConstraint _iter1604 : struct.uniqueConstraints) + for (SQLUniqueConstraint _iter1620 : struct.uniqueConstraints) { - _iter1604.write(oprot); + _iter1620.write(oprot); } } } if (struct.isSetNotNullConstraints()) { { oprot.writeI32(struct.notNullConstraints.size()); - for (SQLNotNullConstraint _iter1605 : struct.notNullConstraints) + for (SQLNotNullConstraint _iter1621 : struct.notNullConstraints) { - _iter1605.write(oprot); + _iter1621.write(oprot); } } } if (struct.isSetDefaultConstraints()) { { oprot.writeI32(struct.defaultConstraints.size()); - for (SQLDefaultConstraint _iter1606 : struct.defaultConstraints) + for (SQLDefaultConstraint _iter1622 : struct.defaultConstraints) { - _iter1606.write(oprot); + _iter1622.write(oprot); } } } if (struct.isSetCheckConstraints()) { { oprot.writeI32(struct.checkConstraints.size()); - for (SQLCheckConstraint _iter1607 : struct.checkConstraints) + for (SQLCheckConstraint _iter1623 : struct.checkConstraints) { - _iter1607.write(oprot); + _iter1623.write(oprot); } } } @@ -79604,84 +79604,84 @@ public void read(org.apache.thrift.protocol.TProtocol prot, create_table_with_co } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1608 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.primaryKeys = new java.util.ArrayList(_list1608.size); - @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1609; - for (int _i1610 = 0; _i1610 < _list1608.size; ++_i1610) + org.apache.thrift.protocol.TList _list1624 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.primaryKeys = new java.util.ArrayList(_list1624.size); + @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1625; + for (int _i1626 = 0; _i1626 < _list1624.size; ++_i1626) { - _elem1609 = new SQLPrimaryKey(); - _elem1609.read(iprot); - struct.primaryKeys.add(_elem1609); + _elem1625 = new SQLPrimaryKey(); + _elem1625.read(iprot); + struct.primaryKeys.add(_elem1625); } } struct.setPrimaryKeysIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1611 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.foreignKeys = new java.util.ArrayList(_list1611.size); - @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1612; - for (int _i1613 = 0; _i1613 < _list1611.size; ++_i1613) + org.apache.thrift.protocol.TList _list1627 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.foreignKeys = new java.util.ArrayList(_list1627.size); + @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1628; + for (int _i1629 = 0; _i1629 < _list1627.size; ++_i1629) { - _elem1612 = new SQLForeignKey(); - _elem1612.read(iprot); - struct.foreignKeys.add(_elem1612); + _elem1628 = new SQLForeignKey(); + _elem1628.read(iprot); + struct.foreignKeys.add(_elem1628); } } struct.setForeignKeysIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.uniqueConstraints = new java.util.ArrayList(_list1614.size); - @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1615; - for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616) + org.apache.thrift.protocol.TList _list1630 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.uniqueConstraints = new java.util.ArrayList(_list1630.size); + @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1631; + for (int _i1632 = 0; _i1632 < _list1630.size; ++_i1632) { - _elem1615 = new SQLUniqueConstraint(); - _elem1615.read(iprot); - struct.uniqueConstraints.add(_elem1615); + _elem1631 = new SQLUniqueConstraint(); + _elem1631.read(iprot); + struct.uniqueConstraints.add(_elem1631); } } struct.setUniqueConstraintsIsSet(true); } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1617 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.notNullConstraints = new java.util.ArrayList(_list1617.size); - @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1618; - for (int _i1619 = 0; _i1619 < _list1617.size; ++_i1619) + org.apache.thrift.protocol.TList _list1633 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.notNullConstraints = new java.util.ArrayList(_list1633.size); + @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1634; + for (int _i1635 = 0; _i1635 < _list1633.size; ++_i1635) { - _elem1618 = new SQLNotNullConstraint(); - _elem1618.read(iprot); - struct.notNullConstraints.add(_elem1618); + _elem1634 = new SQLNotNullConstraint(); + _elem1634.read(iprot); + struct.notNullConstraints.add(_elem1634); } } struct.setNotNullConstraintsIsSet(true); } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1620 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.defaultConstraints = new java.util.ArrayList(_list1620.size); - @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1621; - for (int _i1622 = 0; _i1622 < _list1620.size; ++_i1622) + org.apache.thrift.protocol.TList _list1636 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.defaultConstraints = new java.util.ArrayList(_list1636.size); + @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1637; + for (int _i1638 = 0; _i1638 < _list1636.size; ++_i1638) { - _elem1621 = new SQLDefaultConstraint(); - _elem1621.read(iprot); - struct.defaultConstraints.add(_elem1621); + _elem1637 = new SQLDefaultConstraint(); + _elem1637.read(iprot); + struct.defaultConstraints.add(_elem1637); } } struct.setDefaultConstraintsIsSet(true); } if (incoming.get(6)) { { - org.apache.thrift.protocol.TList _list1623 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.checkConstraints = new java.util.ArrayList(_list1623.size); - @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1624; - for (int _i1625 = 0; _i1625 < _list1623.size; ++_i1625) + org.apache.thrift.protocol.TList _list1639 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.checkConstraints = new java.util.ArrayList(_list1639.size); + @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1640; + for (int _i1641 = 0; _i1641 < _list1639.size; ++_i1641) { - _elem1624 = new SQLCheckConstraint(); - _elem1624.read(iprot); - struct.checkConstraints.add(_elem1624); + _elem1640 = new SQLCheckConstraint(); + _elem1640.read(iprot); + struct.checkConstraints.add(_elem1640); } } struct.setCheckConstraintsIsSet(true); @@ -91113,13 +91113,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, truncate_table_args case 3: // PART_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1626 = iprot.readListBegin(); - struct.partNames = new java.util.ArrayList(_list1626.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1627; - for (int _i1628 = 0; _i1628 < _list1626.size; ++_i1628) + org.apache.thrift.protocol.TList _list1642 = iprot.readListBegin(); + struct.partNames = new java.util.ArrayList(_list1642.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1643; + for (int _i1644 = 0; _i1644 < _list1642.size; ++_i1644) { - _elem1627 = iprot.readString(); - struct.partNames.add(_elem1627); + _elem1643 = iprot.readString(); + struct.partNames.add(_elem1643); } iprot.readListEnd(); } @@ -91155,9 +91155,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, truncate_table_arg oprot.writeFieldBegin(PART_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size())); - for (java.lang.String _iter1629 : struct.partNames) + for (java.lang.String _iter1645 : struct.partNames) { - oprot.writeString(_iter1629); + oprot.writeString(_iter1645); } oprot.writeListEnd(); } @@ -91200,9 +91200,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, truncate_table_args if (struct.isSetPartNames()) { { oprot.writeI32(struct.partNames.size()); - for (java.lang.String _iter1630 : struct.partNames) + for (java.lang.String _iter1646 : struct.partNames) { - oprot.writeString(_iter1630); + oprot.writeString(_iter1646); } } } @@ -91222,13 +91222,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, truncate_table_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1631 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partNames = new java.util.ArrayList(_list1631.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1632; - for (int _i1633 = 0; _i1633 < _list1631.size; ++_i1633) + org.apache.thrift.protocol.TList _list1647 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partNames = new java.util.ArrayList(_list1647.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1648; + for (int _i1649 = 0; _i1649 < _list1647.size; ++_i1649) { - _elem1632 = iprot.readString(); - struct.partNames.add(_elem1632); + _elem1648 = iprot.readString(); + struct.partNames.add(_elem1648); } } struct.setPartNamesIsSet(true); @@ -93306,13 +93306,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1634 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1634.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1635; - for (int _i1636 = 0; _i1636 < _list1634.size; ++_i1636) + org.apache.thrift.protocol.TList _list1650 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1650.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1651; + for (int _i1652 = 0; _i1652 < _list1650.size; ++_i1652) { - _elem1635 = iprot.readString(); - struct.success.add(_elem1635); + _elem1651 = iprot.readString(); + struct.success.add(_elem1651); } iprot.readListEnd(); } @@ -93347,9 +93347,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1637 : struct.success) + for (java.lang.String _iter1653 : struct.success) { - oprot.writeString(_iter1637); + oprot.writeString(_iter1653); } oprot.writeListEnd(); } @@ -93388,9 +93388,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1638 : struct.success) + for (java.lang.String _iter1654 : struct.success) { - oprot.writeString(_iter1638); + oprot.writeString(_iter1654); } } } @@ -93405,13 +93405,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result st java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1639 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1639.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1640; - for (int _i1641 = 0; _i1641 < _list1639.size; ++_i1641) + org.apache.thrift.protocol.TList _list1655 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1655.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1656; + for (int _i1657 = 0; _i1657 < _list1655.size; ++_i1657) { - _elem1640 = iprot.readString(); - struct.success.add(_elem1640); + _elem1656 = iprot.readString(); + struct.success.add(_elem1656); } } struct.setSuccessIsSet(true); @@ -94394,13 +94394,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_by_type_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1642 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1642.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1643; - for (int _i1644 = 0; _i1644 < _list1642.size; ++_i1644) + org.apache.thrift.protocol.TList _list1658 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1658.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1659; + for (int _i1660 = 0; _i1660 < _list1658.size; ++_i1660) { - _elem1643 = iprot.readString(); - struct.success.add(_elem1643); + _elem1659 = iprot.readString(); + struct.success.add(_elem1659); } iprot.readListEnd(); } @@ -94435,9 +94435,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_by_type oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1645 : struct.success) + for (java.lang.String _iter1661 : struct.success) { - oprot.writeString(_iter1645); + oprot.writeString(_iter1661); } oprot.writeListEnd(); } @@ -94476,9 +94476,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1646 : struct.success) + for (java.lang.String _iter1662 : struct.success) { - oprot.writeString(_iter1646); + oprot.writeString(_iter1662); } } } @@ -94493,13 +94493,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_r java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1647 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1647.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1648; - for (int _i1649 = 0; _i1649 < _list1647.size; ++_i1649) + org.apache.thrift.protocol.TList _list1663 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1663.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1664; + for (int _i1665 = 0; _i1665 < _list1663.size; ++_i1665) { - _elem1648 = iprot.readString(); - struct.success.add(_elem1648); + _elem1664 = iprot.readString(); + struct.success.add(_elem1664); } } struct.setSuccessIsSet(true); @@ -95165,14 +95165,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_materialize case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1650 = iprot.readListBegin(); - struct.success = new java.util.ArrayList
(_list1650.size); - @org.apache.thrift.annotation.Nullable Table _elem1651; - for (int _i1652 = 0; _i1652 < _list1650.size; ++_i1652) + org.apache.thrift.protocol.TList _list1666 = iprot.readListBegin(); + struct.success = new java.util.ArrayList
(_list1666.size); + @org.apache.thrift.annotation.Nullable Table _elem1667; + for (int _i1668 = 0; _i1668 < _list1666.size; ++_i1668) { - _elem1651 = new Table(); - _elem1651.read(iprot); - struct.success.add(_elem1651); + _elem1667 = new Table(); + _elem1667.read(iprot); + struct.success.add(_elem1667); } iprot.readListEnd(); } @@ -95207,9 +95207,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_materializ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Table _iter1653 : struct.success) + for (Table _iter1669 : struct.success) { - _iter1653.write(oprot); + _iter1669.write(oprot); } oprot.writeListEnd(); } @@ -95248,9 +95248,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_materialize if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Table _iter1654 : struct.success) + for (Table _iter1670 : struct.success) { - _iter1654.write(oprot); + _iter1670.write(oprot); } } } @@ -95265,14 +95265,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_materialized java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1655 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList
(_list1655.size); - @org.apache.thrift.annotation.Nullable Table _elem1656; - for (int _i1657 = 0; _i1657 < _list1655.size; ++_i1657) + org.apache.thrift.protocol.TList _list1671 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList
(_list1671.size); + @org.apache.thrift.annotation.Nullable Table _elem1672; + for (int _i1673 = 0; _i1673 < _list1671.size; ++_i1673) { - _elem1656 = new Table(); - _elem1656.read(iprot); - struct.success.add(_elem1656); + _elem1672 = new Table(); + _elem1672.read(iprot); + struct.success.add(_elem1672); } } struct.setSuccessIsSet(true); @@ -96047,13 +96047,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_materialized_vi case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1658 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1658.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1659; - for (int _i1660 = 0; _i1660 < _list1658.size; ++_i1660) + org.apache.thrift.protocol.TList _list1674 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1674.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1675; + for (int _i1676 = 0; _i1676 < _list1674.size; ++_i1676) { - _elem1659 = iprot.readString(); - struct.success.add(_elem1659); + _elem1675 = iprot.readString(); + struct.success.add(_elem1675); } iprot.readListEnd(); } @@ -96088,9 +96088,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_materialized_v oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1661 : struct.success) + for (java.lang.String _iter1677 : struct.success) { - oprot.writeString(_iter1661); + oprot.writeString(_iter1677); } oprot.writeListEnd(); } @@ -96129,9 +96129,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_materialized_vi if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1662 : struct.success) + for (java.lang.String _iter1678 : struct.success) { - oprot.writeString(_iter1662); + oprot.writeString(_iter1678); } } } @@ -96146,13 +96146,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_materialized_vie java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1663 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1663.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1664; - for (int _i1665 = 0; _i1665 < _list1663.size; ++_i1665) + org.apache.thrift.protocol.TList _list1679 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1679.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1680; + for (int _i1681 = 0; _i1681 < _list1679.size; ++_i1681) { - _elem1664 = iprot.readString(); - struct.success.add(_elem1664); + _elem1680 = iprot.readString(); + struct.success.add(_elem1680); } } struct.setSuccessIsSet(true); @@ -96662,13 +96662,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_args case 3: // TBL_TYPES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1666 = iprot.readListBegin(); - struct.tbl_types = new java.util.ArrayList(_list1666.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1667; - for (int _i1668 = 0; _i1668 < _list1666.size; ++_i1668) + org.apache.thrift.protocol.TList _list1682 = iprot.readListBegin(); + struct.tbl_types = new java.util.ArrayList(_list1682.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1683; + for (int _i1684 = 0; _i1684 < _list1682.size; ++_i1684) { - _elem1667 = iprot.readString(); - struct.tbl_types.add(_elem1667); + _elem1683 = iprot.readString(); + struct.tbl_types.add(_elem1683); } iprot.readListEnd(); } @@ -96704,9 +96704,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_arg oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size())); - for (java.lang.String _iter1669 : struct.tbl_types) + for (java.lang.String _iter1685 : struct.tbl_types) { - oprot.writeString(_iter1669); + oprot.writeString(_iter1685); } oprot.writeListEnd(); } @@ -96749,9 +96749,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args if (struct.isSetTbl_types()) { { oprot.writeI32(struct.tbl_types.size()); - for (java.lang.String _iter1670 : struct.tbl_types) + for (java.lang.String _iter1686 : struct.tbl_types) { - oprot.writeString(_iter1670); + oprot.writeString(_iter1686); } } } @@ -96771,13 +96771,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1671 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.tbl_types = new java.util.ArrayList(_list1671.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1672; - for (int _i1673 = 0; _i1673 < _list1671.size; ++_i1673) + org.apache.thrift.protocol.TList _list1687 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tbl_types = new java.util.ArrayList(_list1687.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1688; + for (int _i1689 = 0; _i1689 < _list1687.size; ++_i1689) { - _elem1672 = iprot.readString(); - struct.tbl_types.add(_elem1672); + _elem1688 = iprot.readString(); + struct.tbl_types.add(_elem1688); } } struct.setTbl_typesIsSet(true); @@ -97188,14 +97188,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1674 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1674.size); - @org.apache.thrift.annotation.Nullable TableMeta _elem1675; - for (int _i1676 = 0; _i1676 < _list1674.size; ++_i1676) + org.apache.thrift.protocol.TList _list1690 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1690.size); + @org.apache.thrift.annotation.Nullable TableMeta _elem1691; + for (int _i1692 = 0; _i1692 < _list1690.size; ++_i1692) { - _elem1675 = new TableMeta(); - _elem1675.read(iprot); - struct.success.add(_elem1675); + _elem1691 = new TableMeta(); + _elem1691.read(iprot); + struct.success.add(_elem1691); } iprot.readListEnd(); } @@ -97230,9 +97230,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TableMeta _iter1677 : struct.success) + for (TableMeta _iter1693 : struct.success) { - _iter1677.write(oprot); + _iter1693.write(oprot); } oprot.writeListEnd(); } @@ -97271,9 +97271,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TableMeta _iter1678 : struct.success) + for (TableMeta _iter1694 : struct.success) { - _iter1678.write(oprot); + _iter1694.write(oprot); } } } @@ -97288,14 +97288,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1679 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1679.size); - @org.apache.thrift.annotation.Nullable TableMeta _elem1680; - for (int _i1681 = 0; _i1681 < _list1679.size; ++_i1681) + org.apache.thrift.protocol.TList _list1695 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1695.size); + @org.apache.thrift.annotation.Nullable TableMeta _elem1696; + for (int _i1697 = 0; _i1697 < _list1695.size; ++_i1697) { - _elem1680 = new TableMeta(); - _elem1680.read(iprot); - struct.success.add(_elem1680); + _elem1696 = new TableMeta(); + _elem1696.read(iprot); + struct.success.add(_elem1696); } } struct.setSuccessIsSet(true); @@ -98070,13 +98070,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_tables_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1682 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1682.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1683; - for (int _i1684 = 0; _i1684 < _list1682.size; ++_i1684) + org.apache.thrift.protocol.TList _list1698 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1698.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1699; + for (int _i1700 = 0; _i1700 < _list1698.size; ++_i1700) { - _elem1683 = iprot.readString(); - struct.success.add(_elem1683); + _elem1699 = iprot.readString(); + struct.success.add(_elem1699); } iprot.readListEnd(); } @@ -98111,9 +98111,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_tables_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1685 : struct.success) + for (java.lang.String _iter1701 : struct.success) { - oprot.writeString(_iter1685); + oprot.writeString(_iter1701); } oprot.writeListEnd(); } @@ -98152,9 +98152,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_tables_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1686 : struct.success) + for (java.lang.String _iter1702 : struct.success) { - oprot.writeString(_iter1686); + oprot.writeString(_iter1702); } } } @@ -98169,13 +98169,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_tables_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1687 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1687.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1688; - for (int _i1689 = 0; _i1689 < _list1687.size; ++_i1689) + org.apache.thrift.protocol.TList _list1703 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1703.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1704; + for (int _i1705 = 0; _i1705 < _list1703.size; ++_i1705) { - _elem1688 = iprot.readString(); - struct.success.add(_elem1688); + _elem1704 = iprot.readString(); + struct.success.add(_elem1704); } } struct.setSuccessIsSet(true); @@ -99641,13 +99641,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_objects_b case 2: // TBL_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1690 = iprot.readListBegin(); - struct.tbl_names = new java.util.ArrayList(_list1690.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1691; - for (int _i1692 = 0; _i1692 < _list1690.size; ++_i1692) + org.apache.thrift.protocol.TList _list1706 = iprot.readListBegin(); + struct.tbl_names = new java.util.ArrayList(_list1706.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1707; + for (int _i1708 = 0; _i1708 < _list1706.size; ++_i1708) { - _elem1691 = iprot.readString(); - struct.tbl_names.add(_elem1691); + _elem1707 = iprot.readString(); + struct.tbl_names.add(_elem1707); } iprot.readListEnd(); } @@ -99678,9 +99678,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_objects_ oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size())); - for (java.lang.String _iter1693 : struct.tbl_names) + for (java.lang.String _iter1709 : struct.tbl_names) { - oprot.writeString(_iter1693); + oprot.writeString(_iter1709); } oprot.writeListEnd(); } @@ -99717,9 +99717,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_objects_b if (struct.isSetTbl_names()) { { oprot.writeI32(struct.tbl_names.size()); - for (java.lang.String _iter1694 : struct.tbl_names) + for (java.lang.String _iter1710 : struct.tbl_names) { - oprot.writeString(_iter1694); + oprot.writeString(_iter1710); } } } @@ -99735,13 +99735,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_objects_by } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1695 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.tbl_names = new java.util.ArrayList(_list1695.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1696; - for (int _i1697 = 0; _i1697 < _list1695.size; ++_i1697) + org.apache.thrift.protocol.TList _list1711 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tbl_names = new java.util.ArrayList(_list1711.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1712; + for (int _i1713 = 0; _i1713 < _list1711.size; ++_i1713) { - _elem1696 = iprot.readString(); - struct.tbl_names.add(_elem1696); + _elem1712 = iprot.readString(); + struct.tbl_names.add(_elem1712); } } struct.setTbl_namesIsSet(true); @@ -100071,14 +100071,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_objects_b case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1698 = iprot.readListBegin(); - struct.success = new java.util.ArrayList
(_list1698.size); - @org.apache.thrift.annotation.Nullable Table _elem1699; - for (int _i1700 = 0; _i1700 < _list1698.size; ++_i1700) + org.apache.thrift.protocol.TList _list1714 = iprot.readListBegin(); + struct.success = new java.util.ArrayList
(_list1714.size); + @org.apache.thrift.annotation.Nullable Table _elem1715; + for (int _i1716 = 0; _i1716 < _list1714.size; ++_i1716) { - _elem1699 = new Table(); - _elem1699.read(iprot); - struct.success.add(_elem1699); + _elem1715 = new Table(); + _elem1715.read(iprot); + struct.success.add(_elem1715); } iprot.readListEnd(); } @@ -100104,9 +100104,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_objects_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Table _iter1701 : struct.success) + for (Table _iter1717 : struct.success) { - _iter1701.write(oprot); + _iter1717.write(oprot); } oprot.writeListEnd(); } @@ -100137,9 +100137,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_objects_b if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Table _iter1702 : struct.success) + for (Table _iter1718 : struct.success) { - _iter1702.write(oprot); + _iter1718.write(oprot); } } } @@ -100151,14 +100151,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_objects_by java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1703 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList
(_list1703.size); - @org.apache.thrift.annotation.Nullable Table _elem1704; - for (int _i1705 = 0; _i1705 < _list1703.size; ++_i1705) + org.apache.thrift.protocol.TList _list1719 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList
(_list1719.size); + @org.apache.thrift.annotation.Nullable Table _elem1720; + for (int _i1721 = 0; _i1721 < _list1719.size; ++_i1721) { - _elem1704 = new Table(); - _elem1704.read(iprot); - struct.success.add(_elem1704); + _elem1720 = new Table(); + _elem1720.read(iprot); + struct.success.add(_elem1720); } } struct.setSuccessIsSet(true); @@ -100936,14 +100936,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_ext_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1706 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1706.size); - @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1707; - for (int _i1708 = 0; _i1708 < _list1706.size; ++_i1708) + org.apache.thrift.protocol.TList _list1722 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1722.size); + @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1723; + for (int _i1724 = 0; _i1724 < _list1722.size; ++_i1724) { - _elem1707 = new ExtendedTableInfo(); - _elem1707.read(iprot); - struct.success.add(_elem1707); + _elem1723 = new ExtendedTableInfo(); + _elem1723.read(iprot); + struct.success.add(_elem1723); } iprot.readListEnd(); } @@ -100978,9 +100978,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_ext_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (ExtendedTableInfo _iter1709 : struct.success) + for (ExtendedTableInfo _iter1725 : struct.success) { - _iter1709.write(oprot); + _iter1725.write(oprot); } oprot.writeListEnd(); } @@ -101019,9 +101019,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_ext_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (ExtendedTableInfo _iter1710 : struct.success) + for (ExtendedTableInfo _iter1726 : struct.success) { - _iter1710.write(oprot); + _iter1726.write(oprot); } } } @@ -101036,14 +101036,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_ext_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1711 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1711.size); - @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1712; - for (int _i1713 = 0; _i1713 < _list1711.size; ++_i1713) + org.apache.thrift.protocol.TList _list1727 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1727.size); + @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1728; + for (int _i1729 = 0; _i1729 < _list1727.size; ++_i1729) { - _elem1712 = new ExtendedTableInfo(); - _elem1712.read(iprot); - struct.success.add(_elem1712); + _elem1728 = new ExtendedTableInfo(); + _elem1728.read(iprot); + struct.success.add(_elem1728); } } struct.setSuccessIsSet(true); @@ -106594,13 +106594,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_names_by_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1714 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1714.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1715; - for (int _i1716 = 0; _i1716 < _list1714.size; ++_i1716) + org.apache.thrift.protocol.TList _list1730 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1730.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1731; + for (int _i1732 = 0; _i1732 < _list1730.size; ++_i1732) { - _elem1715 = iprot.readString(); - struct.success.add(_elem1715); + _elem1731 = iprot.readString(); + struct.success.add(_elem1731); } iprot.readListEnd(); } @@ -106653,9 +106653,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_names_by oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1717 : struct.success) + for (java.lang.String _iter1733 : struct.success) { - oprot.writeString(_iter1717); + oprot.writeString(_iter1733); } oprot.writeListEnd(); } @@ -106710,9 +106710,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_names_by_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1718 : struct.success) + for (java.lang.String _iter1734 : struct.success) { - oprot.writeString(_iter1718); + oprot.writeString(_iter1734); } } } @@ -106733,13 +106733,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_names_by_f java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1719 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1719.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1720; - for (int _i1721 = 0; _i1721 < _list1719.size; ++_i1721) + org.apache.thrift.protocol.TList _list1735 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1735.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1736; + for (int _i1737 = 0; _i1737 < _list1735.size; ++_i1737) { - _elem1720 = iprot.readString(); - struct.success.add(_elem1720); + _elem1736 = iprot.readString(); + struct.success.add(_elem1736); } } struct.setSuccessIsSet(true); @@ -113586,14 +113586,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_partitions_args case 1: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1722 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1722.size); - @org.apache.thrift.annotation.Nullable Partition _elem1723; - for (int _i1724 = 0; _i1724 < _list1722.size; ++_i1724) + org.apache.thrift.protocol.TList _list1738 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1738.size); + @org.apache.thrift.annotation.Nullable Partition _elem1739; + for (int _i1740 = 0; _i1740 < _list1738.size; ++_i1740) { - _elem1723 = new Partition(); - _elem1723.read(iprot); - struct.new_parts.add(_elem1723); + _elem1739 = new Partition(); + _elem1739.read(iprot); + struct.new_parts.add(_elem1739); } iprot.readListEnd(); } @@ -113619,9 +113619,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_partitions_arg oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter1725 : struct.new_parts) + for (Partition _iter1741 : struct.new_parts) { - _iter1725.write(oprot); + _iter1741.write(oprot); } oprot.writeListEnd(); } @@ -113652,9 +113652,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_partitions_args if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter1726 : struct.new_parts) + for (Partition _iter1742 : struct.new_parts) { - _iter1726.write(oprot); + _iter1742.write(oprot); } } } @@ -113666,14 +113666,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_partitions_args java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1727 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1727.size); - @org.apache.thrift.annotation.Nullable Partition _elem1728; - for (int _i1729 = 0; _i1729 < _list1727.size; ++_i1729) + org.apache.thrift.protocol.TList _list1743 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1743.size); + @org.apache.thrift.annotation.Nullable Partition _elem1744; + for (int _i1745 = 0; _i1745 < _list1743.size; ++_i1745) { - _elem1728 = new Partition(); - _elem1728.read(iprot); - struct.new_parts.add(_elem1728); + _elem1744 = new Partition(); + _elem1744.read(iprot); + struct.new_parts.add(_elem1744); } } struct.setNew_partsIsSet(true); @@ -114680,14 +114680,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_partitions_pspe case 1: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1730 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1730.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1731; - for (int _i1732 = 0; _i1732 < _list1730.size; ++_i1732) + org.apache.thrift.protocol.TList _list1746 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1746.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1747; + for (int _i1748 = 0; _i1748 < _list1746.size; ++_i1748) { - _elem1731 = new PartitionSpec(); - _elem1731.read(iprot); - struct.new_parts.add(_elem1731); + _elem1747 = new PartitionSpec(); + _elem1747.read(iprot); + struct.new_parts.add(_elem1747); } iprot.readListEnd(); } @@ -114713,9 +114713,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_partitions_psp oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (PartitionSpec _iter1733 : struct.new_parts) + for (PartitionSpec _iter1749 : struct.new_parts) { - _iter1733.write(oprot); + _iter1749.write(oprot); } oprot.writeListEnd(); } @@ -114746,9 +114746,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_partitions_pspe if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (PartitionSpec _iter1734 : struct.new_parts) + for (PartitionSpec _iter1750 : struct.new_parts) { - _iter1734.write(oprot); + _iter1750.write(oprot); } } } @@ -114760,14 +114760,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_partitions_pspec java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1735 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1735.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1736; - for (int _i1737 = 0; _i1737 < _list1735.size; ++_i1737) + org.apache.thrift.protocol.TList _list1751 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1751.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1752; + for (int _i1753 = 0; _i1753 < _list1751.size; ++_i1753) { - _elem1736 = new PartitionSpec(); - _elem1736.read(iprot); - struct.new_parts.add(_elem1736); + _elem1752 = new PartitionSpec(); + _elem1752.read(iprot); + struct.new_parts.add(_elem1752); } } struct.setNew_partsIsSet(true); @@ -115949,13 +115949,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, append_partition_ar case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1738 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1738.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1739; - for (int _i1740 = 0; _i1740 < _list1738.size; ++_i1740) + org.apache.thrift.protocol.TList _list1754 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1754.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1755; + for (int _i1756 = 0; _i1756 < _list1754.size; ++_i1756) { - _elem1739 = iprot.readString(); - struct.part_vals.add(_elem1739); + _elem1755 = iprot.readString(); + struct.part_vals.add(_elem1755); } iprot.readListEnd(); } @@ -115991,9 +115991,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, append_partition_a oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1741 : struct.part_vals) + for (java.lang.String _iter1757 : struct.part_vals) { - oprot.writeString(_iter1741); + oprot.writeString(_iter1757); } oprot.writeListEnd(); } @@ -116036,9 +116036,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, append_partition_ar if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1742 : struct.part_vals) + for (java.lang.String _iter1758 : struct.part_vals) { - oprot.writeString(_iter1742); + oprot.writeString(_iter1758); } } } @@ -116058,13 +116058,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, append_partition_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1743 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1743.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1744; - for (int _i1745 = 0; _i1745 < _list1743.size; ++_i1745) + org.apache.thrift.protocol.TList _list1759 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1759.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1760; + for (int _i1761 = 0; _i1761 < _list1759.size; ++_i1761) { - _elem1744 = iprot.readString(); - struct.part_vals.add(_elem1744); + _elem1760 = iprot.readString(); + struct.part_vals.add(_elem1760); } } struct.setPart_valsIsSet(true); @@ -118390,13 +118390,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, append_partition_wi case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1746 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1746.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1747; - for (int _i1748 = 0; _i1748 < _list1746.size; ++_i1748) + org.apache.thrift.protocol.TList _list1762 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1762.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1763; + for (int _i1764 = 0; _i1764 < _list1762.size; ++_i1764) { - _elem1747 = iprot.readString(); - struct.part_vals.add(_elem1747); + _elem1763 = iprot.readString(); + struct.part_vals.add(_elem1763); } iprot.readListEnd(); } @@ -118441,9 +118441,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, append_partition_w oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1749 : struct.part_vals) + for (java.lang.String _iter1765 : struct.part_vals) { - oprot.writeString(_iter1749); + oprot.writeString(_iter1765); } oprot.writeListEnd(); } @@ -118494,9 +118494,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, append_partition_wi if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1750 : struct.part_vals) + for (java.lang.String _iter1766 : struct.part_vals) { - oprot.writeString(_iter1750); + oprot.writeString(_iter1766); } } } @@ -118519,13 +118519,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, append_partition_wit } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1751 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1751.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1752; - for (int _i1753 = 0; _i1753 < _list1751.size; ++_i1753) + org.apache.thrift.protocol.TList _list1767 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1767.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1768; + for (int _i1769 = 0; _i1769 < _list1767.size; ++_i1769) { - _elem1752 = iprot.readString(); - struct.part_vals.add(_elem1752); + _elem1768 = iprot.readString(); + struct.part_vals.add(_elem1768); } } struct.setPart_valsIsSet(true); @@ -122417,13 +122417,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, drop_partition_args case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1754 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1754.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1755; - for (int _i1756 = 0; _i1756 < _list1754.size; ++_i1756) + org.apache.thrift.protocol.TList _list1770 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1770.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1771; + for (int _i1772 = 0; _i1772 < _list1770.size; ++_i1772) { - _elem1755 = iprot.readString(); - struct.part_vals.add(_elem1755); + _elem1771 = iprot.readString(); + struct.part_vals.add(_elem1771); } iprot.readListEnd(); } @@ -122467,9 +122467,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, drop_partition_arg oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1757 : struct.part_vals) + for (java.lang.String _iter1773 : struct.part_vals) { - oprot.writeString(_iter1757); + oprot.writeString(_iter1773); } oprot.writeListEnd(); } @@ -122518,9 +122518,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, drop_partition_args if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1758 : struct.part_vals) + for (java.lang.String _iter1774 : struct.part_vals) { - oprot.writeString(_iter1758); + oprot.writeString(_iter1774); } } } @@ -122543,13 +122543,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, drop_partition_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1759 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1759.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1760; - for (int _i1761 = 0; _i1761 < _list1759.size; ++_i1761) + org.apache.thrift.protocol.TList _list1775 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1775.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1776; + for (int _i1777 = 0; _i1777 < _list1775.size; ++_i1777) { - _elem1760 = iprot.readString(); - struct.part_vals.add(_elem1760); + _elem1776 = iprot.readString(); + struct.part_vals.add(_elem1776); } } struct.setPart_valsIsSet(true); @@ -123791,13 +123791,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, drop_partition_with case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1762 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1762.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1763; - for (int _i1764 = 0; _i1764 < _list1762.size; ++_i1764) + org.apache.thrift.protocol.TList _list1778 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1778.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1779; + for (int _i1780 = 0; _i1780 < _list1778.size; ++_i1780) { - _elem1763 = iprot.readString(); - struct.part_vals.add(_elem1763); + _elem1779 = iprot.readString(); + struct.part_vals.add(_elem1779); } iprot.readListEnd(); } @@ -123850,9 +123850,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, drop_partition_wit oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1765 : struct.part_vals) + for (java.lang.String _iter1781 : struct.part_vals) { - oprot.writeString(_iter1765); + oprot.writeString(_iter1781); } oprot.writeListEnd(); } @@ -123909,9 +123909,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, drop_partition_with if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1766 : struct.part_vals) + for (java.lang.String _iter1782 : struct.part_vals) { - oprot.writeString(_iter1766); + oprot.writeString(_iter1782); } } } @@ -123937,13 +123937,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, drop_partition_with_ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1767 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1767.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1768; - for (int _i1769 = 0; _i1769 < _list1767.size; ++_i1769) + org.apache.thrift.protocol.TList _list1783 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1783.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1784; + for (int _i1785 = 0; _i1785 < _list1783.size; ++_i1785) { - _elem1768 = iprot.readString(); - struct.part_vals.add(_elem1768); + _elem1784 = iprot.readString(); + struct.part_vals.add(_elem1784); } } struct.setPart_valsIsSet(true); @@ -128563,13 +128563,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_args case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1770 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1770.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1771; - for (int _i1772 = 0; _i1772 < _list1770.size; ++_i1772) + org.apache.thrift.protocol.TList _list1786 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1786.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1787; + for (int _i1788 = 0; _i1788 < _list1786.size; ++_i1788) { - _elem1771 = iprot.readString(); - struct.part_vals.add(_elem1771); + _elem1787 = iprot.readString(); + struct.part_vals.add(_elem1787); } iprot.readListEnd(); } @@ -128605,9 +128605,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_args oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1773 : struct.part_vals) + for (java.lang.String _iter1789 : struct.part_vals) { - oprot.writeString(_iter1773); + oprot.writeString(_iter1789); } oprot.writeListEnd(); } @@ -128650,9 +128650,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_args if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1774 : struct.part_vals) + for (java.lang.String _iter1790 : struct.part_vals) { - oprot.writeString(_iter1774); + oprot.writeString(_iter1790); } } } @@ -128672,13 +128672,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_args s } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1775 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1775.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1776; - for (int _i1777 = 0; _i1777 < _list1775.size; ++_i1777) + org.apache.thrift.protocol.TList _list1791 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1791.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1792; + for (int _i1793 = 0; _i1793 < _list1791.size; ++_i1793) { - _elem1776 = iprot.readString(); - struct.part_vals.add(_elem1776); + _elem1792 = iprot.readString(); + struct.part_vals.add(_elem1792); } } struct.setPart_valsIsSet(true); @@ -130850,15 +130850,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partition_ case 1: // PARTITION_SPECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1778 = iprot.readMapBegin(); - struct.partitionSpecs = new java.util.HashMap(2*_map1778.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1779; - @org.apache.thrift.annotation.Nullable java.lang.String _val1780; - for (int _i1781 = 0; _i1781 < _map1778.size; ++_i1781) + org.apache.thrift.protocol.TMap _map1794 = iprot.readMapBegin(); + struct.partitionSpecs = new java.util.HashMap(2*_map1794.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1795; + @org.apache.thrift.annotation.Nullable java.lang.String _val1796; + for (int _i1797 = 0; _i1797 < _map1794.size; ++_i1797) { - _key1779 = iprot.readString(); - _val1780 = iprot.readString(); - struct.partitionSpecs.put(_key1779, _val1780); + _key1795 = iprot.readString(); + _val1796 = iprot.readString(); + struct.partitionSpecs.put(_key1795, _val1796); } iprot.readMapEnd(); } @@ -130916,10 +130916,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size())); - for (java.util.Map.Entry _iter1782 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1798 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1782.getKey()); - oprot.writeString(_iter1782.getValue()); + oprot.writeString(_iter1798.getKey()); + oprot.writeString(_iter1798.getValue()); } oprot.writeMapEnd(); } @@ -130982,10 +130982,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partition_ if (struct.isSetPartitionSpecs()) { { oprot.writeI32(struct.partitionSpecs.size()); - for (java.util.Map.Entry _iter1783 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1799 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1783.getKey()); - oprot.writeString(_iter1783.getValue()); + oprot.writeString(_iter1799.getKey()); + oprot.writeString(_iter1799.getValue()); } } } @@ -131009,15 +131009,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partition_a java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map1784 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.partitionSpecs = new java.util.HashMap(2*_map1784.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1785; - @org.apache.thrift.annotation.Nullable java.lang.String _val1786; - for (int _i1787 = 0; _i1787 < _map1784.size; ++_i1787) + org.apache.thrift.protocol.TMap _map1800 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.partitionSpecs = new java.util.HashMap(2*_map1800.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1801; + @org.apache.thrift.annotation.Nullable java.lang.String _val1802; + for (int _i1803 = 0; _i1803 < _map1800.size; ++_i1803) { - _key1785 = iprot.readString(); - _val1786 = iprot.readString(); - struct.partitionSpecs.put(_key1785, _val1786); + _key1801 = iprot.readString(); + _val1802 = iprot.readString(); + struct.partitionSpecs.put(_key1801, _val1802); } } struct.setPartitionSpecsIsSet(true); @@ -132471,15 +132471,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partitions case 1: // PARTITION_SPECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1788 = iprot.readMapBegin(); - struct.partitionSpecs = new java.util.HashMap(2*_map1788.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1789; - @org.apache.thrift.annotation.Nullable java.lang.String _val1790; - for (int _i1791 = 0; _i1791 < _map1788.size; ++_i1791) + org.apache.thrift.protocol.TMap _map1804 = iprot.readMapBegin(); + struct.partitionSpecs = new java.util.HashMap(2*_map1804.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1805; + @org.apache.thrift.annotation.Nullable java.lang.String _val1806; + for (int _i1807 = 0; _i1807 < _map1804.size; ++_i1807) { - _key1789 = iprot.readString(); - _val1790 = iprot.readString(); - struct.partitionSpecs.put(_key1789, _val1790); + _key1805 = iprot.readString(); + _val1806 = iprot.readString(); + struct.partitionSpecs.put(_key1805, _val1806); } iprot.readMapEnd(); } @@ -132537,10 +132537,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size())); - for (java.util.Map.Entry _iter1792 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1808 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1792.getKey()); - oprot.writeString(_iter1792.getValue()); + oprot.writeString(_iter1808.getKey()); + oprot.writeString(_iter1808.getValue()); } oprot.writeMapEnd(); } @@ -132603,10 +132603,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partitions if (struct.isSetPartitionSpecs()) { { oprot.writeI32(struct.partitionSpecs.size()); - for (java.util.Map.Entry _iter1793 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1809 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1793.getKey()); - oprot.writeString(_iter1793.getValue()); + oprot.writeString(_iter1809.getKey()); + oprot.writeString(_iter1809.getValue()); } } } @@ -132630,15 +132630,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partitions_ java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map1794 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.partitionSpecs = new java.util.HashMap(2*_map1794.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1795; - @org.apache.thrift.annotation.Nullable java.lang.String _val1796; - for (int _i1797 = 0; _i1797 < _map1794.size; ++_i1797) + org.apache.thrift.protocol.TMap _map1810 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.partitionSpecs = new java.util.HashMap(2*_map1810.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1811; + @org.apache.thrift.annotation.Nullable java.lang.String _val1812; + for (int _i1813 = 0; _i1813 < _map1810.size; ++_i1813) { - _key1795 = iprot.readString(); - _val1796 = iprot.readString(); - struct.partitionSpecs.put(_key1795, _val1796); + _key1811 = iprot.readString(); + _val1812 = iprot.readString(); + struct.partitionSpecs.put(_key1811, _val1812); } } struct.setPartitionSpecsIsSet(true); @@ -133308,14 +133308,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partitions case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1798 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1798.size); - @org.apache.thrift.annotation.Nullable Partition _elem1799; - for (int _i1800 = 0; _i1800 < _list1798.size; ++_i1800) + org.apache.thrift.protocol.TList _list1814 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1814.size); + @org.apache.thrift.annotation.Nullable Partition _elem1815; + for (int _i1816 = 0; _i1816 < _list1814.size; ++_i1816) { - _elem1799 = new Partition(); - _elem1799.read(iprot); - struct.success.add(_elem1799); + _elem1815 = new Partition(); + _elem1815.read(iprot); + struct.success.add(_elem1815); } iprot.readListEnd(); } @@ -133377,9 +133377,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1801 : struct.success) + for (Partition _iter1817 : struct.success) { - _iter1801.write(oprot); + _iter1817.write(oprot); } oprot.writeListEnd(); } @@ -133442,9 +133442,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partitions if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1802 : struct.success) + for (Partition _iter1818 : struct.success) { - _iter1802.write(oprot); + _iter1818.write(oprot); } } } @@ -133468,14 +133468,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partitions_ java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1803 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1803.size); - @org.apache.thrift.annotation.Nullable Partition _elem1804; - for (int _i1805 = 0; _i1805 < _list1803.size; ++_i1805) + org.apache.thrift.protocol.TList _list1819 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1819.size); + @org.apache.thrift.annotation.Nullable Partition _elem1820; + for (int _i1821 = 0; _i1821 < _list1819.size; ++_i1821) { - _elem1804 = new Partition(); - _elem1804.read(iprot); - struct.success.add(_elem1804); + _elem1820 = new Partition(); + _elem1820.read(iprot); + struct.success.add(_elem1820); } } struct.setSuccessIsSet(true); @@ -134180,13 +134180,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_with_ case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1806 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1806.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1807; - for (int _i1808 = 0; _i1808 < _list1806.size; ++_i1808) + org.apache.thrift.protocol.TList _list1822 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1822.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1823; + for (int _i1824 = 0; _i1824 < _list1822.size; ++_i1824) { - _elem1807 = iprot.readString(); - struct.part_vals.add(_elem1807); + _elem1823 = iprot.readString(); + struct.part_vals.add(_elem1823); } iprot.readListEnd(); } @@ -134206,13 +134206,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_with_ case 5: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1809 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list1809.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1810; - for (int _i1811 = 0; _i1811 < _list1809.size; ++_i1811) + org.apache.thrift.protocol.TList _list1825 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list1825.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1826; + for (int _i1827 = 0; _i1827 < _list1825.size; ++_i1827) { - _elem1810 = iprot.readString(); - struct.group_names.add(_elem1810); + _elem1826 = iprot.readString(); + struct.group_names.add(_elem1826); } iprot.readListEnd(); } @@ -134248,9 +134248,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_with oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1812 : struct.part_vals) + for (java.lang.String _iter1828 : struct.part_vals) { - oprot.writeString(_iter1812); + oprot.writeString(_iter1828); } oprot.writeListEnd(); } @@ -134265,9 +134265,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_with oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter1813 : struct.group_names) + for (java.lang.String _iter1829 : struct.group_names) { - oprot.writeString(_iter1813); + oprot.writeString(_iter1829); } oprot.writeListEnd(); } @@ -134316,9 +134316,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_with_ if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1814 : struct.part_vals) + for (java.lang.String _iter1830 : struct.part_vals) { - oprot.writeString(_iter1814); + oprot.writeString(_iter1830); } } } @@ -134328,9 +134328,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_with_ if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter1815 : struct.group_names) + for (java.lang.String _iter1831 : struct.group_names) { - oprot.writeString(_iter1815); + oprot.writeString(_iter1831); } } } @@ -134350,13 +134350,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_with_a } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1816 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1816.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1817; - for (int _i1818 = 0; _i1818 < _list1816.size; ++_i1818) + org.apache.thrift.protocol.TList _list1832 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1832.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1833; + for (int _i1834 = 0; _i1834 < _list1832.size; ++_i1834) { - _elem1817 = iprot.readString(); - struct.part_vals.add(_elem1817); + _elem1833 = iprot.readString(); + struct.part_vals.add(_elem1833); } } struct.setPart_valsIsSet(true); @@ -134367,13 +134367,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_with_a } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1819 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list1819.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1820; - for (int _i1821 = 0; _i1821 < _list1819.size; ++_i1821) + org.apache.thrift.protocol.TList _list1835 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list1835.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1836; + for (int _i1837 = 0; _i1837 < _list1835.size; ++_i1837) { - _elem1820 = iprot.readString(); - struct.group_names.add(_elem1820); + _elem1836 = iprot.readString(); + struct.group_names.add(_elem1836); } } struct.setGroup_namesIsSet(true); @@ -137160,14 +137160,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1822 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1822.size); - @org.apache.thrift.annotation.Nullable Partition _elem1823; - for (int _i1824 = 0; _i1824 < _list1822.size; ++_i1824) + org.apache.thrift.protocol.TList _list1838 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1838.size); + @org.apache.thrift.annotation.Nullable Partition _elem1839; + for (int _i1840 = 0; _i1840 < _list1838.size; ++_i1840) { - _elem1823 = new Partition(); - _elem1823.read(iprot); - struct.success.add(_elem1823); + _elem1839 = new Partition(); + _elem1839.read(iprot); + struct.success.add(_elem1839); } iprot.readListEnd(); } @@ -137211,9 +137211,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1825 : struct.success) + for (Partition _iter1841 : struct.success) { - _iter1825.write(oprot); + _iter1841.write(oprot); } oprot.writeListEnd(); } @@ -137260,9 +137260,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1826 : struct.success) + for (Partition _iter1842 : struct.success) { - _iter1826.write(oprot); + _iter1842.write(oprot); } } } @@ -137280,14 +137280,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_resul java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1827 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1827.size); - @org.apache.thrift.annotation.Nullable Partition _elem1828; - for (int _i1829 = 0; _i1829 < _list1827.size; ++_i1829) + org.apache.thrift.protocol.TList _list1843 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1843.size); + @org.apache.thrift.annotation.Nullable Partition _elem1844; + for (int _i1845 = 0; _i1845 < _list1843.size; ++_i1845) { - _elem1828 = new Partition(); - _elem1828.read(iprot); - struct.success.add(_elem1828); + _elem1844 = new Partition(); + _elem1844.read(iprot); + struct.success.add(_elem1844); } } struct.setSuccessIsSet(true); @@ -138925,13 +138925,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_with case 5: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1830 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list1830.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1831; - for (int _i1832 = 0; _i1832 < _list1830.size; ++_i1832) + org.apache.thrift.protocol.TList _list1846 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list1846.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1847; + for (int _i1848 = 0; _i1848 < _list1846.size; ++_i1848) { - _elem1831 = iprot.readString(); - struct.group_names.add(_elem1831); + _elem1847 = iprot.readString(); + struct.group_names.add(_elem1847); } iprot.readListEnd(); } @@ -138975,9 +138975,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_wit oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter1833 : struct.group_names) + for (java.lang.String _iter1849 : struct.group_names) { - oprot.writeString(_iter1833); + oprot.writeString(_iter1849); } oprot.writeListEnd(); } @@ -139032,9 +139032,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_with if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter1834 : struct.group_names) + for (java.lang.String _iter1850 : struct.group_names) { - oprot.writeString(_iter1834); + oprot.writeString(_iter1850); } } } @@ -139062,13 +139062,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_with_ } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1835 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list1835.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1836; - for (int _i1837 = 0; _i1837 < _list1835.size; ++_i1837) + org.apache.thrift.protocol.TList _list1851 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list1851.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1852; + for (int _i1853 = 0; _i1853 < _list1851.size; ++_i1853) { - _elem1836 = iprot.readString(); - struct.group_names.add(_elem1836); + _elem1852 = iprot.readString(); + struct.group_names.add(_elem1852); } } struct.setGroup_namesIsSet(true); @@ -139560,14 +139560,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_with case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1838 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1838.size); - @org.apache.thrift.annotation.Nullable Partition _elem1839; - for (int _i1840 = 0; _i1840 < _list1838.size; ++_i1840) + org.apache.thrift.protocol.TList _list1854 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1854.size); + @org.apache.thrift.annotation.Nullable Partition _elem1855; + for (int _i1856 = 0; _i1856 < _list1854.size; ++_i1856) { - _elem1839 = new Partition(); - _elem1839.read(iprot); - struct.success.add(_elem1839); + _elem1855 = new Partition(); + _elem1855.read(iprot); + struct.success.add(_elem1855); } iprot.readListEnd(); } @@ -139611,9 +139611,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_wit oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1841 : struct.success) + for (Partition _iter1857 : struct.success) { - _iter1841.write(oprot); + _iter1857.write(oprot); } oprot.writeListEnd(); } @@ -139660,9 +139660,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_with if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1842 : struct.success) + for (Partition _iter1858 : struct.success) { - _iter1842.write(oprot); + _iter1858.write(oprot); } } } @@ -139680,14 +139680,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_with_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1843 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1843.size); - @org.apache.thrift.annotation.Nullable Partition _elem1844; - for (int _i1845 = 0; _i1845 < _list1843.size; ++_i1845) + org.apache.thrift.protocol.TList _list1859 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1859.size); + @org.apache.thrift.annotation.Nullable Partition _elem1860; + for (int _i1861 = 0; _i1861 < _list1859.size; ++_i1861) { - _elem1844 = new Partition(); - _elem1844.read(iprot); - struct.success.add(_elem1844); + _elem1860 = new Partition(); + _elem1860.read(iprot); + struct.success.add(_elem1860); } } struct.setSuccessIsSet(true); @@ -140756,14 +140756,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_pspe case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1846 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1846.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1847; - for (int _i1848 = 0; _i1848 < _list1846.size; ++_i1848) + org.apache.thrift.protocol.TList _list1862 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1862.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1863; + for (int _i1864 = 0; _i1864 < _list1862.size; ++_i1864) { - _elem1847 = new PartitionSpec(); - _elem1847.read(iprot); - struct.success.add(_elem1847); + _elem1863 = new PartitionSpec(); + _elem1863.read(iprot); + struct.success.add(_elem1863); } iprot.readListEnd(); } @@ -140807,9 +140807,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_psp oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (PartitionSpec _iter1849 : struct.success) + for (PartitionSpec _iter1865 : struct.success) { - _iter1849.write(oprot); + _iter1865.write(oprot); } oprot.writeListEnd(); } @@ -140856,9 +140856,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_pspe if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (PartitionSpec _iter1850 : struct.success) + for (PartitionSpec _iter1866 : struct.success) { - _iter1850.write(oprot); + _iter1866.write(oprot); } } } @@ -140876,14 +140876,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_pspec java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1851 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1851.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1852; - for (int _i1853 = 0; _i1853 < _list1851.size; ++_i1853) + org.apache.thrift.protocol.TList _list1867 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1867.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1868; + for (int _i1869 = 0; _i1869 < _list1867.size; ++_i1869) { - _elem1852 = new PartitionSpec(); - _elem1852.read(iprot); - struct.success.add(_elem1852); + _elem1868 = new PartitionSpec(); + _elem1868.read(iprot); + struct.success.add(_elem1868); } } struct.setSuccessIsSet(true); @@ -141949,13 +141949,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1854 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1854.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1855; - for (int _i1856 = 0; _i1856 < _list1854.size; ++_i1856) + org.apache.thrift.protocol.TList _list1870 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1870.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1871; + for (int _i1872 = 0; _i1872 < _list1870.size; ++_i1872) { - _elem1855 = iprot.readString(); - struct.success.add(_elem1855); + _elem1871 = iprot.readString(); + struct.success.add(_elem1871); } iprot.readListEnd(); } @@ -141999,9 +141999,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1857 : struct.success) + for (java.lang.String _iter1873 : struct.success) { - oprot.writeString(_iter1857); + oprot.writeString(_iter1873); } oprot.writeListEnd(); } @@ -142048,9 +142048,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1858 : struct.success) + for (java.lang.String _iter1874 : struct.success) { - oprot.writeString(_iter1858); + oprot.writeString(_iter1874); } } } @@ -142068,13 +142068,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1859 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1859.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1860; - for (int _i1861 = 0; _i1861 < _list1859.size; ++_i1861) + org.apache.thrift.protocol.TList _list1875 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1875.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1876; + for (int _i1877 = 0; _i1877 < _list1875.size; ++_i1877) { - _elem1860 = iprot.readString(); - struct.success.add(_elem1860); + _elem1876 = iprot.readString(); + struct.success.add(_elem1876); } } struct.setSuccessIsSet(true); @@ -143615,13 +143615,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_a case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1862 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1862.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1863; - for (int _i1864 = 0; _i1864 < _list1862.size; ++_i1864) + org.apache.thrift.protocol.TList _list1878 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1878.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1879; + for (int _i1880 = 0; _i1880 < _list1878.size; ++_i1880) { - _elem1863 = iprot.readString(); - struct.part_vals.add(_elem1863); + _elem1879 = iprot.readString(); + struct.part_vals.add(_elem1879); } iprot.readListEnd(); } @@ -143665,9 +143665,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1865 : struct.part_vals) + for (java.lang.String _iter1881 : struct.part_vals) { - oprot.writeString(_iter1865); + oprot.writeString(_iter1881); } oprot.writeListEnd(); } @@ -143716,9 +143716,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_a if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1866 : struct.part_vals) + for (java.lang.String _iter1882 : struct.part_vals) { - oprot.writeString(_iter1866); + oprot.writeString(_iter1882); } } } @@ -143741,13 +143741,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1867 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1867.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1868; - for (int _i1869 = 0; _i1869 < _list1867.size; ++_i1869) + org.apache.thrift.protocol.TList _list1883 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1883.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1884; + for (int _i1885 = 0; _i1885 < _list1883.size; ++_i1885) { - _elem1868 = iprot.readString(); - struct.part_vals.add(_elem1868); + _elem1884 = iprot.readString(); + struct.part_vals.add(_elem1884); } } struct.setPart_valsIsSet(true); @@ -144243,14 +144243,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1870 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1870.size); - @org.apache.thrift.annotation.Nullable Partition _elem1871; - for (int _i1872 = 0; _i1872 < _list1870.size; ++_i1872) + org.apache.thrift.protocol.TList _list1886 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1886.size); + @org.apache.thrift.annotation.Nullable Partition _elem1887; + for (int _i1888 = 0; _i1888 < _list1886.size; ++_i1888) { - _elem1871 = new Partition(); - _elem1871.read(iprot); - struct.success.add(_elem1871); + _elem1887 = new Partition(); + _elem1887.read(iprot); + struct.success.add(_elem1887); } iprot.readListEnd(); } @@ -144294,9 +144294,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1873 : struct.success) + for (Partition _iter1889 : struct.success) { - _iter1873.write(oprot); + _iter1889.write(oprot); } oprot.writeListEnd(); } @@ -144343,9 +144343,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1874 : struct.success) + for (Partition _iter1890 : struct.success) { - _iter1874.write(oprot); + _iter1890.write(oprot); } } } @@ -144363,14 +144363,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_re java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1875 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1875.size); - @org.apache.thrift.annotation.Nullable Partition _elem1876; - for (int _i1877 = 0; _i1877 < _list1875.size; ++_i1877) + org.apache.thrift.protocol.TList _list1891 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1891.size); + @org.apache.thrift.annotation.Nullable Partition _elem1892; + for (int _i1893 = 0; _i1893 < _list1891.size; ++_i1893) { - _elem1876 = new Partition(); - _elem1876.read(iprot); - struct.success.add(_elem1876); + _elem1892 = new Partition(); + _elem1892.read(iprot); + struct.success.add(_elem1892); } } struct.setSuccessIsSet(true); @@ -145145,13 +145145,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1878 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1878.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1879; - for (int _i1880 = 0; _i1880 < _list1878.size; ++_i1880) + org.apache.thrift.protocol.TList _list1894 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1894.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1895; + for (int _i1896 = 0; _i1896 < _list1894.size; ++_i1896) { - _elem1879 = iprot.readString(); - struct.part_vals.add(_elem1879); + _elem1895 = iprot.readString(); + struct.part_vals.add(_elem1895); } iprot.readListEnd(); } @@ -145179,13 +145179,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 6: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1881 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list1881.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1882; - for (int _i1883 = 0; _i1883 < _list1881.size; ++_i1883) + org.apache.thrift.protocol.TList _list1897 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list1897.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1898; + for (int _i1899 = 0; _i1899 < _list1897.size; ++_i1899) { - _elem1882 = iprot.readString(); - struct.group_names.add(_elem1882); + _elem1898 = iprot.readString(); + struct.group_names.add(_elem1898); } iprot.readListEnd(); } @@ -145221,9 +145221,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1884 : struct.part_vals) + for (java.lang.String _iter1900 : struct.part_vals) { - oprot.writeString(_iter1884); + oprot.writeString(_iter1900); } oprot.writeListEnd(); } @@ -145241,9 +145241,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter1885 : struct.group_names) + for (java.lang.String _iter1901 : struct.group_names) { - oprot.writeString(_iter1885); + oprot.writeString(_iter1901); } oprot.writeListEnd(); } @@ -145295,9 +145295,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1886 : struct.part_vals) + for (java.lang.String _iter1902 : struct.part_vals) { - oprot.writeString(_iter1886); + oprot.writeString(_iter1902); } } } @@ -145310,9 +145310,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter1887 : struct.group_names) + for (java.lang.String _iter1903 : struct.group_names) { - oprot.writeString(_iter1887); + oprot.writeString(_iter1903); } } } @@ -145332,13 +145332,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1888 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1888.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1889; - for (int _i1890 = 0; _i1890 < _list1888.size; ++_i1890) + org.apache.thrift.protocol.TList _list1904 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1904.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1905; + for (int _i1906 = 0; _i1906 < _list1904.size; ++_i1906) { - _elem1889 = iprot.readString(); - struct.part_vals.add(_elem1889); + _elem1905 = iprot.readString(); + struct.part_vals.add(_elem1905); } } struct.setPart_valsIsSet(true); @@ -145353,13 +145353,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1891 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list1891.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1892; - for (int _i1893 = 0; _i1893 < _list1891.size; ++_i1893) + org.apache.thrift.protocol.TList _list1907 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list1907.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1908; + for (int _i1909 = 0; _i1909 < _list1907.size; ++_i1909) { - _elem1892 = iprot.readString(); - struct.group_names.add(_elem1892); + _elem1908 = iprot.readString(); + struct.group_names.add(_elem1908); } } struct.setGroup_namesIsSet(true); @@ -145851,14 +145851,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1894 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1894.size); - @org.apache.thrift.annotation.Nullable Partition _elem1895; - for (int _i1896 = 0; _i1896 < _list1894.size; ++_i1896) + org.apache.thrift.protocol.TList _list1910 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1910.size); + @org.apache.thrift.annotation.Nullable Partition _elem1911; + for (int _i1912 = 0; _i1912 < _list1910.size; ++_i1912) { - _elem1895 = new Partition(); - _elem1895.read(iprot); - struct.success.add(_elem1895); + _elem1911 = new Partition(); + _elem1911.read(iprot); + struct.success.add(_elem1911); } iprot.readListEnd(); } @@ -145902,9 +145902,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1897 : struct.success) + for (Partition _iter1913 : struct.success) { - _iter1897.write(oprot); + _iter1913.write(oprot); } oprot.writeListEnd(); } @@ -145951,9 +145951,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1898 : struct.success) + for (Partition _iter1914 : struct.success) { - _iter1898.write(oprot); + _iter1914.write(oprot); } } } @@ -145971,14 +145971,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1899 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1899.size); - @org.apache.thrift.annotation.Nullable Partition _elem1900; - for (int _i1901 = 0; _i1901 < _list1899.size; ++_i1901) + org.apache.thrift.protocol.TList _list1915 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1915.size); + @org.apache.thrift.annotation.Nullable Partition _elem1916; + for (int _i1917 = 0; _i1917 < _list1915.size; ++_i1917) { - _elem1900 = new Partition(); - _elem1900.read(iprot); - struct.success.add(_elem1900); + _elem1916 = new Partition(); + _elem1916.read(iprot); + struct.success.add(_elem1916); } } struct.setSuccessIsSet(true); @@ -147519,13 +147519,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1902 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1902.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1903; - for (int _i1904 = 0; _i1904 < _list1902.size; ++_i1904) + org.apache.thrift.protocol.TList _list1918 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1918.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1919; + for (int _i1920 = 0; _i1920 < _list1918.size; ++_i1920) { - _elem1903 = iprot.readString(); - struct.part_vals.add(_elem1903); + _elem1919 = iprot.readString(); + struct.part_vals.add(_elem1919); } iprot.readListEnd(); } @@ -147569,9 +147569,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1905 : struct.part_vals) + for (java.lang.String _iter1921 : struct.part_vals) { - oprot.writeString(_iter1905); + oprot.writeString(_iter1921); } oprot.writeListEnd(); } @@ -147620,9 +147620,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1906 : struct.part_vals) + for (java.lang.String _iter1922 : struct.part_vals) { - oprot.writeString(_iter1906); + oprot.writeString(_iter1922); } } } @@ -147645,13 +147645,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1907 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1907.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1908; - for (int _i1909 = 0; _i1909 < _list1907.size; ++_i1909) + org.apache.thrift.protocol.TList _list1923 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1923.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1924; + for (int _i1925 = 0; _i1925 < _list1923.size; ++_i1925) { - _elem1908 = iprot.readString(); - struct.part_vals.add(_elem1908); + _elem1924 = iprot.readString(); + struct.part_vals.add(_elem1924); } } struct.setPart_valsIsSet(true); @@ -148144,13 +148144,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1910 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1910.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1911; - for (int _i1912 = 0; _i1912 < _list1910.size; ++_i1912) + org.apache.thrift.protocol.TList _list1926 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1926.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1927; + for (int _i1928 = 0; _i1928 < _list1926.size; ++_i1928) { - _elem1911 = iprot.readString(); - struct.success.add(_elem1911); + _elem1927 = iprot.readString(); + struct.success.add(_elem1927); } iprot.readListEnd(); } @@ -148194,9 +148194,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1913 : struct.success) + for (java.lang.String _iter1929 : struct.success) { - oprot.writeString(_iter1913); + oprot.writeString(_iter1929); } oprot.writeListEnd(); } @@ -148243,9 +148243,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1914 : struct.success) + for (java.lang.String _iter1930 : struct.success) { - oprot.writeString(_iter1914); + oprot.writeString(_iter1930); } } } @@ -148263,13 +148263,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1915 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1915.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1916; - for (int _i1917 = 0; _i1917 < _list1915.size; ++_i1917) + org.apache.thrift.protocol.TList _list1931 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1931.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1932; + for (int _i1933 = 0; _i1933 < _list1931.size; ++_i1933) { - _elem1916 = iprot.readString(); - struct.success.add(_elem1916); + _elem1932 = iprot.readString(); + struct.success.add(_elem1932); } } struct.setSuccessIsSet(true); @@ -150081,13 +150081,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1918 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1918.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1919; - for (int _i1920 = 0; _i1920 < _list1918.size; ++_i1920) + org.apache.thrift.protocol.TList _list1934 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1934.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1935; + for (int _i1936 = 0; _i1936 < _list1934.size; ++_i1936) { - _elem1919 = iprot.readString(); - struct.success.add(_elem1919); + _elem1935 = iprot.readString(); + struct.success.add(_elem1935); } iprot.readListEnd(); } @@ -150131,9 +150131,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1921 : struct.success) + for (java.lang.String _iter1937 : struct.success) { - oprot.writeString(_iter1921); + oprot.writeString(_iter1937); } oprot.writeListEnd(); } @@ -150180,9 +150180,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1922 : struct.success) + for (java.lang.String _iter1938 : struct.success) { - oprot.writeString(_iter1922); + oprot.writeString(_iter1938); } } } @@ -150200,13 +150200,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1923 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1923.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1924; - for (int _i1925 = 0; _i1925 < _list1923.size; ++_i1925) + org.apache.thrift.protocol.TList _list1939 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1939.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1940; + for (int _i1941 = 0; _i1941 < _list1939.size; ++_i1941) { - _elem1924 = iprot.readString(); - struct.success.add(_elem1924); + _elem1940 = iprot.readString(); + struct.success.add(_elem1940); } } struct.setSuccessIsSet(true); @@ -151379,14 +151379,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1926 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1926.size); - @org.apache.thrift.annotation.Nullable Partition _elem1927; - for (int _i1928 = 0; _i1928 < _list1926.size; ++_i1928) + org.apache.thrift.protocol.TList _list1942 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1942.size); + @org.apache.thrift.annotation.Nullable Partition _elem1943; + for (int _i1944 = 0; _i1944 < _list1942.size; ++_i1944) { - _elem1927 = new Partition(); - _elem1927.read(iprot); - struct.success.add(_elem1927); + _elem1943 = new Partition(); + _elem1943.read(iprot); + struct.success.add(_elem1943); } iprot.readListEnd(); } @@ -151430,9 +151430,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1929 : struct.success) + for (Partition _iter1945 : struct.success) { - _iter1929.write(oprot); + _iter1945.write(oprot); } oprot.writeListEnd(); } @@ -151479,9 +151479,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1930 : struct.success) + for (Partition _iter1946 : struct.success) { - _iter1930.write(oprot); + _iter1946.write(oprot); } } } @@ -151499,14 +151499,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_fi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1931 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1931.size); - @org.apache.thrift.annotation.Nullable Partition _elem1932; - for (int _i1933 = 0; _i1933 < _list1931.size; ++_i1933) + org.apache.thrift.protocol.TList _list1947 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1947.size); + @org.apache.thrift.annotation.Nullable Partition _elem1948; + for (int _i1949 = 0; _i1949 < _list1947.size; ++_i1949) { - _elem1932 = new Partition(); - _elem1932.read(iprot); - struct.success.add(_elem1932); + _elem1948 = new Partition(); + _elem1948.read(iprot); + struct.success.add(_elem1948); } } struct.setSuccessIsSet(true); @@ -152375,14 +152375,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1934 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1934.size); - @org.apache.thrift.annotation.Nullable Partition _elem1935; - for (int _i1936 = 0; _i1936 < _list1934.size; ++_i1936) + org.apache.thrift.protocol.TList _list1950 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1950.size); + @org.apache.thrift.annotation.Nullable Partition _elem1951; + for (int _i1952 = 0; _i1952 < _list1950.size; ++_i1952) { - _elem1935 = new Partition(); - _elem1935.read(iprot); - struct.success.add(_elem1935); + _elem1951 = new Partition(); + _elem1951.read(iprot); + struct.success.add(_elem1951); } iprot.readListEnd(); } @@ -152426,9 +152426,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1937 : struct.success) + for (Partition _iter1953 : struct.success) { - _iter1937.write(oprot); + _iter1953.write(oprot); } oprot.writeListEnd(); } @@ -152475,9 +152475,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1938 : struct.success) + for (Partition _iter1954 : struct.success) { - _iter1938.write(oprot); + _iter1954.write(oprot); } } } @@ -152495,14 +152495,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_fi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1939 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1939.size); - @org.apache.thrift.annotation.Nullable Partition _elem1940; - for (int _i1941 = 0; _i1941 < _list1939.size; ++_i1941) + org.apache.thrift.protocol.TList _list1955 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1955.size); + @org.apache.thrift.annotation.Nullable Partition _elem1956; + for (int _i1957 = 0; _i1957 < _list1955.size; ++_i1957) { - _elem1940 = new Partition(); - _elem1940.read(iprot); - struct.success.add(_elem1940); + _elem1956 = new Partition(); + _elem1956.read(iprot); + struct.success.add(_elem1956); } } struct.setSuccessIsSet(true); @@ -153675,14 +153675,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_part_specs_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1942 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1942.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1943; - for (int _i1944 = 0; _i1944 < _list1942.size; ++_i1944) + org.apache.thrift.protocol.TList _list1958 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1958.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1959; + for (int _i1960 = 0; _i1960 < _list1958.size; ++_i1960) { - _elem1943 = new PartitionSpec(); - _elem1943.read(iprot); - struct.success.add(_elem1943); + _elem1959 = new PartitionSpec(); + _elem1959.read(iprot); + struct.success.add(_elem1959); } iprot.readListEnd(); } @@ -153726,9 +153726,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_part_specs_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (PartitionSpec _iter1945 : struct.success) + for (PartitionSpec _iter1961 : struct.success) { - _iter1945.write(oprot); + _iter1961.write(oprot); } oprot.writeListEnd(); } @@ -153775,9 +153775,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_part_specs_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (PartitionSpec _iter1946 : struct.success) + for (PartitionSpec _iter1962 : struct.success) { - _iter1946.write(oprot); + _iter1962.write(oprot); } } } @@ -153795,14 +153795,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_part_specs_by_fi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1947 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1947.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1948; - for (int _i1949 = 0; _i1949 < _list1947.size; ++_i1949) + org.apache.thrift.protocol.TList _list1963 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1963.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1964; + for (int _i1965 = 0; _i1965 < _list1963.size; ++_i1965) { - _elem1948 = new PartitionSpec(); - _elem1948.read(iprot); - struct.success.add(_elem1948); + _elem1964 = new PartitionSpec(); + _elem1964.read(iprot); + struct.success.add(_elem1964); } } struct.setSuccessIsSet(true); @@ -157350,13 +157350,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_n case 3: // NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1950 = iprot.readListBegin(); - struct.names = new java.util.ArrayList(_list1950.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1951; - for (int _i1952 = 0; _i1952 < _list1950.size; ++_i1952) + org.apache.thrift.protocol.TList _list1966 = iprot.readListBegin(); + struct.names = new java.util.ArrayList(_list1966.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1967; + for (int _i1968 = 0; _i1968 < _list1966.size; ++_i1968) { - _elem1951 = iprot.readString(); - struct.names.add(_elem1951); + _elem1967 = iprot.readString(); + struct.names.add(_elem1967); } iprot.readListEnd(); } @@ -157392,9 +157392,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size())); - for (java.lang.String _iter1953 : struct.names) + for (java.lang.String _iter1969 : struct.names) { - oprot.writeString(_iter1953); + oprot.writeString(_iter1969); } oprot.writeListEnd(); } @@ -157437,9 +157437,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_n if (struct.isSetNames()) { { oprot.writeI32(struct.names.size()); - for (java.lang.String _iter1954 : struct.names) + for (java.lang.String _iter1970 : struct.names) { - oprot.writeString(_iter1954); + oprot.writeString(_iter1970); } } } @@ -157459,13 +157459,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_na } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1955 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.names = new java.util.ArrayList(_list1955.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1956; - for (int _i1957 = 0; _i1957 < _list1955.size; ++_i1957) + org.apache.thrift.protocol.TList _list1971 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.names = new java.util.ArrayList(_list1971.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1972; + for (int _i1973 = 0; _i1973 < _list1971.size; ++_i1973) { - _elem1956 = iprot.readString(); - struct.names.add(_elem1956); + _elem1972 = iprot.readString(); + struct.names.add(_elem1972); } } struct.setNamesIsSet(true); @@ -158038,14 +158038,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_n case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1958 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1958.size); - @org.apache.thrift.annotation.Nullable Partition _elem1959; - for (int _i1960 = 0; _i1960 < _list1958.size; ++_i1960) + org.apache.thrift.protocol.TList _list1974 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1974.size); + @org.apache.thrift.annotation.Nullable Partition _elem1975; + for (int _i1976 = 0; _i1976 < _list1974.size; ++_i1976) { - _elem1959 = new Partition(); - _elem1959.read(iprot); - struct.success.add(_elem1959); + _elem1975 = new Partition(); + _elem1975.read(iprot); + struct.success.add(_elem1975); } iprot.readListEnd(); } @@ -158098,9 +158098,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1961 : struct.success) + for (Partition _iter1977 : struct.success) { - _iter1961.write(oprot); + _iter1977.write(oprot); } oprot.writeListEnd(); } @@ -158155,9 +158155,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_n if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1962 : struct.success) + for (Partition _iter1978 : struct.success) { - _iter1962.write(oprot); + _iter1978.write(oprot); } } } @@ -158178,14 +158178,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_na java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1963 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1963.size); - @org.apache.thrift.annotation.Nullable Partition _elem1964; - for (int _i1965 = 0; _i1965 < _list1963.size; ++_i1965) + org.apache.thrift.protocol.TList _list1979 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1979.size); + @org.apache.thrift.annotation.Nullable Partition _elem1980; + for (int _i1981 = 0; _i1981 < _list1979.size; ++_i1981) { - _elem1964 = new Partition(); - _elem1964.read(iprot); - struct.success.add(_elem1964); + _elem1980 = new Partition(); + _elem1980.read(iprot); + struct.success.add(_elem1980); } } struct.setSuccessIsSet(true); @@ -162689,14 +162689,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, alter_partitions_ar case 3: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1966 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1966.size); - @org.apache.thrift.annotation.Nullable Partition _elem1967; - for (int _i1968 = 0; _i1968 < _list1966.size; ++_i1968) + org.apache.thrift.protocol.TList _list1982 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1982.size); + @org.apache.thrift.annotation.Nullable Partition _elem1983; + for (int _i1984 = 0; _i1984 < _list1982.size; ++_i1984) { - _elem1967 = new Partition(); - _elem1967.read(iprot); - struct.new_parts.add(_elem1967); + _elem1983 = new Partition(); + _elem1983.read(iprot); + struct.new_parts.add(_elem1983); } iprot.readListEnd(); } @@ -162732,9 +162732,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, alter_partitions_a oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter1969 : struct.new_parts) + for (Partition _iter1985 : struct.new_parts) { - _iter1969.write(oprot); + _iter1985.write(oprot); } oprot.writeListEnd(); } @@ -162777,9 +162777,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, alter_partitions_ar if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter1970 : struct.new_parts) + for (Partition _iter1986 : struct.new_parts) { - _iter1970.write(oprot); + _iter1986.write(oprot); } } } @@ -162799,14 +162799,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, alter_partitions_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1971 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1971.size); - @org.apache.thrift.annotation.Nullable Partition _elem1972; - for (int _i1973 = 0; _i1973 < _list1971.size; ++_i1973) + org.apache.thrift.protocol.TList _list1987 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1987.size); + @org.apache.thrift.annotation.Nullable Partition _elem1988; + for (int _i1989 = 0; _i1989 < _list1987.size; ++_i1989) { - _elem1972 = new Partition(); - _elem1972.read(iprot); - struct.new_parts.add(_elem1972); + _elem1988 = new Partition(); + _elem1988.read(iprot); + struct.new_parts.add(_elem1988); } } struct.setNew_partsIsSet(true); @@ -163868,14 +163868,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, alter_partitions_wi case 3: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1974 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1974.size); - @org.apache.thrift.annotation.Nullable Partition _elem1975; - for (int _i1976 = 0; _i1976 < _list1974.size; ++_i1976) + org.apache.thrift.protocol.TList _list1990 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1990.size); + @org.apache.thrift.annotation.Nullable Partition _elem1991; + for (int _i1992 = 0; _i1992 < _list1990.size; ++_i1992) { - _elem1975 = new Partition(); - _elem1975.read(iprot); - struct.new_parts.add(_elem1975); + _elem1991 = new Partition(); + _elem1991.read(iprot); + struct.new_parts.add(_elem1991); } iprot.readListEnd(); } @@ -163920,9 +163920,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, alter_partitions_w oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter1977 : struct.new_parts) + for (Partition _iter1993 : struct.new_parts) { - _iter1977.write(oprot); + _iter1993.write(oprot); } oprot.writeListEnd(); } @@ -163973,9 +163973,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, alter_partitions_wi if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter1978 : struct.new_parts) + for (Partition _iter1994 : struct.new_parts) { - _iter1978.write(oprot); + _iter1994.write(oprot); } } } @@ -163998,14 +163998,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, alter_partitions_wit } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1979 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1979.size); - @org.apache.thrift.annotation.Nullable Partition _elem1980; - for (int _i1981 = 0; _i1981 < _list1979.size; ++_i1981) + org.apache.thrift.protocol.TList _list1995 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1995.size); + @org.apache.thrift.annotation.Nullable Partition _elem1996; + for (int _i1997 = 0; _i1997 < _list1995.size; ++_i1997) { - _elem1980 = new Partition(); - _elem1980.read(iprot); - struct.new_parts.add(_elem1980); + _elem1996 = new Partition(); + _elem1996.read(iprot); + struct.new_parts.add(_elem1996); } } struct.setNew_partsIsSet(true); @@ -167169,13 +167169,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, rename_partition_ar case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1982 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1982.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1983; - for (int _i1984 = 0; _i1984 < _list1982.size; ++_i1984) + org.apache.thrift.protocol.TList _list1998 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1998.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1999; + for (int _i2000 = 0; _i2000 < _list1998.size; ++_i2000) { - _elem1983 = iprot.readString(); - struct.part_vals.add(_elem1983); + _elem1999 = iprot.readString(); + struct.part_vals.add(_elem1999); } iprot.readListEnd(); } @@ -167220,9 +167220,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, rename_partition_a oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1985 : struct.part_vals) + for (java.lang.String _iter2001 : struct.part_vals) { - oprot.writeString(_iter1985); + oprot.writeString(_iter2001); } oprot.writeListEnd(); } @@ -167273,9 +167273,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, rename_partition_ar if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1986 : struct.part_vals) + for (java.lang.String _iter2002 : struct.part_vals) { - oprot.writeString(_iter1986); + oprot.writeString(_iter2002); } } } @@ -167298,13 +167298,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, rename_partition_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1987 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1987.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1988; - for (int _i1989 = 0; _i1989 < _list1987.size; ++_i1989) + org.apache.thrift.protocol.TList _list2003 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list2003.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2004; + for (int _i2005 = 0; _i2005 < _list2003.size; ++_i2005) { - _elem1988 = iprot.readString(); - struct.part_vals.add(_elem1988); + _elem2004 = iprot.readString(); + struct.part_vals.add(_elem2004); } } struct.setPart_valsIsSet(true); @@ -169130,13 +169130,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_has_ case 1: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1990 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1990.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1991; - for (int _i1992 = 0; _i1992 < _list1990.size; ++_i1992) + org.apache.thrift.protocol.TList _list2006 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list2006.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2007; + for (int _i2008 = 0; _i2008 < _list2006.size; ++_i2008) { - _elem1991 = iprot.readString(); - struct.part_vals.add(_elem1991); + _elem2007 = iprot.readString(); + struct.part_vals.add(_elem2007); } iprot.readListEnd(); } @@ -169170,9 +169170,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_has oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1993 : struct.part_vals) + for (java.lang.String _iter2009 : struct.part_vals) { - oprot.writeString(_iter1993); + oprot.writeString(_iter2009); } oprot.writeListEnd(); } @@ -169209,9 +169209,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_has_ if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1994 : struct.part_vals) + for (java.lang.String _iter2010 : struct.part_vals) { - oprot.writeString(_iter1994); + oprot.writeString(_iter2010); } } } @@ -169226,13 +169226,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_has_v java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1995 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1995.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1996; - for (int _i1997 = 0; _i1997 < _list1995.size; ++_i1997) + org.apache.thrift.protocol.TList _list2011 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list2011.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2012; + for (int _i2013 = 0; _i2013 < _list2011.size; ++_i2013) { - _elem1996 = iprot.readString(); - struct.part_vals.add(_elem1996); + _elem2012 = iprot.readString(); + struct.part_vals.add(_elem2012); } } struct.setPart_valsIsSet(true); @@ -171405,13 +171405,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_to_v case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1998 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1998.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1999; - for (int _i2000 = 0; _i2000 < _list1998.size; ++_i2000) + org.apache.thrift.protocol.TList _list2014 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2014.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2015; + for (int _i2016 = 0; _i2016 < _list2014.size; ++_i2016) { - _elem1999 = iprot.readString(); - struct.success.add(_elem1999); + _elem2015 = iprot.readString(); + struct.success.add(_elem2015); } iprot.readListEnd(); } @@ -171446,9 +171446,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_to_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2001 : struct.success) + for (java.lang.String _iter2017 : struct.success) { - oprot.writeString(_iter2001); + oprot.writeString(_iter2017); } oprot.writeListEnd(); } @@ -171487,9 +171487,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_to_v if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2002 : struct.success) + for (java.lang.String _iter2018 : struct.success) { - oprot.writeString(_iter2002); + oprot.writeString(_iter2018); } } } @@ -171504,13 +171504,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_to_va java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2003 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2003.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2004; - for (int _i2005 = 0; _i2005 < _list2003.size; ++_i2005) + org.apache.thrift.protocol.TList _list2019 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2019.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2020; + for (int _i2021 = 0; _i2021 < _list2019.size; ++_i2021) { - _elem2004 = iprot.readString(); - struct.success.add(_elem2004); + _elem2020 = iprot.readString(); + struct.success.add(_elem2020); } } struct.setSuccessIsSet(true); @@ -172281,15 +172281,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_to_s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2006 = iprot.readMapBegin(); - struct.success = new java.util.HashMap(2*_map2006.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2007; - @org.apache.thrift.annotation.Nullable java.lang.String _val2008; - for (int _i2009 = 0; _i2009 < _map2006.size; ++_i2009) + org.apache.thrift.protocol.TMap _map2022 = iprot.readMapBegin(); + struct.success = new java.util.HashMap(2*_map2022.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2023; + @org.apache.thrift.annotation.Nullable java.lang.String _val2024; + for (int _i2025 = 0; _i2025 < _map2022.size; ++_i2025) { - _key2007 = iprot.readString(); - _val2008 = iprot.readString(); - struct.success.put(_key2007, _val2008); + _key2023 = iprot.readString(); + _val2024 = iprot.readString(); + struct.success.put(_key2023, _val2024); } iprot.readMapEnd(); } @@ -172324,10 +172324,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_to_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.util.Map.Entry _iter2010 : struct.success.entrySet()) + for (java.util.Map.Entry _iter2026 : struct.success.entrySet()) { - oprot.writeString(_iter2010.getKey()); - oprot.writeString(_iter2010.getValue()); + oprot.writeString(_iter2026.getKey()); + oprot.writeString(_iter2026.getValue()); } oprot.writeMapEnd(); } @@ -172366,10 +172366,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_to_s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.util.Map.Entry _iter2011 : struct.success.entrySet()) + for (java.util.Map.Entry _iter2027 : struct.success.entrySet()) { - oprot.writeString(_iter2011.getKey()); - oprot.writeString(_iter2011.getValue()); + oprot.writeString(_iter2027.getKey()); + oprot.writeString(_iter2027.getValue()); } } } @@ -172384,15 +172384,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_to_sp java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map2012 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.HashMap(2*_map2012.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2013; - @org.apache.thrift.annotation.Nullable java.lang.String _val2014; - for (int _i2015 = 0; _i2015 < _map2012.size; ++_i2015) + org.apache.thrift.protocol.TMap _map2028 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.HashMap(2*_map2028.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2029; + @org.apache.thrift.annotation.Nullable java.lang.String _val2030; + for (int _i2031 = 0; _i2031 < _map2028.size; ++_i2031) { - _key2013 = iprot.readString(); - _val2014 = iprot.readString(); - struct.success.put(_key2013, _val2014); + _key2029 = iprot.readString(); + _val2030 = iprot.readString(); + struct.success.put(_key2029, _val2030); } } struct.setSuccessIsSet(true); @@ -172991,15 +172991,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, markPartitionForEve case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2016 = iprot.readMapBegin(); - struct.part_vals = new java.util.HashMap(2*_map2016.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2017; - @org.apache.thrift.annotation.Nullable java.lang.String _val2018; - for (int _i2019 = 0; _i2019 < _map2016.size; ++_i2019) + org.apache.thrift.protocol.TMap _map2032 = iprot.readMapBegin(); + struct.part_vals = new java.util.HashMap(2*_map2032.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2033; + @org.apache.thrift.annotation.Nullable java.lang.String _val2034; + for (int _i2035 = 0; _i2035 < _map2032.size; ++_i2035) { - _key2017 = iprot.readString(); - _val2018 = iprot.readString(); - struct.part_vals.put(_key2017, _val2018); + _key2033 = iprot.readString(); + _val2034 = iprot.readString(); + struct.part_vals.put(_key2033, _val2034); } iprot.readMapEnd(); } @@ -173043,10 +173043,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, markPartitionForEv oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.util.Map.Entry _iter2020 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2036 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2020.getKey()); - oprot.writeString(_iter2020.getValue()); + oprot.writeString(_iter2036.getKey()); + oprot.writeString(_iter2036.getValue()); } oprot.writeMapEnd(); } @@ -173097,10 +173097,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, markPartitionForEve if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.util.Map.Entry _iter2021 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2037 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2021.getKey()); - oprot.writeString(_iter2021.getValue()); + oprot.writeString(_iter2037.getKey()); + oprot.writeString(_iter2037.getValue()); } } } @@ -173123,15 +173123,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, markPartitionForEven } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map2022 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.HashMap(2*_map2022.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2023; - @org.apache.thrift.annotation.Nullable java.lang.String _val2024; - for (int _i2025 = 0; _i2025 < _map2022.size; ++_i2025) + org.apache.thrift.protocol.TMap _map2038 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.HashMap(2*_map2038.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2039; + @org.apache.thrift.annotation.Nullable java.lang.String _val2040; + for (int _i2041 = 0; _i2041 < _map2038.size; ++_i2041) { - _key2023 = iprot.readString(); - _val2024 = iprot.readString(); - struct.part_vals.put(_key2023, _val2024); + _key2039 = iprot.readString(); + _val2040 = iprot.readString(); + struct.part_vals.put(_key2039, _val2040); } } struct.setPart_valsIsSet(true); @@ -174623,15 +174623,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isPartitionMarkedFo case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2026 = iprot.readMapBegin(); - struct.part_vals = new java.util.HashMap(2*_map2026.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2027; - @org.apache.thrift.annotation.Nullable java.lang.String _val2028; - for (int _i2029 = 0; _i2029 < _map2026.size; ++_i2029) + org.apache.thrift.protocol.TMap _map2042 = iprot.readMapBegin(); + struct.part_vals = new java.util.HashMap(2*_map2042.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2043; + @org.apache.thrift.annotation.Nullable java.lang.String _val2044; + for (int _i2045 = 0; _i2045 < _map2042.size; ++_i2045) { - _key2027 = iprot.readString(); - _val2028 = iprot.readString(); - struct.part_vals.put(_key2027, _val2028); + _key2043 = iprot.readString(); + _val2044 = iprot.readString(); + struct.part_vals.put(_key2043, _val2044); } iprot.readMapEnd(); } @@ -174675,10 +174675,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isPartitionMarkedF oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.util.Map.Entry _iter2030 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2046 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2030.getKey()); - oprot.writeString(_iter2030.getValue()); + oprot.writeString(_iter2046.getKey()); + oprot.writeString(_iter2046.getValue()); } oprot.writeMapEnd(); } @@ -174729,10 +174729,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, isPartitionMarkedFo if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.util.Map.Entry _iter2031 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2047 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2031.getKey()); - oprot.writeString(_iter2031.getValue()); + oprot.writeString(_iter2047.getKey()); + oprot.writeString(_iter2047.getValue()); } } } @@ -174755,15 +174755,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, isPartitionMarkedFor } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map2032 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.HashMap(2*_map2032.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2033; - @org.apache.thrift.annotation.Nullable java.lang.String _val2034; - for (int _i2035 = 0; _i2035 < _map2032.size; ++_i2035) + org.apache.thrift.protocol.TMap _map2048 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.HashMap(2*_map2048.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2049; + @org.apache.thrift.annotation.Nullable java.lang.String _val2050; + for (int _i2051 = 0; _i2051 < _map2048.size; ++_i2051) { - _key2033 = iprot.readString(); - _val2034 = iprot.readString(); - struct.part_vals.put(_key2033, _val2034); + _key2049 = iprot.readString(); + _val2050 = iprot.readString(); + struct.part_vals.put(_key2049, _val2050); } } struct.setPart_valsIsSet(true); @@ -201467,13 +201467,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_functions_resul case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2036 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2036.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2037; - for (int _i2038 = 0; _i2038 < _list2036.size; ++_i2038) + org.apache.thrift.protocol.TList _list2052 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2052.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2053; + for (int _i2054 = 0; _i2054 < _list2052.size; ++_i2054) { - _elem2037 = iprot.readString(); - struct.success.add(_elem2037); + _elem2053 = iprot.readString(); + struct.success.add(_elem2053); } iprot.readListEnd(); } @@ -201508,9 +201508,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_functions_resu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2039 : struct.success) + for (java.lang.String _iter2055 : struct.success) { - oprot.writeString(_iter2039); + oprot.writeString(_iter2055); } oprot.writeListEnd(); } @@ -201549,9 +201549,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_functions_resul if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2040 : struct.success) + for (java.lang.String _iter2056 : struct.success) { - oprot.writeString(_iter2040); + oprot.writeString(_iter2056); } } } @@ -201566,13 +201566,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_functions_result java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2041 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2041.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2042; - for (int _i2043 = 0; _i2043 < _list2041.size; ++_i2043) + org.apache.thrift.protocol.TList _list2057 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2057.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2058; + for (int _i2059 = 0; _i2059 < _list2057.size; ++_i2059) { - _elem2042 = iprot.readString(); - struct.success.add(_elem2042); + _elem2058 = iprot.readString(); + struct.success.add(_elem2058); } } struct.setSuccessIsSet(true); @@ -205662,13 +205662,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_role_names_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2044 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2044.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2045; - for (int _i2046 = 0; _i2046 < _list2044.size; ++_i2046) + org.apache.thrift.protocol.TList _list2060 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2060.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2061; + for (int _i2062 = 0; _i2062 < _list2060.size; ++_i2062) { - _elem2045 = iprot.readString(); - struct.success.add(_elem2045); + _elem2061 = iprot.readString(); + struct.success.add(_elem2061); } iprot.readListEnd(); } @@ -205703,9 +205703,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_role_names_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2047 : struct.success) + for (java.lang.String _iter2063 : struct.success) { - oprot.writeString(_iter2047); + oprot.writeString(_iter2063); } oprot.writeListEnd(); } @@ -205744,9 +205744,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_role_names_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2048 : struct.success) + for (java.lang.String _iter2064 : struct.success) { - oprot.writeString(_iter2048); + oprot.writeString(_iter2064); } } } @@ -205761,13 +205761,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_role_names_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2049 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2049.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2050; - for (int _i2051 = 0; _i2051 < _list2049.size; ++_i2051) + org.apache.thrift.protocol.TList _list2065 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2065.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2066; + for (int _i2067 = 0; _i2067 < _list2065.size; ++_i2067) { - _elem2050 = iprot.readString(); - struct.success.add(_elem2050); + _elem2066 = iprot.readString(); + struct.success.add(_elem2066); } } struct.setSuccessIsSet(true); @@ -209074,14 +209074,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_roles_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2052 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2052.size); - @org.apache.thrift.annotation.Nullable Role _elem2053; - for (int _i2054 = 0; _i2054 < _list2052.size; ++_i2054) + org.apache.thrift.protocol.TList _list2068 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2068.size); + @org.apache.thrift.annotation.Nullable Role _elem2069; + for (int _i2070 = 0; _i2070 < _list2068.size; ++_i2070) { - _elem2053 = new Role(); - _elem2053.read(iprot); - struct.success.add(_elem2053); + _elem2069 = new Role(); + _elem2069.read(iprot); + struct.success.add(_elem2069); } iprot.readListEnd(); } @@ -209116,9 +209116,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_roles_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Role _iter2055 : struct.success) + for (Role _iter2071 : struct.success) { - _iter2055.write(oprot); + _iter2071.write(oprot); } oprot.writeListEnd(); } @@ -209157,9 +209157,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_roles_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Role _iter2056 : struct.success) + for (Role _iter2072 : struct.success) { - _iter2056.write(oprot); + _iter2072.write(oprot); } } } @@ -209174,14 +209174,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_roles_result st java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2057 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2057.size); - @org.apache.thrift.annotation.Nullable Role _elem2058; - for (int _i2059 = 0; _i2059 < _list2057.size; ++_i2059) + org.apache.thrift.protocol.TList _list2073 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2073.size); + @org.apache.thrift.annotation.Nullable Role _elem2074; + for (int _i2075 = 0; _i2075 < _list2073.size; ++_i2075) { - _elem2058 = new Role(); - _elem2058.read(iprot); - struct.success.add(_elem2058); + _elem2074 = new Role(); + _elem2074.read(iprot); + struct.success.add(_elem2074); } } struct.setSuccessIsSet(true); @@ -212215,13 +212215,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_privilege_set_a case 3: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2060 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list2060.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2061; - for (int _i2062 = 0; _i2062 < _list2060.size; ++_i2062) + org.apache.thrift.protocol.TList _list2076 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list2076.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2077; + for (int _i2078 = 0; _i2078 < _list2076.size; ++_i2078) { - _elem2061 = iprot.readString(); - struct.group_names.add(_elem2061); + _elem2077 = iprot.readString(); + struct.group_names.add(_elem2077); } iprot.readListEnd(); } @@ -212257,9 +212257,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_privilege_set_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter2063 : struct.group_names) + for (java.lang.String _iter2079 : struct.group_names) { - oprot.writeString(_iter2063); + oprot.writeString(_iter2079); } oprot.writeListEnd(); } @@ -212302,9 +212302,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_a if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter2064 : struct.group_names) + for (java.lang.String _iter2080 : struct.group_names) { - oprot.writeString(_iter2064); + oprot.writeString(_iter2080); } } } @@ -212325,13 +212325,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list2065 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list2065.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2066; - for (int _i2067 = 0; _i2067 < _list2065.size; ++_i2067) + org.apache.thrift.protocol.TList _list2081 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list2081.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2082; + for (int _i2083 = 0; _i2083 < _list2081.size; ++_i2083) { - _elem2066 = iprot.readString(); - struct.group_names.add(_elem2066); + _elem2082 = iprot.readString(); + struct.group_names.add(_elem2082); } } struct.setGroup_namesIsSet(true); @@ -213802,14 +213802,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_privileges_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2068 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2068.size); - @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2069; - for (int _i2070 = 0; _i2070 < _list2068.size; ++_i2070) + org.apache.thrift.protocol.TList _list2084 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2084.size); + @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2085; + for (int _i2086 = 0; _i2086 < _list2084.size; ++_i2086) { - _elem2069 = new HiveObjectPrivilege(); - _elem2069.read(iprot); - struct.success.add(_elem2069); + _elem2085 = new HiveObjectPrivilege(); + _elem2085.read(iprot); + struct.success.add(_elem2085); } iprot.readListEnd(); } @@ -213844,9 +213844,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_privileges_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (HiveObjectPrivilege _iter2071 : struct.success) + for (HiveObjectPrivilege _iter2087 : struct.success) { - _iter2071.write(oprot); + _iter2087.write(oprot); } oprot.writeListEnd(); } @@ -213885,9 +213885,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_privileges_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (HiveObjectPrivilege _iter2072 : struct.success) + for (HiveObjectPrivilege _iter2088 : struct.success) { - _iter2072.write(oprot); + _iter2088.write(oprot); } } } @@ -213902,14 +213902,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_privileges_resu java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2073 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2073.size); - @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2074; - for (int _i2075 = 0; _i2075 < _list2073.size; ++_i2075) + org.apache.thrift.protocol.TList _list2089 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2089.size); + @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2090; + for (int _i2091 = 0; _i2091 < _list2089.size; ++_i2091) { - _elem2074 = new HiveObjectPrivilege(); - _elem2074.read(iprot); - struct.success.add(_elem2074); + _elem2090 = new HiveObjectPrivilege(); + _elem2090.read(iprot); + struct.success.add(_elem2090); } } struct.setSuccessIsSet(true); @@ -217887,13 +217887,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_args struct case 2: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2076 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list2076.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2077; - for (int _i2078 = 0; _i2078 < _list2076.size; ++_i2078) + org.apache.thrift.protocol.TList _list2092 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list2092.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2093; + for (int _i2094 = 0; _i2094 < _list2092.size; ++_i2094) { - _elem2077 = iprot.readString(); - struct.group_names.add(_elem2077); + _elem2093 = iprot.readString(); + struct.group_names.add(_elem2093); } iprot.readListEnd(); } @@ -217924,9 +217924,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_args struc oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter2079 : struct.group_names) + for (java.lang.String _iter2095 : struct.group_names) { - oprot.writeString(_iter2079); + oprot.writeString(_iter2095); } oprot.writeListEnd(); } @@ -217963,9 +217963,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter2080 : struct.group_names) + for (java.lang.String _iter2096 : struct.group_names) { - oprot.writeString(_iter2080); + oprot.writeString(_iter2096); } } } @@ -217981,13 +217981,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct) } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list2081 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list2081.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2082; - for (int _i2083 = 0; _i2083 < _list2081.size; ++_i2083) + org.apache.thrift.protocol.TList _list2097 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list2097.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2098; + for (int _i2099 = 0; _i2099 < _list2097.size; ++_i2099) { - _elem2082 = iprot.readString(); - struct.group_names.add(_elem2082); + _elem2098 = iprot.readString(); + struct.group_names.add(_elem2098); } } struct.setGroup_namesIsSet(true); @@ -218395,13 +218395,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_result stru case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2084 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2084.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2085; - for (int _i2086 = 0; _i2086 < _list2084.size; ++_i2086) + org.apache.thrift.protocol.TList _list2100 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2100.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2101; + for (int _i2102 = 0; _i2102 < _list2100.size; ++_i2102) { - _elem2085 = iprot.readString(); - struct.success.add(_elem2085); + _elem2101 = iprot.readString(); + struct.success.add(_elem2101); } iprot.readListEnd(); } @@ -218436,9 +218436,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_result str oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2087 : struct.success) + for (java.lang.String _iter2103 : struct.success) { - oprot.writeString(_iter2087); + oprot.writeString(_iter2103); } oprot.writeListEnd(); } @@ -218477,9 +218477,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_result stru if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2088 : struct.success) + for (java.lang.String _iter2104 : struct.success) { - oprot.writeString(_iter2088); + oprot.writeString(_iter2104); } } } @@ -218494,13 +218494,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_result struc java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2089 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2089.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2090; - for (int _i2091 = 0; _i2091 < _list2089.size; ++_i2091) + org.apache.thrift.protocol.TList _list2105 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2105.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2106; + for (int _i2107 = 0; _i2107 < _list2105.size; ++_i2107) { - _elem2090 = iprot.readString(); - struct.success.add(_elem2090); + _elem2106 = iprot.readString(); + struct.success.add(_elem2106); } } struct.setSuccessIsSet(true); @@ -223839,13 +223839,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_token_ident case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2092 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2092.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2093; - for (int _i2094 = 0; _i2094 < _list2092.size; ++_i2094) + org.apache.thrift.protocol.TList _list2108 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2108.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2109; + for (int _i2110 = 0; _i2110 < _list2108.size; ++_i2110) { - _elem2093 = iprot.readString(); - struct.success.add(_elem2093); + _elem2109 = iprot.readString(); + struct.success.add(_elem2109); } iprot.readListEnd(); } @@ -223871,9 +223871,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_token_iden oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2095 : struct.success) + for (java.lang.String _iter2111 : struct.success) { - oprot.writeString(_iter2095); + oprot.writeString(_iter2111); } oprot.writeListEnd(); } @@ -223904,9 +223904,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_token_ident if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2096 : struct.success) + for (java.lang.String _iter2112 : struct.success) { - oprot.writeString(_iter2096); + oprot.writeString(_iter2112); } } } @@ -223918,13 +223918,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_token_identi java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2097 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2097.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2098; - for (int _i2099 = 0; _i2099 < _list2097.size; ++_i2099) + org.apache.thrift.protocol.TList _list2113 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2113.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2114; + for (int _i2115 = 0; _i2115 < _list2113.size; ++_i2115) { - _elem2098 = iprot.readString(); - struct.success.add(_elem2098); + _elem2114 = iprot.readString(); + struct.success.add(_elem2114); } } struct.setSuccessIsSet(true); @@ -226975,13 +226975,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_master_keys_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2100 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2100.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2101; - for (int _i2102 = 0; _i2102 < _list2100.size; ++_i2102) + org.apache.thrift.protocol.TList _list2116 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2116.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2117; + for (int _i2118 = 0; _i2118 < _list2116.size; ++_i2118) { - _elem2101 = iprot.readString(); - struct.success.add(_elem2101); + _elem2117 = iprot.readString(); + struct.success.add(_elem2117); } iprot.readListEnd(); } @@ -227007,9 +227007,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_master_keys_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2103 : struct.success) + for (java.lang.String _iter2119 : struct.success) { - oprot.writeString(_iter2103); + oprot.writeString(_iter2119); } oprot.writeListEnd(); } @@ -227040,9 +227040,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_master_keys_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2104 : struct.success) + for (java.lang.String _iter2120 : struct.success) { - oprot.writeString(_iter2104); + oprot.writeString(_iter2120); } } } @@ -227054,13 +227054,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_master_keys_resu java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2105 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2105.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2106; - for (int _i2107 = 0; _i2107 < _list2105.size; ++_i2107) + org.apache.thrift.protocol.TList _list2121 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2121.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2122; + for (int _i2123 = 0; _i2123 < _list2121.size; ++_i2123) { - _elem2106 = iprot.readString(); - struct.success.add(_elem2106); + _elem2122 = iprot.readString(); + struct.success.add(_elem2122); } } struct.setSuccessIsSet(true); @@ -234121,15 +234121,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_write_ids_to_mi case 2: // WRITE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2108 = iprot.readMapBegin(); - struct.writeIds = new java.util.HashMap(2*_map2108.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2109; - long _val2110; - for (int _i2111 = 0; _i2111 < _map2108.size; ++_i2111) + org.apache.thrift.protocol.TMap _map2124 = iprot.readMapBegin(); + struct.writeIds = new java.util.HashMap(2*_map2124.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2125; + long _val2126; + for (int _i2127 = 0; _i2127 < _map2124.size; ++_i2127) { - _key2109 = iprot.readString(); - _val2110 = iprot.readI64(); - struct.writeIds.put(_key2109, _val2110); + _key2125 = iprot.readString(); + _val2126 = iprot.readI64(); + struct.writeIds.put(_key2125, _val2126); } iprot.readMapEnd(); } @@ -234158,10 +234158,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_write_ids_to_m oprot.writeFieldBegin(WRITE_IDS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.writeIds.size())); - for (java.util.Map.Entry _iter2112 : struct.writeIds.entrySet()) + for (java.util.Map.Entry _iter2128 : struct.writeIds.entrySet()) { - oprot.writeString(_iter2112.getKey()); - oprot.writeI64(_iter2112.getValue()); + oprot.writeString(_iter2128.getKey()); + oprot.writeI64(_iter2128.getValue()); } oprot.writeMapEnd(); } @@ -234198,10 +234198,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_mi if (struct.isSetWriteIds()) { { oprot.writeI32(struct.writeIds.size()); - for (java.util.Map.Entry _iter2113 : struct.writeIds.entrySet()) + for (java.util.Map.Entry _iter2129 : struct.writeIds.entrySet()) { - oprot.writeString(_iter2113.getKey()); - oprot.writeI64(_iter2113.getValue()); + oprot.writeString(_iter2129.getKey()); + oprot.writeI64(_iter2129.getValue()); } } } @@ -234217,15 +234217,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_min } if (incoming.get(1)) { { - org.apache.thrift.protocol.TMap _map2114 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64); - struct.writeIds = new java.util.HashMap(2*_map2114.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2115; - long _val2116; - for (int _i2117 = 0; _i2117 < _map2114.size; ++_i2117) + org.apache.thrift.protocol.TMap _map2130 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64); + struct.writeIds = new java.util.HashMap(2*_map2130.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2131; + long _val2132; + for (int _i2133 = 0; _i2133 < _map2130.size; ++_i2133) { - _key2115 = iprot.readString(); - _val2116 = iprot.readI64(); - struct.writeIds.put(_key2115, _val2116); + _key2131 = iprot.readString(); + _val2132 = iprot.readI64(); + struct.writeIds.put(_key2131, _val2132); } } struct.setWriteIdsIsSet(true); @@ -250222,13 +250222,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, find_columns_with_s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2118 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2118.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2119; - for (int _i2120 = 0; _i2120 < _list2118.size; ++_i2120) + org.apache.thrift.protocol.TList _list2134 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2134.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2135; + for (int _i2136 = 0; _i2136 < _list2134.size; ++_i2136) { - _elem2119 = iprot.readString(); - struct.success.add(_elem2119); + _elem2135 = iprot.readString(); + struct.success.add(_elem2135); } iprot.readListEnd(); } @@ -250254,9 +250254,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, find_columns_with_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2121 : struct.success) + for (java.lang.String _iter2137 : struct.success) { - oprot.writeString(_iter2121); + oprot.writeString(_iter2137); } oprot.writeListEnd(); } @@ -250287,9 +250287,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, find_columns_with_s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2122 : struct.success) + for (java.lang.String _iter2138 : struct.success) { - oprot.writeString(_iter2122); + oprot.writeString(_iter2138); } } } @@ -250301,13 +250301,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, find_columns_with_st java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2123 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2123.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2124; - for (int _i2125 = 0; _i2125 < _list2123.size; ++_i2125) + org.apache.thrift.protocol.TList _list2139 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2139.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2140; + for (int _i2141 = 0; _i2141 < _list2139.size; ++_i2141) { - _elem2124 = iprot.readString(); - struct.success.add(_elem2124); + _elem2140 = iprot.readString(); + struct.success.add(_elem2140); } } struct.setSuccessIsSet(true); @@ -291289,14 +291289,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_all_vers case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2126 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2126.size); - @org.apache.thrift.annotation.Nullable SchemaVersion _elem2127; - for (int _i2128 = 0; _i2128 < _list2126.size; ++_i2128) + org.apache.thrift.protocol.TList _list2142 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2142.size); + @org.apache.thrift.annotation.Nullable SchemaVersion _elem2143; + for (int _i2144 = 0; _i2144 < _list2142.size; ++_i2144) { - _elem2127 = new SchemaVersion(); - _elem2127.read(iprot); - struct.success.add(_elem2127); + _elem2143 = new SchemaVersion(); + _elem2143.read(iprot); + struct.success.add(_elem2143); } iprot.readListEnd(); } @@ -291340,9 +291340,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_all_ver oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (SchemaVersion _iter2129 : struct.success) + for (SchemaVersion _iter2145 : struct.success) { - _iter2129.write(oprot); + _iter2145.write(oprot); } oprot.writeListEnd(); } @@ -291389,9 +291389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_all_vers if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (SchemaVersion _iter2130 : struct.success) + for (SchemaVersion _iter2146 : struct.success) { - _iter2130.write(oprot); + _iter2146.write(oprot); } } } @@ -291409,14 +291409,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_all_versi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2131 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2131.size); - @org.apache.thrift.annotation.Nullable SchemaVersion _elem2132; - for (int _i2133 = 0; _i2133 < _list2131.size; ++_i2133) + org.apache.thrift.protocol.TList _list2147 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2147.size); + @org.apache.thrift.annotation.Nullable SchemaVersion _elem2148; + for (int _i2149 = 0; _i2149 < _list2147.size; ++_i2149) { - _elem2132 = new SchemaVersion(); - _elem2132.read(iprot); - struct.success.add(_elem2132); + _elem2148 = new SchemaVersion(); + _elem2148.read(iprot); + struct.success.add(_elem2148); } } struct.setSuccessIsSet(true); @@ -300031,14 +300031,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_runtime_stats_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2134 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2134.size); - @org.apache.thrift.annotation.Nullable RuntimeStat _elem2135; - for (int _i2136 = 0; _i2136 < _list2134.size; ++_i2136) + org.apache.thrift.protocol.TList _list2150 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2150.size); + @org.apache.thrift.annotation.Nullable RuntimeStat _elem2151; + for (int _i2152 = 0; _i2152 < _list2150.size; ++_i2152) { - _elem2135 = new RuntimeStat(); - _elem2135.read(iprot); - struct.success.add(_elem2135); + _elem2151 = new RuntimeStat(); + _elem2151.read(iprot); + struct.success.add(_elem2151); } iprot.readListEnd(); } @@ -300073,9 +300073,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_runtime_stats_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (RuntimeStat _iter2137 : struct.success) + for (RuntimeStat _iter2153 : struct.success) { - _iter2137.write(oprot); + _iter2153.write(oprot); } oprot.writeListEnd(); } @@ -300114,9 +300114,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_runtime_stats_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (RuntimeStat _iter2138 : struct.success) + for (RuntimeStat _iter2154 : struct.success) { - _iter2138.write(oprot); + _iter2154.write(oprot); } } } @@ -300131,14 +300131,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_runtime_stats_re java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2139 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2139.size); - @org.apache.thrift.annotation.Nullable RuntimeStat _elem2140; - for (int _i2141 = 0; _i2141 < _list2139.size; ++_i2141) + org.apache.thrift.protocol.TList _list2155 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2155.size); + @org.apache.thrift.annotation.Nullable RuntimeStat _elem2156; + for (int _i2157 = 0; _i2157 < _list2155.size; ++_i2157) { - _elem2140 = new RuntimeStat(); - _elem2140.read(iprot); - struct.success.add(_elem2140); + _elem2156 = new RuntimeStat(); + _elem2156.read(iprot); + struct.success.add(_elem2156); } } struct.setSuccessIsSet(true); @@ -310249,13 +310249,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_stored_proc case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2142 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2142.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2143; - for (int _i2144 = 0; _i2144 < _list2142.size; ++_i2144) + org.apache.thrift.protocol.TList _list2158 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2158.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2159; + for (int _i2160 = 0; _i2160 < _list2158.size; ++_i2160) { - _elem2143 = iprot.readString(); - struct.success.add(_elem2143); + _elem2159 = iprot.readString(); + struct.success.add(_elem2159); } iprot.readListEnd(); } @@ -310290,9 +310290,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_stored_pro oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2145 : struct.success) + for (java.lang.String _iter2161 : struct.success) { - oprot.writeString(_iter2145); + oprot.writeString(_iter2161); } oprot.writeListEnd(); } @@ -310331,9 +310331,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_stored_proc if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2146 : struct.success) + for (java.lang.String _iter2162 : struct.success) { - oprot.writeString(_iter2146); + oprot.writeString(_iter2162); } } } @@ -310348,13 +310348,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_stored_proce java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2147 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2147.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2148; - for (int _i2149 = 0; _i2149 < _list2147.size; ++_i2149) + org.apache.thrift.protocol.TList _list2163 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2163.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2164; + for (int _i2165 = 0; _i2165 < _list2163.size; ++_i2165) { - _elem2148 = iprot.readString(); - struct.success.add(_elem2148); + _elem2164 = iprot.readString(); + struct.success.add(_elem2164); } } struct.setSuccessIsSet(true); @@ -312811,13 +312811,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_packages_re case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2150 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2150.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2151; - for (int _i2152 = 0; _i2152 < _list2150.size; ++_i2152) + org.apache.thrift.protocol.TList _list2166 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2166.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2167; + for (int _i2168 = 0; _i2168 < _list2166.size; ++_i2168) { - _elem2151 = iprot.readString(); - struct.success.add(_elem2151); + _elem2167 = iprot.readString(); + struct.success.add(_elem2167); } iprot.readListEnd(); } @@ -312852,9 +312852,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_packages_r oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2153 : struct.success) + for (java.lang.String _iter2169 : struct.success) { - oprot.writeString(_iter2153); + oprot.writeString(_iter2169); } oprot.writeListEnd(); } @@ -312893,9 +312893,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_packages_re if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2154 : struct.success) + for (java.lang.String _iter2170 : struct.success) { - oprot.writeString(_iter2154); + oprot.writeString(_iter2170); } } } @@ -312910,13 +312910,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_packages_res java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2155 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2155.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2156; - for (int _i2157 = 0; _i2157 < _list2155.size; ++_i2157) + org.apache.thrift.protocol.TList _list2171 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2171.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2172; + for (int _i2173 = 0; _i2173 < _list2171.size; ++_i2173) { - _elem2156 = iprot.readString(); - struct.success.add(_elem2156); + _elem2172 = iprot.readString(); + struct.success.add(_elem2172); } } struct.setSuccessIsSet(true); @@ -314430,14 +314430,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_write_event case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2158 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2158.size); - @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2159; - for (int _i2160 = 0; _i2160 < _list2158.size; ++_i2160) + org.apache.thrift.protocol.TList _list2174 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2174.size); + @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2175; + for (int _i2176 = 0; _i2176 < _list2174.size; ++_i2176) { - _elem2159 = new WriteEventInfo(); - _elem2159.read(iprot); - struct.success.add(_elem2159); + _elem2175 = new WriteEventInfo(); + _elem2175.read(iprot); + struct.success.add(_elem2175); } iprot.readListEnd(); } @@ -314472,9 +314472,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_write_even oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (WriteEventInfo _iter2161 : struct.success) + for (WriteEventInfo _iter2177 : struct.success) { - _iter2161.write(oprot); + _iter2177.write(oprot); } oprot.writeListEnd(); } @@ -314513,9 +314513,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_write_event if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (WriteEventInfo _iter2162 : struct.success) + for (WriteEventInfo _iter2178 : struct.success) { - _iter2162.write(oprot); + _iter2178.write(oprot); } } } @@ -314530,14 +314530,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_write_event_ java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2163 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2163.size); - @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2164; - for (int _i2165 = 0; _i2165 < _list2163.size; ++_i2165) + org.apache.thrift.protocol.TList _list2179 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2179.size); + @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2180; + for (int _i2181 = 0; _i2181 < _list2179.size; ++_i2181) { - _elem2164 = new WriteEventInfo(); - _elem2164.read(iprot); - struct.success.add(_elem2164); + _elem2180 = new WriteEventInfo(); + _elem2180.read(iprot); + struct.success.add(_elem2180); } } struct.setSuccessIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java index 06a99f7397e8..35e6d1d44a78 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java @@ -733,14 +733,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMFullResourcePlan case 2: // POOLS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1214 = iprot.readListBegin(); - struct.pools = new java.util.ArrayList(_list1214.size); - @org.apache.thrift.annotation.Nullable WMPool _elem1215; - for (int _i1216 = 0; _i1216 < _list1214.size; ++_i1216) + org.apache.thrift.protocol.TList _list1230 = iprot.readListBegin(); + struct.pools = new java.util.ArrayList(_list1230.size); + @org.apache.thrift.annotation.Nullable WMPool _elem1231; + for (int _i1232 = 0; _i1232 < _list1230.size; ++_i1232) { - _elem1215 = new WMPool(); - _elem1215.read(iprot); - struct.pools.add(_elem1215); + _elem1231 = new WMPool(); + _elem1231.read(iprot); + struct.pools.add(_elem1231); } iprot.readListEnd(); } @@ -752,14 +752,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMFullResourcePlan case 3: // MAPPINGS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1217 = iprot.readListBegin(); - struct.mappings = new java.util.ArrayList(_list1217.size); - @org.apache.thrift.annotation.Nullable WMMapping _elem1218; - for (int _i1219 = 0; _i1219 < _list1217.size; ++_i1219) + org.apache.thrift.protocol.TList _list1233 = iprot.readListBegin(); + struct.mappings = new java.util.ArrayList(_list1233.size); + @org.apache.thrift.annotation.Nullable WMMapping _elem1234; + for (int _i1235 = 0; _i1235 < _list1233.size; ++_i1235) { - _elem1218 = new WMMapping(); - _elem1218.read(iprot); - struct.mappings.add(_elem1218); + _elem1234 = new WMMapping(); + _elem1234.read(iprot); + struct.mappings.add(_elem1234); } iprot.readListEnd(); } @@ -771,14 +771,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMFullResourcePlan case 4: // TRIGGERS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1220 = iprot.readListBegin(); - struct.triggers = new java.util.ArrayList(_list1220.size); - @org.apache.thrift.annotation.Nullable WMTrigger _elem1221; - for (int _i1222 = 0; _i1222 < _list1220.size; ++_i1222) + org.apache.thrift.protocol.TList _list1236 = iprot.readListBegin(); + struct.triggers = new java.util.ArrayList(_list1236.size); + @org.apache.thrift.annotation.Nullable WMTrigger _elem1237; + for (int _i1238 = 0; _i1238 < _list1236.size; ++_i1238) { - _elem1221 = new WMTrigger(); - _elem1221.read(iprot); - struct.triggers.add(_elem1221); + _elem1237 = new WMTrigger(); + _elem1237.read(iprot); + struct.triggers.add(_elem1237); } iprot.readListEnd(); } @@ -790,14 +790,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMFullResourcePlan case 5: // POOL_TRIGGERS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1223 = iprot.readListBegin(); - struct.poolTriggers = new java.util.ArrayList(_list1223.size); - @org.apache.thrift.annotation.Nullable WMPoolTrigger _elem1224; - for (int _i1225 = 0; _i1225 < _list1223.size; ++_i1225) + org.apache.thrift.protocol.TList _list1239 = iprot.readListBegin(); + struct.poolTriggers = new java.util.ArrayList(_list1239.size); + @org.apache.thrift.annotation.Nullable WMPoolTrigger _elem1240; + for (int _i1241 = 0; _i1241 < _list1239.size; ++_i1241) { - _elem1224 = new WMPoolTrigger(); - _elem1224.read(iprot); - struct.poolTriggers.add(_elem1224); + _elem1240 = new WMPoolTrigger(); + _elem1240.read(iprot); + struct.poolTriggers.add(_elem1240); } iprot.readListEnd(); } @@ -828,9 +828,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMFullResourcePlan oprot.writeFieldBegin(POOLS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size())); - for (WMPool _iter1226 : struct.pools) + for (WMPool _iter1242 : struct.pools) { - _iter1226.write(oprot); + _iter1242.write(oprot); } oprot.writeListEnd(); } @@ -841,9 +841,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMFullResourcePlan oprot.writeFieldBegin(MAPPINGS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size())); - for (WMMapping _iter1227 : struct.mappings) + for (WMMapping _iter1243 : struct.mappings) { - _iter1227.write(oprot); + _iter1243.write(oprot); } oprot.writeListEnd(); } @@ -855,9 +855,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMFullResourcePlan oprot.writeFieldBegin(TRIGGERS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size())); - for (WMTrigger _iter1228 : struct.triggers) + for (WMTrigger _iter1244 : struct.triggers) { - _iter1228.write(oprot); + _iter1244.write(oprot); } oprot.writeListEnd(); } @@ -869,9 +869,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMFullResourcePlan oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size())); - for (WMPoolTrigger _iter1229 : struct.poolTriggers) + for (WMPoolTrigger _iter1245 : struct.poolTriggers) { - _iter1229.write(oprot); + _iter1245.write(oprot); } oprot.writeListEnd(); } @@ -898,9 +898,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WMFullResourcePlan struct.plan.write(oprot); { oprot.writeI32(struct.pools.size()); - for (WMPool _iter1230 : struct.pools) + for (WMPool _iter1246 : struct.pools) { - _iter1230.write(oprot); + _iter1246.write(oprot); } } java.util.BitSet optionals = new java.util.BitSet(); @@ -917,27 +917,27 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WMFullResourcePlan if (struct.isSetMappings()) { { oprot.writeI32(struct.mappings.size()); - for (WMMapping _iter1231 : struct.mappings) + for (WMMapping _iter1247 : struct.mappings) { - _iter1231.write(oprot); + _iter1247.write(oprot); } } } if (struct.isSetTriggers()) { { oprot.writeI32(struct.triggers.size()); - for (WMTrigger _iter1232 : struct.triggers) + for (WMTrigger _iter1248 : struct.triggers) { - _iter1232.write(oprot); + _iter1248.write(oprot); } } } if (struct.isSetPoolTriggers()) { { oprot.writeI32(struct.poolTriggers.size()); - for (WMPoolTrigger _iter1233 : struct.poolTriggers) + for (WMPoolTrigger _iter1249 : struct.poolTriggers) { - _iter1233.write(oprot); + _iter1249.write(oprot); } } } @@ -950,56 +950,56 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WMFullResourcePlan s struct.plan.read(iprot); struct.setPlanIsSet(true); { - org.apache.thrift.protocol.TList _list1234 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.pools = new java.util.ArrayList(_list1234.size); - @org.apache.thrift.annotation.Nullable WMPool _elem1235; - for (int _i1236 = 0; _i1236 < _list1234.size; ++_i1236) + org.apache.thrift.protocol.TList _list1250 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.pools = new java.util.ArrayList(_list1250.size); + @org.apache.thrift.annotation.Nullable WMPool _elem1251; + for (int _i1252 = 0; _i1252 < _list1250.size; ++_i1252) { - _elem1235 = new WMPool(); - _elem1235.read(iprot); - struct.pools.add(_elem1235); + _elem1251 = new WMPool(); + _elem1251.read(iprot); + struct.pools.add(_elem1251); } } struct.setPoolsIsSet(true); java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1237 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.mappings = new java.util.ArrayList(_list1237.size); - @org.apache.thrift.annotation.Nullable WMMapping _elem1238; - for (int _i1239 = 0; _i1239 < _list1237.size; ++_i1239) + org.apache.thrift.protocol.TList _list1253 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.mappings = new java.util.ArrayList(_list1253.size); + @org.apache.thrift.annotation.Nullable WMMapping _elem1254; + for (int _i1255 = 0; _i1255 < _list1253.size; ++_i1255) { - _elem1238 = new WMMapping(); - _elem1238.read(iprot); - struct.mappings.add(_elem1238); + _elem1254 = new WMMapping(); + _elem1254.read(iprot); + struct.mappings.add(_elem1254); } } struct.setMappingsIsSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1240 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.triggers = new java.util.ArrayList(_list1240.size); - @org.apache.thrift.annotation.Nullable WMTrigger _elem1241; - for (int _i1242 = 0; _i1242 < _list1240.size; ++_i1242) + org.apache.thrift.protocol.TList _list1256 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.triggers = new java.util.ArrayList(_list1256.size); + @org.apache.thrift.annotation.Nullable WMTrigger _elem1257; + for (int _i1258 = 0; _i1258 < _list1256.size; ++_i1258) { - _elem1241 = new WMTrigger(); - _elem1241.read(iprot); - struct.triggers.add(_elem1241); + _elem1257 = new WMTrigger(); + _elem1257.read(iprot); + struct.triggers.add(_elem1257); } } struct.setTriggersIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1243 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.poolTriggers = new java.util.ArrayList(_list1243.size); - @org.apache.thrift.annotation.Nullable WMPoolTrigger _elem1244; - for (int _i1245 = 0; _i1245 < _list1243.size; ++_i1245) + org.apache.thrift.protocol.TList _list1259 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.poolTriggers = new java.util.ArrayList(_list1259.size); + @org.apache.thrift.annotation.Nullable WMPoolTrigger _elem1260; + for (int _i1261 = 0; _i1261 < _list1259.size; ++_i1261) { - _elem1244 = new WMPoolTrigger(); - _elem1244.read(iprot); - struct.poolTriggers.add(_elem1244); + _elem1260 = new WMPoolTrigger(); + _elem1260.read(iprot); + struct.poolTriggers.add(_elem1260); } } struct.setPoolTriggersIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java index b77979a16131..3a332defef9c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java @@ -321,14 +321,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMGetAllResourcePla case 1: // RESOURCE_PLANS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1246 = iprot.readListBegin(); - struct.resourcePlans = new java.util.ArrayList(_list1246.size); - @org.apache.thrift.annotation.Nullable WMResourcePlan _elem1247; - for (int _i1248 = 0; _i1248 < _list1246.size; ++_i1248) + org.apache.thrift.protocol.TList _list1262 = iprot.readListBegin(); + struct.resourcePlans = new java.util.ArrayList(_list1262.size); + @org.apache.thrift.annotation.Nullable WMResourcePlan _elem1263; + for (int _i1264 = 0; _i1264 < _list1262.size; ++_i1264) { - _elem1247 = new WMResourcePlan(); - _elem1247.read(iprot); - struct.resourcePlans.add(_elem1247); + _elem1263 = new WMResourcePlan(); + _elem1263.read(iprot); + struct.resourcePlans.add(_elem1263); } iprot.readListEnd(); } @@ -355,9 +355,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMGetAllResourcePl oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size())); - for (WMResourcePlan _iter1249 : struct.resourcePlans) + for (WMResourcePlan _iter1265 : struct.resourcePlans) { - _iter1249.write(oprot); + _iter1265.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WMGetAllResourcePla if (struct.isSetResourcePlans()) { { oprot.writeI32(struct.resourcePlans.size()); - for (WMResourcePlan _iter1250 : struct.resourcePlans) + for (WMResourcePlan _iter1266 : struct.resourcePlans) { - _iter1250.write(oprot); + _iter1266.write(oprot); } } } @@ -403,14 +403,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WMGetAllResourcePlan java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1251 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.resourcePlans = new java.util.ArrayList(_list1251.size); - @org.apache.thrift.annotation.Nullable WMResourcePlan _elem1252; - for (int _i1253 = 0; _i1253 < _list1251.size; ++_i1253) + org.apache.thrift.protocol.TList _list1267 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.resourcePlans = new java.util.ArrayList(_list1267.size); + @org.apache.thrift.annotation.Nullable WMResourcePlan _elem1268; + for (int _i1269 = 0; _i1269 < _list1267.size; ++_i1269) { - _elem1252 = new WMResourcePlan(); - _elem1252.read(iprot); - struct.resourcePlans.add(_elem1252); + _elem1268 = new WMResourcePlan(); + _elem1268.read(iprot); + struct.resourcePlans.add(_elem1268); } } struct.setResourcePlansIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java index d7db7bcbfeb5..e22c7c47f364 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java @@ -321,14 +321,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMGetTriggersForRes case 1: // TRIGGERS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1270 = iprot.readListBegin(); - struct.triggers = new java.util.ArrayList(_list1270.size); - @org.apache.thrift.annotation.Nullable WMTrigger _elem1271; - for (int _i1272 = 0; _i1272 < _list1270.size; ++_i1272) + org.apache.thrift.protocol.TList _list1286 = iprot.readListBegin(); + struct.triggers = new java.util.ArrayList(_list1286.size); + @org.apache.thrift.annotation.Nullable WMTrigger _elem1287; + for (int _i1288 = 0; _i1288 < _list1286.size; ++_i1288) { - _elem1271 = new WMTrigger(); - _elem1271.read(iprot); - struct.triggers.add(_elem1271); + _elem1287 = new WMTrigger(); + _elem1287.read(iprot); + struct.triggers.add(_elem1287); } iprot.readListEnd(); } @@ -355,9 +355,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMGetTriggersForRe oprot.writeFieldBegin(TRIGGERS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size())); - for (WMTrigger _iter1273 : struct.triggers) + for (WMTrigger _iter1289 : struct.triggers) { - _iter1273.write(oprot); + _iter1289.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WMGetTriggersForRes if (struct.isSetTriggers()) { { oprot.writeI32(struct.triggers.size()); - for (WMTrigger _iter1274 : struct.triggers) + for (WMTrigger _iter1290 : struct.triggers) { - _iter1274.write(oprot); + _iter1290.write(oprot); } } } @@ -403,14 +403,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WMGetTriggersForReso java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1275 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.triggers = new java.util.ArrayList(_list1275.size); - @org.apache.thrift.annotation.Nullable WMTrigger _elem1276; - for (int _i1277 = 0; _i1277 < _list1275.size; ++_i1277) + org.apache.thrift.protocol.TList _list1291 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.triggers = new java.util.ArrayList(_list1291.size); + @org.apache.thrift.annotation.Nullable WMTrigger _elem1292; + for (int _i1293 = 0; _i1293 < _list1291.size; ++_i1293) { - _elem1276 = new WMTrigger(); - _elem1276.read(iprot); - struct.triggers.add(_elem1276); + _elem1292 = new WMTrigger(); + _elem1292.read(iprot); + struct.triggers.add(_elem1292); } } struct.setTriggersIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java index 134a3e1fe2b7..7e0d6567ba09 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java @@ -417,13 +417,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMValidateResourceP case 1: // ERRORS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1254 = iprot.readListBegin(); - struct.errors = new java.util.ArrayList(_list1254.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1255; - for (int _i1256 = 0; _i1256 < _list1254.size; ++_i1256) + org.apache.thrift.protocol.TList _list1270 = iprot.readListBegin(); + struct.errors = new java.util.ArrayList(_list1270.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1271; + for (int _i1272 = 0; _i1272 < _list1270.size; ++_i1272) { - _elem1255 = iprot.readString(); - struct.errors.add(_elem1255); + _elem1271 = iprot.readString(); + struct.errors.add(_elem1271); } iprot.readListEnd(); } @@ -435,13 +435,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WMValidateResourceP case 2: // WARNINGS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1257 = iprot.readListBegin(); - struct.warnings = new java.util.ArrayList(_list1257.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1258; - for (int _i1259 = 0; _i1259 < _list1257.size; ++_i1259) + org.apache.thrift.protocol.TList _list1273 = iprot.readListBegin(); + struct.warnings = new java.util.ArrayList(_list1273.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1274; + for (int _i1275 = 0; _i1275 < _list1273.size; ++_i1275) { - _elem1258 = iprot.readString(); - struct.warnings.add(_elem1258); + _elem1274 = iprot.readString(); + struct.warnings.add(_elem1274); } iprot.readListEnd(); } @@ -468,9 +468,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMValidateResource oprot.writeFieldBegin(ERRORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size())); - for (java.lang.String _iter1260 : struct.errors) + for (java.lang.String _iter1276 : struct.errors) { - oprot.writeString(_iter1260); + oprot.writeString(_iter1276); } oprot.writeListEnd(); } @@ -482,9 +482,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WMValidateResource oprot.writeFieldBegin(WARNINGS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size())); - for (java.lang.String _iter1261 : struct.warnings) + for (java.lang.String _iter1277 : struct.warnings) { - oprot.writeString(_iter1261); + oprot.writeString(_iter1277); } oprot.writeListEnd(); } @@ -519,18 +519,18 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WMValidateResourceP if (struct.isSetErrors()) { { oprot.writeI32(struct.errors.size()); - for (java.lang.String _iter1262 : struct.errors) + for (java.lang.String _iter1278 : struct.errors) { - oprot.writeString(_iter1262); + oprot.writeString(_iter1278); } } } if (struct.isSetWarnings()) { { oprot.writeI32(struct.warnings.size()); - for (java.lang.String _iter1263 : struct.warnings) + for (java.lang.String _iter1279 : struct.warnings) { - oprot.writeString(_iter1263); + oprot.writeString(_iter1279); } } } @@ -542,26 +542,26 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WMValidateResourcePl java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1264 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.errors = new java.util.ArrayList(_list1264.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1265; - for (int _i1266 = 0; _i1266 < _list1264.size; ++_i1266) + org.apache.thrift.protocol.TList _list1280 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.errors = new java.util.ArrayList(_list1280.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1281; + for (int _i1282 = 0; _i1282 < _list1280.size; ++_i1282) { - _elem1265 = iprot.readString(); - struct.errors.add(_elem1265); + _elem1281 = iprot.readString(); + struct.errors.add(_elem1281); } } struct.setErrorsIsSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1267 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.warnings = new java.util.ArrayList(_list1267.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1268; - for (int _i1269 = 0; _i1269 < _list1267.size; ++_i1269) + org.apache.thrift.protocol.TList _list1283 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.warnings = new java.util.ArrayList(_list1283.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1284; + for (int _i1285 = 0; _i1285 < _list1283.size; ++_i1285) { - _elem1268 = iprot.readString(); - struct.warnings.add(_elem1268); + _elem1284 = iprot.readString(); + struct.warnings.add(_elem1284); } } struct.setWarningsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogBatchRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogBatchRequest.java index bf7114b56cb2..8932fcdfda53 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogBatchRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogBatchRequest.java @@ -608,14 +608,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WriteNotificationLo case 4: // REQUEST_LIST if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin(); - struct.requestList = new java.util.ArrayList(_list1058.size); - @org.apache.thrift.annotation.Nullable WriteNotificationLogRequest _elem1059; - for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060) + org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin(); + struct.requestList = new java.util.ArrayList(_list1074.size); + @org.apache.thrift.annotation.Nullable WriteNotificationLogRequest _elem1075; + for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076) { - _elem1059 = new WriteNotificationLogRequest(); - _elem1059.read(iprot); - struct.requestList.add(_elem1059); + _elem1075 = new WriteNotificationLogRequest(); + _elem1075.read(iprot); + struct.requestList.add(_elem1075); } iprot.readListEnd(); } @@ -656,9 +656,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WriteNotificationL oprot.writeFieldBegin(REQUEST_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.requestList.size())); - for (WriteNotificationLogRequest _iter1061 : struct.requestList) + for (WriteNotificationLogRequest _iter1077 : struct.requestList) { - _iter1061.write(oprot); + _iter1077.write(oprot); } oprot.writeListEnd(); } @@ -686,9 +686,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLo oprot.writeString(struct.table); { oprot.writeI32(struct.requestList.size()); - for (WriteNotificationLogRequest _iter1062 : struct.requestList) + for (WriteNotificationLogRequest _iter1078 : struct.requestList) { - _iter1062.write(oprot); + _iter1078.write(oprot); } } } @@ -703,14 +703,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLog struct.table = iprot.readString(); struct.setTableIsSet(true); { - org.apache.thrift.protocol.TList _list1063 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.requestList = new java.util.ArrayList(_list1063.size); - @org.apache.thrift.annotation.Nullable WriteNotificationLogRequest _elem1064; - for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065) + org.apache.thrift.protocol.TList _list1079 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.requestList = new java.util.ArrayList(_list1079.size); + @org.apache.thrift.annotation.Nullable WriteNotificationLogRequest _elem1080; + for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081) { - _elem1064 = new WriteNotificationLogRequest(); - _elem1064.read(iprot); - struct.requestList.add(_elem1064); + _elem1080 = new WriteNotificationLogRequest(); + _elem1080.read(iprot); + struct.requestList.add(_elem1080); } } struct.setRequestListIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java index 9a5ca97a69d4..66e9cf796247 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java @@ -782,13 +782,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, WriteNotificationLo case 6: // PARTITION_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin(); - struct.partitionVals = new java.util.ArrayList(_list1050.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1051; - for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052) + org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin(); + struct.partitionVals = new java.util.ArrayList(_list1066.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1067; + for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068) { - _elem1051 = iprot.readString(); - struct.partitionVals.add(_elem1051); + _elem1067 = iprot.readString(); + struct.partitionVals.add(_elem1067); } iprot.readListEnd(); } @@ -836,9 +836,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, WriteNotificationL oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size())); - for (java.lang.String _iter1053 : struct.partitionVals) + for (java.lang.String _iter1069 : struct.partitionVals) { - oprot.writeString(_iter1053); + oprot.writeString(_iter1069); } oprot.writeListEnd(); } @@ -875,9 +875,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLo if (struct.isSetPartitionVals()) { { oprot.writeI32(struct.partitionVals.size()); - for (java.lang.String _iter1054 : struct.partitionVals) + for (java.lang.String _iter1070 : struct.partitionVals) { - oprot.writeString(_iter1054); + oprot.writeString(_iter1070); } } } @@ -900,13 +900,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLog java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1055 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partitionVals = new java.util.ArrayList(_list1055.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1056; - for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057) + org.apache.thrift.protocol.TList _list1071 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partitionVals = new java.util.ArrayList(_list1071.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1072; + for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073) { - _elem1056 = iprot.readString(); - struct.partitionVals.add(_elem1056); + _elem1072 = iprot.readString(); + struct.partitionVals.add(_elem1072); } } struct.setPartitionValsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterPartitionsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterPartitionsRequest.php index 342c23f7c3f5..b69c711f4c44 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterPartitionsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterPartitionsRequest.php @@ -192,14 +192,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->partitions = array(); - $_size1199 = 0; - $_etype1202 = 0; - $xfer += $input->readListBegin($_etype1202, $_size1199); - for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203) { - $elem1204 = null; - $elem1204 = new \metastore\Partition(); - $xfer += $elem1204->read($input); - $this->partitions []= $elem1204; + $_size1213 = 0; + $_etype1216 = 0; + $xfer += $input->readListBegin($_etype1216, $_size1213); + for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217) { + $elem1218 = null; + $elem1218 = new \metastore\Partition(); + $xfer += $elem1218->read($input); + $this->partitions []= $elem1218; } $xfer += $input->readListEnd(); } else { @@ -238,14 +238,14 @@ public function read($input) case 9: if ($ftype == TType::LST) { $this->partitionColSchema = array(); - $_size1205 = 0; - $_etype1208 = 0; - $xfer += $input->readListBegin($_etype1208, $_size1205); - for ($_i1209 = 0; $_i1209 < $_size1205; ++$_i1209) { - $elem1210 = null; - $elem1210 = new \metastore\FieldSchema(); - $xfer += $elem1210->read($input); - $this->partitionColSchema []= $elem1210; + $_size1219 = 0; + $_etype1222 = 0; + $xfer += $input->readListBegin($_etype1222, $_size1219); + for ($_i1223 = 0; $_i1223 < $_size1219; ++$_i1223) { + $elem1224 = null; + $elem1224 = new \metastore\FieldSchema(); + $xfer += $elem1224->read($input); + $this->partitionColSchema []= $elem1224; } $xfer += $input->readListEnd(); } else { @@ -287,8 +287,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitions', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->partitions)); - foreach ($this->partitions as $iter1211) { - $xfer += $iter1211->write($output); + foreach ($this->partitions as $iter1225) { + $xfer += $iter1225->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -322,8 +322,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionColSchema', TType::LST, 9); $output->writeListBegin(TType::STRUCT, count($this->partitionColSchema)); - foreach ($this->partitionColSchema as $iter1212) { - $xfer += $iter1212->write($output); + foreach ($this->partitionColSchema as $iter1226) { + $xfer += $iter1226->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterTableRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterTableRequest.php index 3b9ac75fb4af..6cd3d7e8ac57 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterTableRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/AlterTableRequest.php @@ -241,13 +241,13 @@ public function read($input) case 8: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1220 = 0; - $_etype1223 = 0; - $xfer += $input->readListBegin($_etype1223, $_size1220); - for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224) { - $elem1225 = null; - $xfer += $input->readString($elem1225); - $this->processorCapabilities []= $elem1225; + $_size1234 = 0; + $_etype1237 = 0; + $xfer += $input->readListBegin($_etype1237, $_size1234); + for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238) { + $elem1239 = null; + $xfer += $input->readString($elem1239); + $this->processorCapabilities []= $elem1239; } $xfer += $input->readListEnd(); } else { @@ -336,8 +336,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 8); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1226) { - $xfer += $output->writeString($iter1226); + foreach ($this->processorCapabilities as $iter1240) { + $xfer += $output->writeString($iter1240); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClearFileMetadataRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClearFileMetadataRequest.php index c9ce49af83c8..34ec6abc029d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClearFileMetadataRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClearFileMetadataRequest.php @@ -68,13 +68,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fileIds = array(); - $_size987 = 0; - $_etype990 = 0; - $xfer += $input->readListBegin($_etype990, $_size987); - for ($_i991 = 0; $_i991 < $_size987; ++$_i991) { - $elem992 = null; - $xfer += $input->readI64($elem992); - $this->fileIds []= $elem992; + $_size1001 = 0; + $_etype1004 = 0; + $xfer += $input->readListBegin($_etype1004, $_size1001); + for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005) { + $elem1006 = null; + $xfer += $input->readI64($elem1006); + $this->fileIds []= $elem1006; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1); $output->writeListBegin(TType::I64, count($this->fileIds)); - foreach ($this->fileIds as $iter993) { - $xfer += $output->writeI64($iter993); + foreach ($this->fileIds as $iter1007) { + $xfer += $output->writeI64($iter1007); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClientCapabilities.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClientCapabilities.php index b53b5d027176..637fe8a78e5e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClientCapabilities.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ClientCapabilities.php @@ -69,13 +69,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->values = array(); - $_size1001 = 0; - $_etype1004 = 0; - $xfer += $input->readListBegin($_etype1004, $_size1001); - for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005) { - $elem1006 = null; - $xfer += $input->readI32($elem1006); - $this->values []= $elem1006; + $_size1015 = 0; + $_etype1018 = 0; + $xfer += $input->readListBegin($_etype1018, $_size1015); + for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019) { + $elem1020 = null; + $xfer += $input->readI32($elem1020); + $this->values []= $elem1020; } $xfer += $input->readListEnd(); } else { @@ -102,8 +102,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('values', TType::LST, 1); $output->writeListBegin(TType::I32, count($this->values)); - foreach ($this->values as $iter1007) { - $xfer += $output->writeI32($iter1007); + foreach ($this->values as $iter1021) { + $xfer += $output->writeI32($iter1021); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateDatabaseRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateDatabaseRequest.php index dae2f101bd72..e3804ac7a63c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateDatabaseRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateDatabaseRequest.php @@ -227,16 +227,16 @@ public function read($input) case 4: if ($ftype == TType::MAP) { $this->parameters = array(); - $_size1190 = 0; - $_ktype1191 = 0; - $_vtype1192 = 0; - $xfer += $input->readMapBegin($_ktype1191, $_vtype1192, $_size1190); - for ($_i1194 = 0; $_i1194 < $_size1190; ++$_i1194) { - $key1195 = ''; - $val1196 = ''; - $xfer += $input->readString($key1195); - $xfer += $input->readString($val1196); - $this->parameters[$key1195] = $val1196; + $_size1204 = 0; + $_ktype1205 = 0; + $_vtype1206 = 0; + $xfer += $input->readMapBegin($_ktype1205, $_vtype1206, $_size1204); + for ($_i1208 = 0; $_i1208 < $_size1204; ++$_i1208) { + $key1209 = ''; + $val1210 = ''; + $xfer += $input->readString($key1209); + $xfer += $input->readString($val1210); + $this->parameters[$key1209] = $val1210; } $xfer += $input->readMapEnd(); } else { @@ -335,9 +335,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('parameters', TType::MAP, 4); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters)); - foreach ($this->parameters as $kiter1197 => $viter1198) { - $xfer += $output->writeString($kiter1197); - $xfer += $output->writeString($viter1198); + foreach ($this->parameters as $kiter1211 => $viter1212) { + $xfer += $output->writeString($kiter1211); + $xfer += $output->writeString($viter1212); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateTableRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateTableRequest.php index dcc0c6598264..00a335a8accb 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateTableRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CreateTableRequest.php @@ -224,14 +224,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->primaryKeys = array(); - $_size1141 = 0; - $_etype1144 = 0; - $xfer += $input->readListBegin($_etype1144, $_size1141); - for ($_i1145 = 0; $_i1145 < $_size1141; ++$_i1145) { - $elem1146 = null; - $elem1146 = new \metastore\SQLPrimaryKey(); - $xfer += $elem1146->read($input); - $this->primaryKeys []= $elem1146; + $_size1155 = 0; + $_etype1158 = 0; + $xfer += $input->readListBegin($_etype1158, $_size1155); + for ($_i1159 = 0; $_i1159 < $_size1155; ++$_i1159) { + $elem1160 = null; + $elem1160 = new \metastore\SQLPrimaryKey(); + $xfer += $elem1160->read($input); + $this->primaryKeys []= $elem1160; } $xfer += $input->readListEnd(); } else { @@ -241,14 +241,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->foreignKeys = array(); - $_size1147 = 0; - $_etype1150 = 0; - $xfer += $input->readListBegin($_etype1150, $_size1147); - for ($_i1151 = 0; $_i1151 < $_size1147; ++$_i1151) { - $elem1152 = null; - $elem1152 = new \metastore\SQLForeignKey(); - $xfer += $elem1152->read($input); - $this->foreignKeys []= $elem1152; + $_size1161 = 0; + $_etype1164 = 0; + $xfer += $input->readListBegin($_etype1164, $_size1161); + for ($_i1165 = 0; $_i1165 < $_size1161; ++$_i1165) { + $elem1166 = null; + $elem1166 = new \metastore\SQLForeignKey(); + $xfer += $elem1166->read($input); + $this->foreignKeys []= $elem1166; } $xfer += $input->readListEnd(); } else { @@ -258,14 +258,14 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->uniqueConstraints = array(); - $_size1153 = 0; - $_etype1156 = 0; - $xfer += $input->readListBegin($_etype1156, $_size1153); - for ($_i1157 = 0; $_i1157 < $_size1153; ++$_i1157) { - $elem1158 = null; - $elem1158 = new \metastore\SQLUniqueConstraint(); - $xfer += $elem1158->read($input); - $this->uniqueConstraints []= $elem1158; + $_size1167 = 0; + $_etype1170 = 0; + $xfer += $input->readListBegin($_etype1170, $_size1167); + for ($_i1171 = 0; $_i1171 < $_size1167; ++$_i1171) { + $elem1172 = null; + $elem1172 = new \metastore\SQLUniqueConstraint(); + $xfer += $elem1172->read($input); + $this->uniqueConstraints []= $elem1172; } $xfer += $input->readListEnd(); } else { @@ -275,14 +275,14 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->notNullConstraints = array(); - $_size1159 = 0; - $_etype1162 = 0; - $xfer += $input->readListBegin($_etype1162, $_size1159); - for ($_i1163 = 0; $_i1163 < $_size1159; ++$_i1163) { - $elem1164 = null; - $elem1164 = new \metastore\SQLNotNullConstraint(); - $xfer += $elem1164->read($input); - $this->notNullConstraints []= $elem1164; + $_size1173 = 0; + $_etype1176 = 0; + $xfer += $input->readListBegin($_etype1176, $_size1173); + for ($_i1177 = 0; $_i1177 < $_size1173; ++$_i1177) { + $elem1178 = null; + $elem1178 = new \metastore\SQLNotNullConstraint(); + $xfer += $elem1178->read($input); + $this->notNullConstraints []= $elem1178; } $xfer += $input->readListEnd(); } else { @@ -292,14 +292,14 @@ public function read($input) case 7: if ($ftype == TType::LST) { $this->defaultConstraints = array(); - $_size1165 = 0; - $_etype1168 = 0; - $xfer += $input->readListBegin($_etype1168, $_size1165); - for ($_i1169 = 0; $_i1169 < $_size1165; ++$_i1169) { - $elem1170 = null; - $elem1170 = new \metastore\SQLDefaultConstraint(); - $xfer += $elem1170->read($input); - $this->defaultConstraints []= $elem1170; + $_size1179 = 0; + $_etype1182 = 0; + $xfer += $input->readListBegin($_etype1182, $_size1179); + for ($_i1183 = 0; $_i1183 < $_size1179; ++$_i1183) { + $elem1184 = null; + $elem1184 = new \metastore\SQLDefaultConstraint(); + $xfer += $elem1184->read($input); + $this->defaultConstraints []= $elem1184; } $xfer += $input->readListEnd(); } else { @@ -309,14 +309,14 @@ public function read($input) case 8: if ($ftype == TType::LST) { $this->checkConstraints = array(); - $_size1171 = 0; - $_etype1174 = 0; - $xfer += $input->readListBegin($_etype1174, $_size1171); - for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175) { - $elem1176 = null; - $elem1176 = new \metastore\SQLCheckConstraint(); - $xfer += $elem1176->read($input); - $this->checkConstraints []= $elem1176; + $_size1185 = 0; + $_etype1188 = 0; + $xfer += $input->readListBegin($_etype1188, $_size1185); + for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189) { + $elem1190 = null; + $elem1190 = new \metastore\SQLCheckConstraint(); + $xfer += $elem1190->read($input); + $this->checkConstraints []= $elem1190; } $xfer += $input->readListEnd(); } else { @@ -326,13 +326,13 @@ public function read($input) case 9: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1177 = 0; - $_etype1180 = 0; - $xfer += $input->readListBegin($_etype1180, $_size1177); - for ($_i1181 = 0; $_i1181 < $_size1177; ++$_i1181) { - $elem1182 = null; - $xfer += $input->readString($elem1182); - $this->processorCapabilities []= $elem1182; + $_size1191 = 0; + $_etype1194 = 0; + $xfer += $input->readListBegin($_etype1194, $_size1191); + for ($_i1195 = 0; $_i1195 < $_size1191; ++$_i1195) { + $elem1196 = null; + $xfer += $input->readString($elem1196); + $this->processorCapabilities []= $elem1196; } $xfer += $input->readListEnd(); } else { @@ -382,8 +382,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('primaryKeys', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->primaryKeys)); - foreach ($this->primaryKeys as $iter1183) { - $xfer += $iter1183->write($output); + foreach ($this->primaryKeys as $iter1197) { + $xfer += $iter1197->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -394,8 +394,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('foreignKeys', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->foreignKeys)); - foreach ($this->foreignKeys as $iter1184) { - $xfer += $iter1184->write($output); + foreach ($this->foreignKeys as $iter1198) { + $xfer += $iter1198->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -406,8 +406,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('uniqueConstraints', TType::LST, 5); $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints)); - foreach ($this->uniqueConstraints as $iter1185) { - $xfer += $iter1185->write($output); + foreach ($this->uniqueConstraints as $iter1199) { + $xfer += $iter1199->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -418,8 +418,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('notNullConstraints', TType::LST, 6); $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints)); - foreach ($this->notNullConstraints as $iter1186) { - $xfer += $iter1186->write($output); + foreach ($this->notNullConstraints as $iter1200) { + $xfer += $iter1200->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -430,8 +430,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('defaultConstraints', TType::LST, 7); $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints)); - foreach ($this->defaultConstraints as $iter1187) { - $xfer += $iter1187->write($output); + foreach ($this->defaultConstraints as $iter1201) { + $xfer += $iter1201->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -442,8 +442,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('checkConstraints', TType::LST, 8); $output->writeListBegin(TType::STRUCT, count($this->checkConstraints)); - foreach ($this->checkConstraints as $iter1188) { - $xfer += $iter1188->write($output); + foreach ($this->checkConstraints as $iter1202) { + $xfer += $iter1202->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -454,8 +454,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 9); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1189) { - $xfer += $output->writeString($iter1189); + foreach ($this->processorCapabilities as $iter1203) { + $xfer += $output->writeString($iter1203); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ExtendedTableInfo.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ExtendedTableInfo.php index 0767241d5b0a..37d58198cc3e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ExtendedTableInfo.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ExtendedTableInfo.php @@ -122,13 +122,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->requiredReadCapabilities = array(); - $_size1050 = 0; - $_etype1053 = 0; - $xfer += $input->readListBegin($_etype1053, $_size1050); - for ($_i1054 = 0; $_i1054 < $_size1050; ++$_i1054) { - $elem1055 = null; - $xfer += $input->readString($elem1055); - $this->requiredReadCapabilities []= $elem1055; + $_size1064 = 0; + $_etype1067 = 0; + $xfer += $input->readListBegin($_etype1067, $_size1064); + for ($_i1068 = 0; $_i1068 < $_size1064; ++$_i1068) { + $elem1069 = null; + $xfer += $input->readString($elem1069); + $this->requiredReadCapabilities []= $elem1069; } $xfer += $input->readListEnd(); } else { @@ -138,13 +138,13 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->requiredWriteCapabilities = array(); - $_size1056 = 0; - $_etype1059 = 0; - $xfer += $input->readListBegin($_etype1059, $_size1056); - for ($_i1060 = 0; $_i1060 < $_size1056; ++$_i1060) { - $elem1061 = null; - $xfer += $input->readString($elem1061); - $this->requiredWriteCapabilities []= $elem1061; + $_size1070 = 0; + $_etype1073 = 0; + $xfer += $input->readListBegin($_etype1073, $_size1070); + for ($_i1074 = 0; $_i1074 < $_size1070; ++$_i1074) { + $elem1075 = null; + $xfer += $input->readString($elem1075); + $this->requiredWriteCapabilities []= $elem1075; } $xfer += $input->readListEnd(); } else { @@ -181,8 +181,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('requiredReadCapabilities', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->requiredReadCapabilities)); - foreach ($this->requiredReadCapabilities as $iter1062) { - $xfer += $output->writeString($iter1062); + foreach ($this->requiredReadCapabilities as $iter1076) { + $xfer += $output->writeString($iter1076); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -193,8 +193,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('requiredWriteCapabilities', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->requiredWriteCapabilities)); - foreach ($this->requiredWriteCapabilities as $iter1063) { - $xfer += $output->writeString($iter1063); + foreach ($this->requiredWriteCapabilities as $iter1077) { + $xfer += $output->writeString($iter1077); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FindSchemasByColsResp.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FindSchemasByColsResp.php index 82c0d50b4465..aea2d4bffce0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FindSchemasByColsResp.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FindSchemasByColsResp.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->schemaVersions = array(); - $_size1134 = 0; - $_etype1137 = 0; - $xfer += $input->readListBegin($_etype1137, $_size1134); - for ($_i1138 = 0; $_i1138 < $_size1134; ++$_i1138) { - $elem1139 = null; - $elem1139 = new \metastore\SchemaVersionDescriptor(); - $xfer += $elem1139->read($input); - $this->schemaVersions []= $elem1139; + $_size1148 = 0; + $_etype1151 = 0; + $xfer += $input->readListBegin($_etype1151, $_size1148); + for ($_i1152 = 0; $_i1152 < $_size1148; ++$_i1152) { + $elem1153 = null; + $elem1153 = new \metastore\SchemaVersionDescriptor(); + $xfer += $elem1153->read($input); + $this->schemaVersions []= $elem1153; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('schemaVersions', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->schemaVersions)); - foreach ($this->schemaVersions as $iter1140) { - $xfer += $iter1140->write($output); + foreach ($this->schemaVersions as $iter1154) { + $xfer += $iter1154->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequest.php index ca94f48db5ea..96e38dcf332c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequest.php @@ -178,13 +178,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->partitionVals = array(); - $_size904 = 0; - $_etype907 = 0; - $xfer += $input->readListBegin($_etype907, $_size904); - for ($_i908 = 0; $_i908 < $_size904; ++$_i908) { - $elem909 = null; - $xfer += $input->readString($elem909); - $this->partitionVals []= $elem909; + $_size918 = 0; + $_etype921 = 0; + $xfer += $input->readListBegin($_etype921, $_size918); + for ($_i922 = 0; $_i922 < $_size918; ++$_i922) { + $elem923 = null; + $xfer += $input->readString($elem923); + $this->partitionVals []= $elem923; } $xfer += $input->readListEnd(); } else { @@ -201,16 +201,16 @@ public function read($input) case 7: if ($ftype == TType::MAP) { $this->tblParams = array(); - $_size910 = 0; - $_ktype911 = 0; - $_vtype912 = 0; - $xfer += $input->readMapBegin($_ktype911, $_vtype912, $_size910); - for ($_i914 = 0; $_i914 < $_size910; ++$_i914) { - $key915 = ''; - $val916 = ''; - $xfer += $input->readString($key915); - $xfer += $input->readString($val916); - $this->tblParams[$key915] = $val916; + $_size924 = 0; + $_ktype925 = 0; + $_vtype926 = 0; + $xfer += $input->readMapBegin($_ktype925, $_vtype926, $_size924); + for ($_i928 = 0; $_i928 < $_size924; ++$_i928) { + $key929 = ''; + $val930 = ''; + $xfer += $input->readString($key929); + $xfer += $input->readString($val930); + $this->tblParams[$key929] = $val930; } $xfer += $input->readMapEnd(); } else { @@ -260,8 +260,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->partitionVals)); - foreach ($this->partitionVals as $iter917) { - $xfer += $output->writeString($iter917); + foreach ($this->partitionVals as $iter931) { + $xfer += $output->writeString($iter931); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -277,9 +277,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('tblParams', TType::MAP, 7); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->tblParams)); - foreach ($this->tblParams as $kiter918 => $viter919) { - $xfer += $output->writeString($kiter918); - $xfer += $output->writeString($viter919); + foreach ($this->tblParams as $kiter932 => $viter933) { + $xfer += $output->writeString($kiter932); + $xfer += $output->writeString($viter933); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequestData.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequestData.php index 9d0cc22e8d5b..9129f5ca20ad 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequestData.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventRequestData.php @@ -102,14 +102,14 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->insertDatas = array(); - $_size897 = 0; - $_etype900 = 0; - $xfer += $input->readListBegin($_etype900, $_size897); - for ($_i901 = 0; $_i901 < $_size897; ++$_i901) { - $elem902 = null; - $elem902 = new \metastore\InsertEventRequestData(); - $xfer += $elem902->read($input); - $this->insertDatas []= $elem902; + $_size911 = 0; + $_etype914 = 0; + $xfer += $input->readListBegin($_etype914, $_size911); + for ($_i915 = 0; $_i915 < $_size911; ++$_i915) { + $elem916 = null; + $elem916 = new \metastore\InsertEventRequestData(); + $xfer += $elem916->read($input); + $this->insertDatas []= $elem916; } $xfer += $input->readListEnd(); } else { @@ -151,8 +151,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('insertDatas', TType::LST, 2); $output->writeListBegin(TType::STRUCT, count($this->insertDatas)); - foreach ($this->insertDatas as $iter903) { - $xfer += $iter903->write($output); + foreach ($this->insertDatas as $iter917) { + $xfer += $iter917->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventResponse.php index eaa71ebc5ad0..32ab25030cf9 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/FireEventResponse.php @@ -68,13 +68,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->eventIds = array(); - $_size920 = 0; - $_etype923 = 0; - $xfer += $input->readListBegin($_etype923, $_size920); - for ($_i924 = 0; $_i924 < $_size920; ++$_i924) { - $elem925 = null; - $xfer += $input->readI64($elem925); - $this->eventIds []= $elem925; + $_size934 = 0; + $_etype937 = 0; + $xfer += $input->readListBegin($_etype937, $_size934); + for ($_i938 = 0; $_i938 < $_size934; ++$_i938) { + $elem939 = null; + $xfer += $input->readI64($elem939); + $this->eventIds []= $elem939; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('eventIds', TType::LST, 1); $output->writeListBegin(TType::I64, count($this->eventIds)); - foreach ($this->eventIds as $iter926) { - $xfer += $output->writeI64($iter926); + foreach ($this->eventIds as $iter940) { + $xfer += $output->writeI64($iter940); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetAllFunctionsResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetAllFunctionsResponse.php index 0fc5aaf2447d..cce24359eaa4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetAllFunctionsResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetAllFunctionsResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->functions = array(); - $_size994 = 0; - $_etype997 = 0; - $xfer += $input->readListBegin($_etype997, $_size994); - for ($_i998 = 0; $_i998 < $_size994; ++$_i998) { - $elem999 = null; - $elem999 = new \metastore\Function(); - $xfer += $elem999->read($input); - $this->functions []= $elem999; + $_size1008 = 0; + $_etype1011 = 0; + $xfer += $input->readListBegin($_etype1011, $_size1008); + for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012) { + $elem1013 = null; + $elem1013 = new \metastore\Function(); + $xfer += $elem1013->read($input); + $this->functions []= $elem1013; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('functions', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->functions)); - foreach ($this->functions as $iter1000) { - $xfer += $iter1000->write($output); + foreach ($this->functions as $iter1014) { + $xfer += $iter1014->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetDatabaseRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetDatabaseRequest.php index 7827ac92c304..30a29760e0b4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetDatabaseRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetDatabaseRequest.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1064 = 0; - $_etype1067 = 0; - $xfer += $input->readListBegin($_etype1067, $_size1064); - for ($_i1068 = 0; $_i1068 < $_size1064; ++$_i1068) { - $elem1069 = null; - $xfer += $input->readString($elem1069); - $this->processorCapabilities []= $elem1069; + $_size1078 = 0; + $_etype1081 = 0; + $xfer += $input->readListBegin($_etype1081, $_size1078); + for ($_i1082 = 0; $_i1082 < $_size1078; ++$_i1082) { + $elem1083 = null; + $xfer += $input->readString($elem1083); + $this->processorCapabilities []= $elem1083; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1070) { - $xfer += $output->writeString($iter1070); + foreach ($this->processorCapabilities as $iter1084) { + $xfer += $output->writeString($iter1084); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFieldsResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFieldsResponse.php index f1682e6deefe..07053e9b9903 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFieldsResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFieldsResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fields = array(); - $_size1255 = 0; - $_etype1258 = 0; - $xfer += $input->readListBegin($_etype1258, $_size1255); - for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259) { - $elem1260 = null; - $elem1260 = new \metastore\FieldSchema(); - $xfer += $elem1260->read($input); - $this->fields []= $elem1260; + $_size1269 = 0; + $_etype1272 = 0; + $xfer += $input->readListBegin($_etype1272, $_size1269); + for ($_i1273 = 0; $_i1273 < $_size1269; ++$_i1273) { + $elem1274 = null; + $elem1274 = new \metastore\FieldSchema(); + $xfer += $elem1274->read($input); + $this->fields []= $elem1274; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fields', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->fields)); - foreach ($this->fields as $iter1261) { - $xfer += $iter1261->write($output); + foreach ($this->fields as $iter1275) { + $xfer += $iter1275->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprRequest.php index 2ab7ef07e082..99add351ec79 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprRequest.php @@ -105,13 +105,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fileIds = array(); - $_size950 = 0; - $_etype953 = 0; - $xfer += $input->readListBegin($_etype953, $_size950); - for ($_i954 = 0; $_i954 < $_size950; ++$_i954) { - $elem955 = null; - $xfer += $input->readI64($elem955); - $this->fileIds []= $elem955; + $_size964 = 0; + $_etype967 = 0; + $xfer += $input->readListBegin($_etype967, $_size964); + for ($_i968 = 0; $_i968 < $_size964; ++$_i968) { + $elem969 = null; + $xfer += $input->readI64($elem969); + $this->fileIds []= $elem969; } $xfer += $input->readListEnd(); } else { @@ -159,8 +159,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1); $output->writeListBegin(TType::I64, count($this->fileIds)); - foreach ($this->fileIds as $iter956) { - $xfer += $output->writeI64($iter956); + foreach ($this->fileIds as $iter970) { + $xfer += $output->writeI64($iter970); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprResult.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprResult.php index 8a9a9d7941db..d3afb13a316f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprResult.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataByExprResult.php @@ -85,17 +85,17 @@ public function read($input) case 1: if ($ftype == TType::MAP) { $this->metadata = array(); - $_size941 = 0; - $_ktype942 = 0; - $_vtype943 = 0; - $xfer += $input->readMapBegin($_ktype942, $_vtype943, $_size941); - for ($_i945 = 0; $_i945 < $_size941; ++$_i945) { - $key946 = 0; - $val947 = new \metastore\MetadataPpdResult(); - $xfer += $input->readI64($key946); - $val947 = new \metastore\MetadataPpdResult(); - $xfer += $val947->read($input); - $this->metadata[$key946] = $val947; + $_size955 = 0; + $_ktype956 = 0; + $_vtype957 = 0; + $xfer += $input->readMapBegin($_ktype956, $_vtype957, $_size955); + for ($_i959 = 0; $_i959 < $_size955; ++$_i959) { + $key960 = 0; + $val961 = new \metastore\MetadataPpdResult(); + $xfer += $input->readI64($key960); + $val961 = new \metastore\MetadataPpdResult(); + $xfer += $val961->read($input); + $this->metadata[$key960] = $val961; } $xfer += $input->readMapEnd(); } else { @@ -129,9 +129,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('metadata', TType::MAP, 1); $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata)); - foreach ($this->metadata as $kiter948 => $viter949) { - $xfer += $output->writeI64($kiter948); - $xfer += $viter949->write($output); + foreach ($this->metadata as $kiter962 => $viter963) { + $xfer += $output->writeI64($kiter962); + $xfer += $viter963->write($output); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataRequest.php index da6cd385bb05..cf8f3f6a0d54 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataRequest.php @@ -68,13 +68,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fileIds = array(); - $_size966 = 0; - $_etype969 = 0; - $xfer += $input->readListBegin($_etype969, $_size966); - for ($_i970 = 0; $_i970 < $_size966; ++$_i970) { - $elem971 = null; - $xfer += $input->readI64($elem971); - $this->fileIds []= $elem971; + $_size980 = 0; + $_etype983 = 0; + $xfer += $input->readListBegin($_etype983, $_size980); + for ($_i984 = 0; $_i984 < $_size980; ++$_i984) { + $elem985 = null; + $xfer += $input->readI64($elem985); + $this->fileIds []= $elem985; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1); $output->writeListBegin(TType::I64, count($this->fileIds)); - foreach ($this->fileIds as $iter972) { - $xfer += $output->writeI64($iter972); + foreach ($this->fileIds as $iter986) { + $xfer += $output->writeI64($iter986); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataResult.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataResult.php index c2daa4ada5ad..77a704da132a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataResult.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetFileMetadataResult.php @@ -84,16 +84,16 @@ public function read($input) case 1: if ($ftype == TType::MAP) { $this->metadata = array(); - $_size957 = 0; - $_ktype958 = 0; - $_vtype959 = 0; - $xfer += $input->readMapBegin($_ktype958, $_vtype959, $_size957); - for ($_i961 = 0; $_i961 < $_size957; ++$_i961) { - $key962 = 0; - $val963 = ''; - $xfer += $input->readI64($key962); - $xfer += $input->readString($val963); - $this->metadata[$key962] = $val963; + $_size971 = 0; + $_ktype972 = 0; + $_vtype973 = 0; + $xfer += $input->readMapBegin($_ktype972, $_vtype973, $_size971); + for ($_i975 = 0; $_i975 < $_size971; ++$_i975) { + $key976 = 0; + $val977 = ''; + $xfer += $input->readI64($key976); + $xfer += $input->readString($val977); + $this->metadata[$key976] = $val977; } $xfer += $input->readMapEnd(); } else { @@ -127,9 +127,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('metadata', TType::MAP, 1); $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata)); - foreach ($this->metadata as $kiter964 => $viter965) { - $xfer += $output->writeI64($kiter964); - $xfer += $output->writeString($viter965); + foreach ($this->metadata as $kiter978 => $viter979) { + $xfer += $output->writeI64($kiter978); + $xfer += $output->writeString($viter979); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php index 21bc2342e16c..9b2db717d0f4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php @@ -69,13 +69,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->excludeTxnTypes = array(); - $_size1325 = 0; - $_etype1328 = 0; - $xfer += $input->readListBegin($_etype1328, $_size1325); - for ($_i1329 = 0; $_i1329 < $_size1325; ++$_i1329) { - $elem1330 = null; - $xfer += $input->readI32($elem1330); - $this->excludeTxnTypes []= $elem1330; + $_size1339 = 0; + $_etype1342 = 0; + $xfer += $input->readListBegin($_etype1342, $_size1339); + for ($_i1343 = 0; $_i1343 < $_size1339; ++$_i1343) { + $elem1344 = null; + $xfer += $input->readI32($elem1344); + $this->excludeTxnTypes []= $elem1344; } $xfer += $input->readListEnd(); } else { @@ -102,8 +102,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('excludeTxnTypes', TType::LST, 1); $output->writeListBegin(TType::I32, count($this->excludeTxnTypes)); - foreach ($this->excludeTxnTypes as $iter1331) { - $xfer += $output->writeI32($iter1331); + foreach ($this->excludeTxnTypes as $iter1345) { + $xfer += $output->writeI32($iter1345); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsRequest.php index 3f3d930fe257..ec59a5e1956d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsRequest.php @@ -161,13 +161,13 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->partValues = array(); - $_size1283 = 0; - $_etype1286 = 0; - $xfer += $input->readListBegin($_etype1286, $_size1283); - for ($_i1287 = 0; $_i1287 < $_size1283; ++$_i1287) { - $elem1288 = null; - $xfer += $input->readString($elem1288); - $this->partValues []= $elem1288; + $_size1297 = 0; + $_etype1300 = 0; + $xfer += $input->readListBegin($_etype1300, $_size1297); + for ($_i1301 = 0; $_i1301 < $_size1297; ++$_i1301) { + $elem1302 = null; + $xfer += $input->readString($elem1302); + $this->partValues []= $elem1302; } $xfer += $input->readListEnd(); } else { @@ -230,8 +230,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partValues', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->partValues)); - foreach ($this->partValues as $iter1289) { - $xfer += $output->writeString($iter1289); + foreach ($this->partValues as $iter1303) { + $xfer += $output->writeString($iter1303); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsResponse.php index 0e37452d5b03..cc139ed39b36 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionNamesPsResponse.php @@ -68,13 +68,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->names = array(); - $_size1290 = 0; - $_etype1293 = 0; - $xfer += $input->readListBegin($_etype1293, $_size1290); - for ($_i1294 = 0; $_i1294 < $_size1290; ++$_i1294) { - $elem1295 = null; - $xfer += $input->readString($elem1295); - $this->names []= $elem1295; + $_size1304 = 0; + $_etype1307 = 0; + $xfer += $input->readListBegin($_etype1307, $_size1304); + for ($_i1308 = 0; $_i1308 < $_size1304; ++$_i1308) { + $elem1309 = null; + $xfer += $input->readString($elem1309); + $this->names []= $elem1309; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('names', TType::LST, 1); $output->writeListBegin(TType::STRING, count($this->names)); - foreach ($this->names as $iter1296) { - $xfer += $output->writeString($iter1296); + foreach ($this->names as $iter1310) { + $xfer += $output->writeString($iter1310); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionRequest.php index 660ab50126bd..9e1e4f7d7b4f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionRequest.php @@ -149,13 +149,13 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->partVals = array(); - $_size1269 = 0; - $_etype1272 = 0; - $xfer += $input->readListBegin($_etype1272, $_size1269); - for ($_i1273 = 0; $_i1273 < $_size1269; ++$_i1273) { - $elem1274 = null; - $xfer += $input->readString($elem1274); - $this->partVals []= $elem1274; + $_size1283 = 0; + $_etype1286 = 0; + $xfer += $input->readListBegin($_etype1286, $_size1283); + for ($_i1287 = 0; $_i1287 < $_size1283; ++$_i1287) { + $elem1288 = null; + $xfer += $input->readString($elem1288); + $this->partVals []= $elem1288; } $xfer += $input->readListEnd(); } else { @@ -211,8 +211,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partVals', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->partVals)); - foreach ($this->partVals as $iter1275) { - $xfer += $output->writeString($iter1275); + foreach ($this->partVals as $iter1289) { + $xfer += $output->writeString($iter1289); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsFilterSpec.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsFilterSpec.php index 9bd10d58a6f5..4fe105c7e9f7 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsFilterSpec.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsFilterSpec.php @@ -88,13 +88,13 @@ public function read($input) case 8: if ($ftype == TType::LST) { $this->filters = array(); - $_size1227 = 0; - $_etype1230 = 0; - $xfer += $input->readListBegin($_etype1230, $_size1227); - for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231) { - $elem1232 = null; - $xfer += $input->readString($elem1232); - $this->filters []= $elem1232; + $_size1241 = 0; + $_etype1244 = 0; + $xfer += $input->readListBegin($_etype1244, $_size1241); + for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245) { + $elem1246 = null; + $xfer += $input->readString($elem1246); + $this->filters []= $elem1246; } $xfer += $input->readListEnd(); } else { @@ -126,8 +126,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('filters', TType::LST, 8); $output->writeListBegin(TType::STRING, count($this->filters)); - foreach ($this->filters as $iter1233) { - $xfer += $output->writeString($iter1233); + foreach ($this->filters as $iter1247) { + $xfer += $output->writeString($iter1247); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php index 6124f3b78703..0eecd2f5ad9b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php @@ -201,13 +201,13 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->partVals = array(); - $_size1297 = 0; - $_etype1300 = 0; - $xfer += $input->readListBegin($_etype1300, $_size1297); - for ($_i1301 = 0; $_i1301 < $_size1297; ++$_i1301) { - $elem1302 = null; - $xfer += $input->readString($elem1302); - $this->partVals []= $elem1302; + $_size1311 = 0; + $_etype1314 = 0; + $xfer += $input->readListBegin($_etype1314, $_size1311); + for ($_i1315 = 0; $_i1315 < $_size1311; ++$_i1315) { + $elem1316 = null; + $xfer += $input->readString($elem1316); + $this->partVals []= $elem1316; } $xfer += $input->readListEnd(); } else { @@ -231,13 +231,13 @@ public function read($input) case 7: if ($ftype == TType::LST) { $this->groupNames = array(); - $_size1303 = 0; - $_etype1306 = 0; - $xfer += $input->readListBegin($_etype1306, $_size1303); - for ($_i1307 = 0; $_i1307 < $_size1303; ++$_i1307) { - $elem1308 = null; - $xfer += $input->readString($elem1308); - $this->groupNames []= $elem1308; + $_size1317 = 0; + $_etype1320 = 0; + $xfer += $input->readListBegin($_etype1320, $_size1317); + for ($_i1321 = 0; $_i1321 < $_size1317; ++$_i1321) { + $elem1322 = null; + $xfer += $input->readString($elem1322); + $this->groupNames []= $elem1322; } $xfer += $input->readListEnd(); } else { @@ -300,8 +300,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partVals', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->partVals)); - foreach ($this->partVals as $iter1309) { - $xfer += $output->writeString($iter1309); + foreach ($this->partVals as $iter1323) { + $xfer += $output->writeString($iter1323); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -322,8 +322,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('groupNames', TType::LST, 7); $output->writeListBegin(TType::STRING, count($this->groupNames)); - foreach ($this->groupNames as $iter1310) { - $xfer += $output->writeString($iter1310); + foreach ($this->groupNames as $iter1324) { + $xfer += $output->writeString($iter1324); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php index bd8d345c0f04..3c8a7092cd90 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->partitions = array(); - $_size1311 = 0; - $_etype1314 = 0; - $xfer += $input->readListBegin($_etype1314, $_size1311); - for ($_i1315 = 0; $_i1315 < $_size1311; ++$_i1315) { - $elem1316 = null; - $elem1316 = new \metastore\Partition(); - $xfer += $elem1316->read($input); - $this->partitions []= $elem1316; + $_size1325 = 0; + $_etype1328 = 0; + $xfer += $input->readListBegin($_etype1328, $_size1325); + for ($_i1329 = 0; $_i1329 < $_size1325; ++$_i1329) { + $elem1330 = null; + $elem1330 = new \metastore\Partition(); + $xfer += $elem1330->read($input); + $this->partitions []= $elem1330; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitions', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->partitions)); - foreach ($this->partitions as $iter1317) { - $xfer += $iter1317->write($output); + foreach ($this->partitions as $iter1331) { + $xfer += $iter1331->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsRequest.php index 12e0d091c61f..1cbcd00c0cdf 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsRequest.php @@ -229,13 +229,13 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->groupNames = array(); - $_size1241 = 0; - $_etype1244 = 0; - $xfer += $input->readListBegin($_etype1244, $_size1241); - for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245) { - $elem1246 = null; - $xfer += $input->readString($elem1246); - $this->groupNames []= $elem1246; + $_size1255 = 0; + $_etype1258 = 0; + $xfer += $input->readListBegin($_etype1258, $_size1255); + for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259) { + $elem1260 = null; + $xfer += $input->readString($elem1260); + $this->groupNames []= $elem1260; } $xfer += $input->readListEnd(); } else { @@ -261,13 +261,13 @@ public function read($input) case 9: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1247 = 0; - $_etype1250 = 0; - $xfer += $input->readListBegin($_etype1250, $_size1247); - for ($_i1251 = 0; $_i1251 < $_size1247; ++$_i1251) { - $elem1252 = null; - $xfer += $input->readString($elem1252); - $this->processorCapabilities []= $elem1252; + $_size1261 = 0; + $_etype1264 = 0; + $xfer += $input->readListBegin($_etype1264, $_size1261); + for ($_i1265 = 0; $_i1265 < $_size1261; ++$_i1265) { + $elem1266 = null; + $xfer += $input->readString($elem1266); + $this->processorCapabilities []= $elem1266; } $xfer += $input->readListEnd(); } else { @@ -333,8 +333,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('groupNames', TType::LST, 6); $output->writeListBegin(TType::STRING, count($this->groupNames)); - foreach ($this->groupNames as $iter1253) { - $xfer += $output->writeString($iter1253); + foreach ($this->groupNames as $iter1267) { + $xfer += $output->writeString($iter1267); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -361,8 +361,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 9); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1254) { - $xfer += $output->writeString($iter1254); + foreach ($this->processorCapabilities as $iter1268) { + $xfer += $output->writeString($iter1268); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsResponse.php index 40e7db7d6d88..eaba2ac85e38 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->partitionSpec = array(); - $_size1234 = 0; - $_etype1237 = 0; - $xfer += $input->readListBegin($_etype1237, $_size1234); - for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238) { - $elem1239 = null; - $elem1239 = new \metastore\PartitionSpec(); - $xfer += $elem1239->read($input); - $this->partitionSpec []= $elem1239; + $_size1248 = 0; + $_etype1251 = 0; + $xfer += $input->readListBegin($_etype1251, $_size1248); + for ($_i1252 = 0; $_i1252 < $_size1248; ++$_i1252) { + $elem1253 = null; + $elem1253 = new \metastore\PartitionSpec(); + $xfer += $elem1253->read($input); + $this->partitionSpec []= $elem1253; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionSpec', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->partitionSpec)); - foreach ($this->partitionSpec as $iter1240) { - $xfer += $iter1240->write($output); + foreach ($this->partitionSpec as $iter1254) { + $xfer += $iter1254->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetProjectionsSpec.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetProjectionsSpec.php index 4dea1a0be88e..84bff29c58c0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetProjectionsSpec.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetProjectionsSpec.php @@ -92,13 +92,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fieldList = array(); - $_size1008 = 0; - $_etype1011 = 0; - $xfer += $input->readListBegin($_etype1011, $_size1008); - for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012) { - $elem1013 = null; - $xfer += $input->readString($elem1013); - $this->fieldList []= $elem1013; + $_size1022 = 0; + $_etype1025 = 0; + $xfer += $input->readListBegin($_etype1025, $_size1022); + for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026) { + $elem1027 = null; + $xfer += $input->readString($elem1027); + $this->fieldList []= $elem1027; } $xfer += $input->readListEnd(); } else { @@ -139,8 +139,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fieldList', TType::LST, 1); $output->writeListBegin(TType::STRING, count($this->fieldList)); - foreach ($this->fieldList as $iter1014) { - $xfer += $output->writeString($iter1014); + foreach ($this->fieldList as $iter1028) { + $xfer += $output->writeString($iter1028); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetSchemaResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetSchemaResponse.php index cbaa5ad0f7ea..b32d8d023ec6 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetSchemaResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetSchemaResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fields = array(); - $_size1262 = 0; - $_etype1265 = 0; - $xfer += $input->readListBegin($_etype1265, $_size1262); - for ($_i1266 = 0; $_i1266 < $_size1262; ++$_i1266) { - $elem1267 = null; - $elem1267 = new \metastore\FieldSchema(); - $xfer += $elem1267->read($input); - $this->fields []= $elem1267; + $_size1276 = 0; + $_etype1279 = 0; + $xfer += $input->readListBegin($_etype1279, $_size1276); + for ($_i1280 = 0; $_i1280 < $_size1276; ++$_i1280) { + $elem1281 = null; + $elem1281 = new \metastore\FieldSchema(); + $xfer += $elem1281->read($input); + $this->fields []= $elem1281; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fields', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->fields)); - foreach ($this->fields as $iter1268) { - $xfer += $iter1268->write($output); + foreach ($this->fields as $iter1282) { + $xfer += $iter1282->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php index c770be70b20e..3c56331bb0f5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php @@ -220,13 +220,13 @@ public function read($input) case 8: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1015 = 0; - $_etype1018 = 0; - $xfer += $input->readListBegin($_etype1018, $_size1015); - for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019) { - $elem1020 = null; - $xfer += $input->readString($elem1020); - $this->processorCapabilities []= $elem1020; + $_size1029 = 0; + $_etype1032 = 0; + $xfer += $input->readListBegin($_etype1032, $_size1029); + for ($_i1033 = 0; $_i1033 < $_size1029; ++$_i1033) { + $elem1034 = null; + $xfer += $input->readString($elem1034); + $this->processorCapabilities []= $elem1034; } $xfer += $input->readListEnd(); } else { @@ -307,8 +307,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 8); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1021) { - $xfer += $output->writeString($iter1021); + foreach ($this->processorCapabilities as $iter1035) { + $xfer += $output->writeString($iter1035); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesExtRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesExtRequest.php index eab97b73f409..cfe2d8bf4340 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesExtRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesExtRequest.php @@ -175,13 +175,13 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1043 = 0; - $_etype1046 = 0; - $xfer += $input->readListBegin($_etype1046, $_size1043); - for ($_i1047 = 0; $_i1047 < $_size1043; ++$_i1047) { - $elem1048 = null; - $xfer += $input->readString($elem1048); - $this->processorCapabilities []= $elem1048; + $_size1057 = 0; + $_etype1060 = 0; + $xfer += $input->readListBegin($_etype1060, $_size1057); + for ($_i1061 = 0; $_i1061 < $_size1057; ++$_i1061) { + $elem1062 = null; + $xfer += $input->readString($elem1062); + $this->processorCapabilities []= $elem1062; } $xfer += $input->readListEnd(); } else { @@ -240,8 +240,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 6); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1049) { - $xfer += $output->writeString($iter1049); + foreach ($this->processorCapabilities as $iter1063) { + $xfer += $output->writeString($iter1063); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesRequest.php index 2332929cd5ea..54fce8b1c087 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesRequest.php @@ -165,13 +165,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->tblNames = array(); - $_size1022 = 0; - $_etype1025 = 0; - $xfer += $input->readListBegin($_etype1025, $_size1022); - for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026) { - $elem1027 = null; - $xfer += $input->readString($elem1027); - $this->tblNames []= $elem1027; + $_size1036 = 0; + $_etype1039 = 0; + $xfer += $input->readListBegin($_etype1039, $_size1036); + for ($_i1040 = 0; $_i1040 < $_size1036; ++$_i1040) { + $elem1041 = null; + $xfer += $input->readString($elem1041); + $this->tblNames []= $elem1041; } $xfer += $input->readListEnd(); } else { @@ -196,13 +196,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->processorCapabilities = array(); - $_size1028 = 0; - $_etype1031 = 0; - $xfer += $input->readListBegin($_etype1031, $_size1028); - for ($_i1032 = 0; $_i1032 < $_size1028; ++$_i1032) { - $elem1033 = null; - $xfer += $input->readString($elem1033); - $this->processorCapabilities []= $elem1033; + $_size1042 = 0; + $_etype1045 = 0; + $xfer += $input->readListBegin($_etype1045, $_size1042); + for ($_i1046 = 0; $_i1046 < $_size1042; ++$_i1046) { + $elem1047 = null; + $xfer += $input->readString($elem1047); + $this->processorCapabilities []= $elem1047; } $xfer += $input->readListEnd(); } else { @@ -256,8 +256,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('tblNames', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->tblNames)); - foreach ($this->tblNames as $iter1034) { - $xfer += $output->writeString($iter1034); + foreach ($this->tblNames as $iter1048) { + $xfer += $output->writeString($iter1048); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -281,8 +281,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->processorCapabilities)); - foreach ($this->processorCapabilities as $iter1035) { - $xfer += $output->writeString($iter1035); + foreach ($this->processorCapabilities as $iter1049) { + $xfer += $output->writeString($iter1049); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesResult.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesResult.php index fd10e6913da9..111b96df60ec 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesResult.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTablesResult.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->tables = array(); - $_size1036 = 0; - $_etype1039 = 0; - $xfer += $input->readListBegin($_etype1039, $_size1036); - for ($_i1040 = 0; $_i1040 < $_size1036; ++$_i1040) { - $elem1041 = null; - $elem1041 = new \metastore\Table(); - $xfer += $elem1041->read($input); - $this->tables []= $elem1041; + $_size1050 = 0; + $_etype1053 = 0; + $xfer += $input->readListBegin($_etype1053, $_size1050); + for ($_i1054 = 0; $_i1054 < $_size1050; ++$_i1054) { + $elem1055 = null; + $elem1055 = new \metastore\Table(); + $xfer += $elem1055->read($input); + $this->tables []= $elem1055; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('tables', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->tables)); - foreach ($this->tables as $iter1042) { - $xfer += $iter1042->write($output); + foreach ($this->tables as $iter1056) { + $xfer += $iter1056->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/InsertEventRequestData.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/InsertEventRequestData.php index 62541211358f..503e7d01e8b2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/InsertEventRequestData.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/InsertEventRequestData.php @@ -135,13 +135,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->filesAdded = array(); - $_size869 = 0; - $_etype872 = 0; - $xfer += $input->readListBegin($_etype872, $_size869); - for ($_i873 = 0; $_i873 < $_size869; ++$_i873) { - $elem874 = null; - $xfer += $input->readString($elem874); - $this->filesAdded []= $elem874; + $_size883 = 0; + $_etype886 = 0; + $xfer += $input->readListBegin($_etype886, $_size883); + for ($_i887 = 0; $_i887 < $_size883; ++$_i887) { + $elem888 = null; + $xfer += $input->readString($elem888); + $this->filesAdded []= $elem888; } $xfer += $input->readListEnd(); } else { @@ -151,13 +151,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->filesAddedChecksum = array(); - $_size875 = 0; - $_etype878 = 0; - $xfer += $input->readListBegin($_etype878, $_size875); - for ($_i879 = 0; $_i879 < $_size875; ++$_i879) { - $elem880 = null; - $xfer += $input->readString($elem880); - $this->filesAddedChecksum []= $elem880; + $_size889 = 0; + $_etype892 = 0; + $xfer += $input->readListBegin($_etype892, $_size889); + for ($_i893 = 0; $_i893 < $_size889; ++$_i893) { + $elem894 = null; + $xfer += $input->readString($elem894); + $this->filesAddedChecksum []= $elem894; } $xfer += $input->readListEnd(); } else { @@ -167,13 +167,13 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->subDirectoryList = array(); - $_size881 = 0; - $_etype884 = 0; - $xfer += $input->readListBegin($_etype884, $_size881); - for ($_i885 = 0; $_i885 < $_size881; ++$_i885) { - $elem886 = null; - $xfer += $input->readString($elem886); - $this->subDirectoryList []= $elem886; + $_size895 = 0; + $_etype898 = 0; + $xfer += $input->readListBegin($_etype898, $_size895); + for ($_i899 = 0; $_i899 < $_size895; ++$_i899) { + $elem900 = null; + $xfer += $input->readString($elem900); + $this->subDirectoryList []= $elem900; } $xfer += $input->readListEnd(); } else { @@ -183,13 +183,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->partitionVal = array(); - $_size887 = 0; - $_etype890 = 0; - $xfer += $input->readListBegin($_etype890, $_size887); - for ($_i891 = 0; $_i891 < $_size887; ++$_i891) { - $elem892 = null; - $xfer += $input->readString($elem892); - $this->partitionVal []= $elem892; + $_size901 = 0; + $_etype904 = 0; + $xfer += $input->readListBegin($_etype904, $_size901); + for ($_i905 = 0; $_i905 < $_size901; ++$_i905) { + $elem906 = null; + $xfer += $input->readString($elem906); + $this->partitionVal []= $elem906; } $xfer += $input->readListEnd(); } else { @@ -221,8 +221,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('filesAdded', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->filesAdded)); - foreach ($this->filesAdded as $iter893) { - $xfer += $output->writeString($iter893); + foreach ($this->filesAdded as $iter907) { + $xfer += $output->writeString($iter907); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -233,8 +233,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('filesAddedChecksum', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum)); - foreach ($this->filesAddedChecksum as $iter894) { - $xfer += $output->writeString($iter894); + foreach ($this->filesAddedChecksum as $iter908) { + $xfer += $output->writeString($iter908); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -245,8 +245,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('subDirectoryList', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->subDirectoryList)); - foreach ($this->subDirectoryList as $iter895) { - $xfer += $output->writeString($iter895); + foreach ($this->subDirectoryList as $iter909) { + $xfer += $output->writeString($iter909); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -257,8 +257,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionVal', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->partitionVal)); - foreach ($this->partitionVal as $iter896) { - $xfer += $output->writeString($iter896); + foreach ($this->partitionVal as $iter910) { + $xfer += $output->writeString($iter910); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventRequest.php index e900c7feccd0..b669b3e00eac 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventRequest.php @@ -40,6 +40,25 @@ class NotificationEventRequest 'type' => TType::STRING, ), ), + 4 => array( + 'var' => 'catName', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'dbName', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 6 => array( + 'var' => 'tableNames', + 'isRequired' => false, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); /** @@ -54,6 +73,18 @@ class NotificationEventRequest * @var string[] */ public $eventTypeSkipList = null; + /** + * @var string + */ + public $catName = null; + /** + * @var string + */ + public $dbName = null; + /** + * @var string[] + */ + public $tableNames = null; public function __construct($vals = null) { @@ -67,6 +98,15 @@ public function __construct($vals = null) if (isset($vals['eventTypeSkipList'])) { $this->eventTypeSkipList = $vals['eventTypeSkipList']; } + if (isset($vals['catName'])) { + $this->catName = $vals['catName']; + } + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; + } + if (isset($vals['tableNames'])) { + $this->tableNames = $vals['tableNames']; + } } } @@ -119,6 +159,36 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->catName); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbName); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::LST) { + $this->tableNames = array(); + $_size861 = 0; + $_etype864 = 0; + $xfer += $input->readListBegin($_etype864, $_size861); + for ($_i865 = 0; $_i865 < $_size861; ++$_i865) { + $elem866 = null; + $xfer += $input->readString($elem866); + $this->tableNames []= $elem866; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -149,8 +219,30 @@ public function write($output) } $xfer += $output->writeFieldBegin('eventTypeSkipList', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->eventTypeSkipList)); - foreach ($this->eventTypeSkipList as $iter861) { - $xfer += $output->writeString($iter861); + foreach ($this->eventTypeSkipList as $iter867) { + $xfer += $output->writeString($iter867); + } + $output->writeListEnd(); + $xfer += $output->writeFieldEnd(); + } + if ($this->catName !== null) { + $xfer += $output->writeFieldBegin('catName', TType::STRING, 4); + $xfer += $output->writeString($this->catName); + $xfer += $output->writeFieldEnd(); + } + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 5); + $xfer += $output->writeString($this->dbName); + $xfer += $output->writeFieldEnd(); + } + if ($this->tableNames !== null) { + if (!is_array($this->tableNames)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('tableNames', TType::LST, 6); + $output->writeListBegin(TType::STRING, count($this->tableNames)); + foreach ($this->tableNames as $iter868) { + $xfer += $output->writeString($iter868); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventResponse.php index b36613e42e8e..cbcc6f85cc9d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->events = array(); - $_size862 = 0; - $_etype865 = 0; - $xfer += $input->readListBegin($_etype865, $_size862); - for ($_i866 = 0; $_i866 < $_size862; ++$_i866) { - $elem867 = null; - $elem867 = new \metastore\NotificationEvent(); - $xfer += $elem867->read($input); - $this->events []= $elem867; + $_size869 = 0; + $_etype872 = 0; + $xfer += $input->readListBegin($_etype872, $_size869); + for ($_i873 = 0; $_i873 < $_size869; ++$_i873) { + $elem874 = null; + $elem874 = new \metastore\NotificationEvent(); + $xfer += $elem874->read($input); + $this->events []= $elem874; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('events', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->events)); - foreach ($this->events as $iter868) { - $xfer += $iter868->write($output); + foreach ($this->events as $iter875) { + $xfer += $iter875->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventsCountRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventsCountRequest.php index 754d289877a4..76c8a0427840 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventsCountRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/NotificationEventsCountRequest.php @@ -46,6 +46,15 @@ class NotificationEventsCountRequest 'isRequired' => false, 'type' => TType::I64, ), + 6 => array( + 'var' => 'tableNames', + 'isRequired' => false, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); /** @@ -68,6 +77,10 @@ class NotificationEventsCountRequest * @var int */ public $limit = null; + /** + * @var string[] + */ + public $tableNames = null; public function __construct($vals = null) { @@ -87,6 +100,9 @@ public function __construct($vals = null) if (isset($vals['limit'])) { $this->limit = $vals['limit']; } + if (isset($vals['tableNames'])) { + $this->tableNames = $vals['tableNames']; + } } } @@ -144,6 +160,22 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 6: + if ($ftype == TType::LST) { + $this->tableNames = array(); + $_size876 = 0; + $_etype879 = 0; + $xfer += $input->readListBegin($_etype879, $_size876); + for ($_i880 = 0; $_i880 < $_size876; ++$_i880) { + $elem881 = null; + $xfer += $input->readString($elem881); + $this->tableNames []= $elem881; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -183,6 +215,18 @@ public function write($output) $xfer += $output->writeI64($this->limit); $xfer += $output->writeFieldEnd(); } + if ($this->tableNames !== null) { + if (!is_array($this->tableNames)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('tableNames', TType::LST, 6); + $output->writeListBegin(TType::STRING, count($this->tableNames)); + foreach ($this->tableNames as $iter882) { + $xfer += $output->writeString($iter882); + } + $output->writeListEnd(); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsResponse.php index 5a806f2e504d..4a84cd7cf8c8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->partitions = array(); - $_size1276 = 0; - $_etype1279 = 0; - $xfer += $input->readListBegin($_etype1279, $_size1276); - for ($_i1280 = 0; $_i1280 < $_size1276; ++$_i1280) { - $elem1281 = null; - $elem1281 = new \metastore\Partition(); - $xfer += $elem1281->read($input); - $this->partitions []= $elem1281; + $_size1290 = 0; + $_etype1293 = 0; + $xfer += $input->readListBegin($_etype1293, $_size1290); + for ($_i1294 = 0; $_i1294 < $_size1290; ++$_i1294) { + $elem1295 = null; + $elem1295 = new \metastore\Partition(); + $xfer += $elem1295->read($input); + $this->partitions []= $elem1295; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitions', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->partitions)); - foreach ($this->partitions as $iter1282) { - $xfer += $iter1282->write($output); + foreach ($this->partitions as $iter1296) { + $xfer += $iter1296->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PutFileMetadataRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PutFileMetadataRequest.php index 487d402e5439..040ad04c3f08 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PutFileMetadataRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PutFileMetadataRequest.php @@ -97,13 +97,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->fileIds = array(); - $_size973 = 0; - $_etype976 = 0; - $xfer += $input->readListBegin($_etype976, $_size973); - for ($_i977 = 0; $_i977 < $_size973; ++$_i977) { - $elem978 = null; - $xfer += $input->readI64($elem978); - $this->fileIds []= $elem978; + $_size987 = 0; + $_etype990 = 0; + $xfer += $input->readListBegin($_etype990, $_size987); + for ($_i991 = 0; $_i991 < $_size987; ++$_i991) { + $elem992 = null; + $xfer += $input->readI64($elem992); + $this->fileIds []= $elem992; } $xfer += $input->readListEnd(); } else { @@ -113,13 +113,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->metadata = array(); - $_size979 = 0; - $_etype982 = 0; - $xfer += $input->readListBegin($_etype982, $_size979); - for ($_i983 = 0; $_i983 < $_size979; ++$_i983) { - $elem984 = null; - $xfer += $input->readString($elem984); - $this->metadata []= $elem984; + $_size993 = 0; + $_etype996 = 0; + $xfer += $input->readListBegin($_etype996, $_size993); + for ($_i997 = 0; $_i997 < $_size993; ++$_i997) { + $elem998 = null; + $xfer += $input->readString($elem998); + $this->metadata []= $elem998; } $xfer += $input->readListEnd(); } else { @@ -153,8 +153,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('fileIds', TType::LST, 1); $output->writeListBegin(TType::I64, count($this->fileIds)); - foreach ($this->fileIds as $iter985) { - $xfer += $output->writeI64($iter985); + foreach ($this->fileIds as $iter999) { + $xfer += $output->writeI64($iter999); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -165,8 +165,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('metadata', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->metadata)); - foreach ($this->metadata as $iter986) { - $xfer += $output->writeString($iter986); + foreach ($this->metadata as $iter1000) { + $xfer += $output->writeString($iter1000); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/RenamePartitionRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/RenamePartitionRequest.php index e26296e170ff..0cf258782a98 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/RenamePartitionRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/RenamePartitionRequest.php @@ -174,13 +174,13 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->partVals = array(); - $_size1213 = 0; - $_etype1216 = 0; - $xfer += $input->readListBegin($_etype1216, $_size1213); - for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217) { - $elem1218 = null; - $xfer += $input->readString($elem1218); - $this->partVals []= $elem1218; + $_size1227 = 0; + $_etype1230 = 0; + $xfer += $input->readListBegin($_etype1230, $_size1227); + for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231) { + $elem1232 = null; + $xfer += $input->readString($elem1232); + $this->partVals []= $elem1232; } $xfer += $input->readListEnd(); } else { @@ -251,8 +251,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partVals', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->partVals)); - foreach ($this->partVals as $iter1219) { - $xfer += $output->writeString($iter1219); + foreach ($this->partVals as $iter1233) { + $xfer += $output->writeString($iter1233); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php index e368d6ca7259..5c136ac01a8c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->replicationMetricList = array(); - $_size1318 = 0; - $_etype1321 = 0; - $xfer += $input->readListBegin($_etype1321, $_size1318); - for ($_i1322 = 0; $_i1322 < $_size1318; ++$_i1322) { - $elem1323 = null; - $elem1323 = new \metastore\ReplicationMetrics(); - $xfer += $elem1323->read($input); - $this->replicationMetricList []= $elem1323; + $_size1332 = 0; + $_etype1335 = 0; + $xfer += $input->readListBegin($_etype1335, $_size1332); + for ($_i1336 = 0; $_i1336 < $_size1332; ++$_i1336) { + $elem1337 = null; + $elem1337 = new \metastore\ReplicationMetrics(); + $xfer += $elem1337->read($input); + $this->replicationMetricList []= $elem1337; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('replicationMetricList', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->replicationMetricList)); - foreach ($this->replicationMetricList as $iter1324) { - $xfer += $iter1324->write($output); + foreach ($this->replicationMetricList as $iter1338) { + $xfer += $iter1338->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SchemaVersion.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SchemaVersion.php index 4375236cc7c9..7af298afa29d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SchemaVersion.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SchemaVersion.php @@ -202,14 +202,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->cols = array(); - $_size1127 = 0; - $_etype1130 = 0; - $xfer += $input->readListBegin($_etype1130, $_size1127); - for ($_i1131 = 0; $_i1131 < $_size1127; ++$_i1131) { - $elem1132 = null; - $elem1132 = new \metastore\FieldSchema(); - $xfer += $elem1132->read($input); - $this->cols []= $elem1132; + $_size1141 = 0; + $_etype1144 = 0; + $xfer += $input->readListBegin($_etype1144, $_size1141); + for ($_i1145 = 0; $_i1145 < $_size1141; ++$_i1145) { + $elem1146 = null; + $elem1146 = new \metastore\FieldSchema(); + $xfer += $elem1146->read($input); + $this->cols []= $elem1146; } $xfer += $input->readListEnd(); } else { @@ -297,8 +297,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('cols', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->cols)); - foreach ($this->cols as $iter1133) { - $xfer += $iter1133->write($output); + foreach ($this->cols as $iter1147) { + $xfer += $iter1147->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php index dec3b3349dc0..a32966ffdc6f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1516 = 0; - $_etype1519 = 0; - $xfer += $input->readListBegin($_etype1519, $_size1516); - for ($_i1520 = 0; $_i1520 < $_size1516; ++$_i1520) { - $elem1521 = null; - $elem1521 = new \metastore\Partition(); - $xfer += $elem1521->read($input); - $this->new_parts []= $elem1521; + $_size1530 = 0; + $_etype1533 = 0; + $xfer += $input->readListBegin($_etype1533, $_size1530); + for ($_i1534 = 0; $_i1534 < $_size1530; ++$_i1534) { + $elem1535 = null; + $elem1535 = new \metastore\Partition(); + $xfer += $elem1535->read($input); + $this->new_parts []= $elem1535; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1522) { - $xfer += $iter1522->write($output); + foreach ($this->new_parts as $iter1536) { + $xfer += $iter1536->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php index c2899d370379..2bbda0ca9aee 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1523 = 0; - $_etype1526 = 0; - $xfer += $input->readListBegin($_etype1526, $_size1523); - for ($_i1527 = 0; $_i1527 < $_size1523; ++$_i1527) { - $elem1528 = null; - $elem1528 = new \metastore\PartitionSpec(); - $xfer += $elem1528->read($input); - $this->new_parts []= $elem1528; + $_size1537 = 0; + $_etype1540 = 0; + $xfer += $input->readListBegin($_etype1540, $_size1537); + for ($_i1541 = 0; $_i1541 < $_size1537; ++$_i1541) { + $elem1542 = null; + $elem1542 = new \metastore\PartitionSpec(); + $xfer += $elem1542->read($input); + $this->new_parts []= $elem1542; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1529) { - $xfer += $iter1529->write($output); + foreach ($this->new_parts as $iter1543) { + $xfer += $iter1543->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php index aa566de0dcf8..8589337a8431 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php @@ -91,16 +91,16 @@ public function read($input) case 2: if ($ftype == TType::MAP) { $this->writeIds = array(); - $_size1855 = 0; - $_ktype1856 = 0; - $_vtype1857 = 0; - $xfer += $input->readMapBegin($_ktype1856, $_vtype1857, $_size1855); - for ($_i1859 = 0; $_i1859 < $_size1855; ++$_i1859) { - $key1860 = ''; - $val1861 = 0; - $xfer += $input->readString($key1860); - $xfer += $input->readI64($val1861); - $this->writeIds[$key1860] = $val1861; + $_size1869 = 0; + $_ktype1870 = 0; + $_vtype1871 = 0; + $xfer += $input->readMapBegin($_ktype1870, $_vtype1871, $_size1869); + for ($_i1873 = 0; $_i1873 < $_size1869; ++$_i1873) { + $key1874 = ''; + $val1875 = 0; + $xfer += $input->readString($key1874); + $xfer += $input->readI64($val1875); + $this->writeIds[$key1874] = $val1875; } $xfer += $input->readMapEnd(); } else { @@ -132,9 +132,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('writeIds', TType::MAP, 2); $output->writeMapBegin(TType::STRING, TType::I64, count($this->writeIds)); - foreach ($this->writeIds as $kiter1862 => $viter1863) { - $xfer += $output->writeString($kiter1862); - $xfer += $output->writeI64($viter1863); + foreach ($this->writeIds as $kiter1876 => $viter1877) { + $xfer += $output->writeString($kiter1876); + $xfer += $output->writeI64($viter1877); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php index 4f246d7a7262..2dae7ba57dcd 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php @@ -107,14 +107,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1730 = 0; - $_etype1733 = 0; - $xfer += $input->readListBegin($_etype1733, $_size1730); - for ($_i1734 = 0; $_i1734 < $_size1730; ++$_i1734) { - $elem1735 = null; - $elem1735 = new \metastore\Partition(); - $xfer += $elem1735->read($input); - $this->new_parts []= $elem1735; + $_size1744 = 0; + $_etype1747 = 0; + $xfer += $input->readListBegin($_etype1747, $_size1744); + for ($_i1748 = 0; $_i1748 < $_size1744; ++$_i1748) { + $elem1749 = null; + $elem1749 = new \metastore\Partition(); + $xfer += $elem1749->read($input); + $this->new_parts []= $elem1749; } $xfer += $input->readListEnd(); } else { @@ -151,8 +151,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1736) { - $xfer += $iter1736->write($output); + foreach ($this->new_parts as $iter1750) { + $xfer += $iter1750->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php index 2c44b8463f41..b8129f56cb9d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php @@ -120,14 +120,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1737 = 0; - $_etype1740 = 0; - $xfer += $input->readListBegin($_etype1740, $_size1737); - for ($_i1741 = 0; $_i1741 < $_size1737; ++$_i1741) { - $elem1742 = null; - $elem1742 = new \metastore\Partition(); - $xfer += $elem1742->read($input); - $this->new_parts []= $elem1742; + $_size1751 = 0; + $_etype1754 = 0; + $xfer += $input->readListBegin($_etype1754, $_size1751); + for ($_i1755 = 0; $_i1755 < $_size1751; ++$_i1755) { + $elem1756 = null; + $elem1756 = new \metastore\Partition(); + $xfer += $elem1756->read($input); + $this->new_parts []= $elem1756; } $xfer += $input->readListEnd(); } else { @@ -172,8 +172,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1743) { - $xfer += $iter1743->write($output); + foreach ($this->new_parts as $iter1757) { + $xfer += $iter1757->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php index 5127ae7988d0..f20598fc6701 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1530 = 0; - $_etype1533 = 0; - $xfer += $input->readListBegin($_etype1533, $_size1530); - for ($_i1534 = 0; $_i1534 < $_size1530; ++$_i1534) { - $elem1535 = null; - $xfer += $input->readString($elem1535); - $this->part_vals []= $elem1535; + $_size1544 = 0; + $_etype1547 = 0; + $xfer += $input->readListBegin($_etype1547, $_size1544); + for ($_i1548 = 0; $_i1548 < $_size1544; ++$_i1548) { + $elem1549 = null; + $xfer += $input->readString($elem1549); + $this->part_vals []= $elem1549; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1536) { - $xfer += $output->writeString($iter1536); + foreach ($this->part_vals as $iter1550) { + $xfer += $output->writeString($iter1550); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php index f04edaa12bd2..b2e1ffbe9e34 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php @@ -119,13 +119,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1537 = 0; - $_etype1540 = 0; - $xfer += $input->readListBegin($_etype1540, $_size1537); - for ($_i1541 = 0; $_i1541 < $_size1537; ++$_i1541) { - $elem1542 = null; - $xfer += $input->readString($elem1542); - $this->part_vals []= $elem1542; + $_size1551 = 0; + $_etype1554 = 0; + $xfer += $input->readListBegin($_etype1554, $_size1551); + for ($_i1555 = 0; $_i1555 < $_size1551; ++$_i1555) { + $elem1556 = null; + $xfer += $input->readString($elem1556); + $this->part_vals []= $elem1556; } $xfer += $input->readListEnd(); } else { @@ -170,8 +170,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1543) { - $xfer += $output->writeString($iter1543); + foreach ($this->part_vals as $iter1557) { + $xfer += $output->writeString($iter1557); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php index bfad05f3e4ed..6a77f79145d1 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php @@ -175,14 +175,14 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->primaryKeys = array(); - $_size1390 = 0; - $_etype1393 = 0; - $xfer += $input->readListBegin($_etype1393, $_size1390); - for ($_i1394 = 0; $_i1394 < $_size1390; ++$_i1394) { - $elem1395 = null; - $elem1395 = new \metastore\SQLPrimaryKey(); - $xfer += $elem1395->read($input); - $this->primaryKeys []= $elem1395; + $_size1404 = 0; + $_etype1407 = 0; + $xfer += $input->readListBegin($_etype1407, $_size1404); + for ($_i1408 = 0; $_i1408 < $_size1404; ++$_i1408) { + $elem1409 = null; + $elem1409 = new \metastore\SQLPrimaryKey(); + $xfer += $elem1409->read($input); + $this->primaryKeys []= $elem1409; } $xfer += $input->readListEnd(); } else { @@ -192,14 +192,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->foreignKeys = array(); - $_size1396 = 0; - $_etype1399 = 0; - $xfer += $input->readListBegin($_etype1399, $_size1396); - for ($_i1400 = 0; $_i1400 < $_size1396; ++$_i1400) { - $elem1401 = null; - $elem1401 = new \metastore\SQLForeignKey(); - $xfer += $elem1401->read($input); - $this->foreignKeys []= $elem1401; + $_size1410 = 0; + $_etype1413 = 0; + $xfer += $input->readListBegin($_etype1413, $_size1410); + for ($_i1414 = 0; $_i1414 < $_size1410; ++$_i1414) { + $elem1415 = null; + $elem1415 = new \metastore\SQLForeignKey(); + $xfer += $elem1415->read($input); + $this->foreignKeys []= $elem1415; } $xfer += $input->readListEnd(); } else { @@ -209,14 +209,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->uniqueConstraints = array(); - $_size1402 = 0; - $_etype1405 = 0; - $xfer += $input->readListBegin($_etype1405, $_size1402); - for ($_i1406 = 0; $_i1406 < $_size1402; ++$_i1406) { - $elem1407 = null; - $elem1407 = new \metastore\SQLUniqueConstraint(); - $xfer += $elem1407->read($input); - $this->uniqueConstraints []= $elem1407; + $_size1416 = 0; + $_etype1419 = 0; + $xfer += $input->readListBegin($_etype1419, $_size1416); + for ($_i1420 = 0; $_i1420 < $_size1416; ++$_i1420) { + $elem1421 = null; + $elem1421 = new \metastore\SQLUniqueConstraint(); + $xfer += $elem1421->read($input); + $this->uniqueConstraints []= $elem1421; } $xfer += $input->readListEnd(); } else { @@ -226,14 +226,14 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->notNullConstraints = array(); - $_size1408 = 0; - $_etype1411 = 0; - $xfer += $input->readListBegin($_etype1411, $_size1408); - for ($_i1412 = 0; $_i1412 < $_size1408; ++$_i1412) { - $elem1413 = null; - $elem1413 = new \metastore\SQLNotNullConstraint(); - $xfer += $elem1413->read($input); - $this->notNullConstraints []= $elem1413; + $_size1422 = 0; + $_etype1425 = 0; + $xfer += $input->readListBegin($_etype1425, $_size1422); + for ($_i1426 = 0; $_i1426 < $_size1422; ++$_i1426) { + $elem1427 = null; + $elem1427 = new \metastore\SQLNotNullConstraint(); + $xfer += $elem1427->read($input); + $this->notNullConstraints []= $elem1427; } $xfer += $input->readListEnd(); } else { @@ -243,14 +243,14 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->defaultConstraints = array(); - $_size1414 = 0; - $_etype1417 = 0; - $xfer += $input->readListBegin($_etype1417, $_size1414); - for ($_i1418 = 0; $_i1418 < $_size1414; ++$_i1418) { - $elem1419 = null; - $elem1419 = new \metastore\SQLDefaultConstraint(); - $xfer += $elem1419->read($input); - $this->defaultConstraints []= $elem1419; + $_size1428 = 0; + $_etype1431 = 0; + $xfer += $input->readListBegin($_etype1431, $_size1428); + for ($_i1432 = 0; $_i1432 < $_size1428; ++$_i1432) { + $elem1433 = null; + $elem1433 = new \metastore\SQLDefaultConstraint(); + $xfer += $elem1433->read($input); + $this->defaultConstraints []= $elem1433; } $xfer += $input->readListEnd(); } else { @@ -260,14 +260,14 @@ public function read($input) case 7: if ($ftype == TType::LST) { $this->checkConstraints = array(); - $_size1420 = 0; - $_etype1423 = 0; - $xfer += $input->readListBegin($_etype1423, $_size1420); - for ($_i1424 = 0; $_i1424 < $_size1420; ++$_i1424) { - $elem1425 = null; - $elem1425 = new \metastore\SQLCheckConstraint(); - $xfer += $elem1425->read($input); - $this->checkConstraints []= $elem1425; + $_size1434 = 0; + $_etype1437 = 0; + $xfer += $input->readListBegin($_etype1437, $_size1434); + for ($_i1438 = 0; $_i1438 < $_size1434; ++$_i1438) { + $elem1439 = null; + $elem1439 = new \metastore\SQLCheckConstraint(); + $xfer += $elem1439->read($input); + $this->checkConstraints []= $elem1439; } $xfer += $input->readListEnd(); } else { @@ -302,8 +302,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('primaryKeys', TType::LST, 2); $output->writeListBegin(TType::STRUCT, count($this->primaryKeys)); - foreach ($this->primaryKeys as $iter1426) { - $xfer += $iter1426->write($output); + foreach ($this->primaryKeys as $iter1440) { + $xfer += $iter1440->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -314,8 +314,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('foreignKeys', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->foreignKeys)); - foreach ($this->foreignKeys as $iter1427) { - $xfer += $iter1427->write($output); + foreach ($this->foreignKeys as $iter1441) { + $xfer += $iter1441->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -326,8 +326,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('uniqueConstraints', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints)); - foreach ($this->uniqueConstraints as $iter1428) { - $xfer += $iter1428->write($output); + foreach ($this->uniqueConstraints as $iter1442) { + $xfer += $iter1442->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -338,8 +338,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('notNullConstraints', TType::LST, 5); $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints)); - foreach ($this->notNullConstraints as $iter1429) { - $xfer += $iter1429->write($output); + foreach ($this->notNullConstraints as $iter1443) { + $xfer += $iter1443->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -350,8 +350,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('defaultConstraints', TType::LST, 6); $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints)); - foreach ($this->defaultConstraints as $iter1430) { - $xfer += $iter1430->write($output); + foreach ($this->defaultConstraints as $iter1444) { + $xfer += $iter1444->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -362,8 +362,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('checkConstraints', TType::LST, 7); $output->writeListBegin(TType::STRUCT, count($this->checkConstraints)); - foreach ($this->checkConstraints as $iter1431) { - $xfer += $iter1431->write($output); + foreach ($this->checkConstraints as $iter1445) { + $xfer += $iter1445->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php index b761eaf73667..af48d9f1470d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1544 = 0; - $_etype1547 = 0; - $xfer += $input->readListBegin($_etype1547, $_size1544); - for ($_i1548 = 0; $_i1548 < $_size1544; ++$_i1548) { - $elem1549 = null; - $xfer += $input->readString($elem1549); - $this->part_vals []= $elem1549; + $_size1558 = 0; + $_etype1561 = 0; + $xfer += $input->readListBegin($_etype1561, $_size1558); + for ($_i1562 = 0; $_i1562 < $_size1558; ++$_i1562) { + $elem1563 = null; + $xfer += $input->readString($elem1563); + $this->part_vals []= $elem1563; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1550) { - $xfer += $output->writeString($iter1550); + foreach ($this->part_vals as $iter1564) { + $xfer += $output->writeString($iter1564); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php index 740948f8a76f..6d436e6d5d2f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php @@ -131,13 +131,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1551 = 0; - $_etype1554 = 0; - $xfer += $input->readListBegin($_etype1554, $_size1551); - for ($_i1555 = 0; $_i1555 < $_size1551; ++$_i1555) { - $elem1556 = null; - $xfer += $input->readString($elem1556); - $this->part_vals []= $elem1556; + $_size1565 = 0; + $_etype1568 = 0; + $xfer += $input->readListBegin($_etype1568, $_size1565); + for ($_i1569 = 0; $_i1569 < $_size1565; ++$_i1569) { + $elem1570 = null; + $xfer += $input->readString($elem1570); + $this->part_vals []= $elem1570; } $xfer += $input->readListEnd(); } else { @@ -189,8 +189,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1557) { - $xfer += $output->writeString($iter1557); + foreach ($this->part_vals as $iter1571) { + $xfer += $output->writeString($iter1571); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php index 170b624fab9f..226f156cb6f4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php @@ -120,16 +120,16 @@ public function read($input) case 1: if ($ftype == TType::MAP) { $this->partitionSpecs = array(); - $_size1565 = 0; - $_ktype1566 = 0; - $_vtype1567 = 0; - $xfer += $input->readMapBegin($_ktype1566, $_vtype1567, $_size1565); - for ($_i1569 = 0; $_i1569 < $_size1565; ++$_i1569) { - $key1570 = ''; - $val1571 = ''; - $xfer += $input->readString($key1570); - $xfer += $input->readString($val1571); - $this->partitionSpecs[$key1570] = $val1571; + $_size1579 = 0; + $_ktype1580 = 0; + $_vtype1581 = 0; + $xfer += $input->readMapBegin($_ktype1580, $_vtype1581, $_size1579); + for ($_i1583 = 0; $_i1583 < $_size1579; ++$_i1583) { + $key1584 = ''; + $val1585 = ''; + $xfer += $input->readString($key1584); + $xfer += $input->readString($val1585); + $this->partitionSpecs[$key1584] = $val1585; } $xfer += $input->readMapEnd(); } else { @@ -184,9 +184,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); - foreach ($this->partitionSpecs as $kiter1572 => $viter1573) { - $xfer += $output->writeString($kiter1572); - $xfer += $output->writeString($viter1573); + foreach ($this->partitionSpecs as $kiter1586 => $viter1587) { + $xfer += $output->writeString($kiter1586); + $xfer += $output->writeString($viter1587); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php index ab5891a3dd07..582a0668db2d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php @@ -120,16 +120,16 @@ public function read($input) case 1: if ($ftype == TType::MAP) { $this->partitionSpecs = array(); - $_size1574 = 0; - $_ktype1575 = 0; - $_vtype1576 = 0; - $xfer += $input->readMapBegin($_ktype1575, $_vtype1576, $_size1574); - for ($_i1578 = 0; $_i1578 < $_size1574; ++$_i1578) { - $key1579 = ''; - $val1580 = ''; - $xfer += $input->readString($key1579); - $xfer += $input->readString($val1580); - $this->partitionSpecs[$key1579] = $val1580; + $_size1588 = 0; + $_ktype1589 = 0; + $_vtype1590 = 0; + $xfer += $input->readMapBegin($_ktype1589, $_vtype1590, $_size1588); + for ($_i1592 = 0; $_i1592 < $_size1588; ++$_i1592) { + $key1593 = ''; + $val1594 = ''; + $xfer += $input->readString($key1593); + $xfer += $input->readString($val1594); + $this->partitionSpecs[$key1593] = $val1594; } $xfer += $input->readMapEnd(); } else { @@ -184,9 +184,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); - foreach ($this->partitionSpecs as $kiter1581 => $viter1582) { - $xfer += $output->writeString($kiter1581); - $xfer += $output->writeString($viter1582); + foreach ($this->partitionSpecs as $kiter1595 => $viter1596) { + $xfer += $output->writeString($kiter1595); + $xfer += $output->writeString($viter1596); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php index a0bc458c92ff..aa6f6904c5d6 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php @@ -121,14 +121,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1583 = 0; - $_etype1586 = 0; - $xfer += $input->readListBegin($_etype1586, $_size1583); - for ($_i1587 = 0; $_i1587 < $_size1583; ++$_i1587) { - $elem1588 = null; - $elem1588 = new \metastore\Partition(); - $xfer += $elem1588->read($input); - $this->success []= $elem1588; + $_size1597 = 0; + $_etype1600 = 0; + $xfer += $input->readListBegin($_etype1600, $_size1597); + for ($_i1601 = 0; $_i1601 < $_size1597; ++$_i1601) { + $elem1602 = null; + $elem1602 = new \metastore\Partition(); + $xfer += $elem1602->read($input); + $this->success []= $elem1602; } $xfer += $input->readListEnd(); } else { @@ -187,8 +187,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1589) { - $xfer += $iter1589->write($output); + foreach ($this->success as $iter1603) { + $xfer += $iter1603->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php index 96d8d9b69d9c..bf71f8bad539 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php @@ -68,13 +68,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1864 = 0; - $_etype1867 = 0; - $xfer += $input->readListBegin($_etype1867, $_size1864); - for ($_i1868 = 0; $_i1868 < $_size1864; ++$_i1868) { - $elem1869 = null; - $xfer += $input->readString($elem1869); - $this->success []= $elem1869; + $_size1878 = 0; + $_etype1881 = 0; + $xfer += $input->readListBegin($_etype1881, $_size1878); + for ($_i1882 = 0; $_i1882 < $_size1878; ++$_i1882) { + $elem1883 = null; + $xfer += $input->readString($elem1883); + $this->success []= $elem1883; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1870) { - $xfer += $output->writeString($iter1870); + foreach ($this->success as $iter1884) { + $xfer += $output->writeString($iter1884); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php index c972b6b8106a..e52138309bef 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1339 = 0; - $_etype1342 = 0; - $xfer += $input->readListBegin($_etype1342, $_size1339); - for ($_i1343 = 0; $_i1343 < $_size1339; ++$_i1343) { - $elem1344 = null; - $xfer += $input->readString($elem1344); - $this->success []= $elem1344; + $_size1353 = 0; + $_etype1356 = 0; + $xfer += $input->readListBegin($_etype1356, $_size1353); + for ($_i1357 = 0; $_i1357 < $_size1353; ++$_i1357) { + $elem1358 = null; + $xfer += $input->readString($elem1358); + $this->success []= $elem1358; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1345) { - $xfer += $output->writeString($iter1345); + foreach ($this->success as $iter1359) { + $xfer += $output->writeString($iter1359); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php index fbde6eea4331..464a804f21d9 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1453 = 0; - $_etype1456 = 0; - $xfer += $input->readListBegin($_etype1456, $_size1453); - for ($_i1457 = 0; $_i1457 < $_size1453; ++$_i1457) { - $elem1458 = null; - $elem1458 = new \metastore\Table(); - $xfer += $elem1458->read($input); - $this->success []= $elem1458; + $_size1467 = 0; + $_etype1470 = 0; + $xfer += $input->readListBegin($_etype1470, $_size1467); + for ($_i1471 = 0; $_i1471 < $_size1467; ++$_i1471) { + $elem1472 = null; + $elem1472 = new \metastore\Table(); + $xfer += $elem1472->read($input); + $this->success []= $elem1472; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1459) { - $xfer += $iter1459->write($output); + foreach ($this->success as $iter1473) { + $xfer += $iter1473->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php index fd6383e44cff..c14aeda7dc91 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1892 = 0; - $_etype1895 = 0; - $xfer += $input->readListBegin($_etype1895, $_size1892); - for ($_i1896 = 0; $_i1896 < $_size1892; ++$_i1896) { - $elem1897 = null; - $xfer += $input->readString($elem1897); - $this->success []= $elem1897; + $_size1906 = 0; + $_etype1909 = 0; + $xfer += $input->readListBegin($_etype1909, $_size1906); + for ($_i1910 = 0; $_i1910 < $_size1906; ++$_i1910) { + $elem1911 = null; + $xfer += $input->readString($elem1911); + $this->success []= $elem1911; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1898) { - $xfer += $output->writeString($iter1898); + foreach ($this->success as $iter1912) { + $xfer += $output->writeString($iter1912); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php index 6cf3d16b3266..b6c42bf25aea 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1885 = 0; - $_etype1888 = 0; - $xfer += $input->readListBegin($_etype1888, $_size1885); - for ($_i1889 = 0; $_i1889 < $_size1885; ++$_i1889) { - $elem1890 = null; - $xfer += $input->readString($elem1890); - $this->success []= $elem1890; + $_size1899 = 0; + $_etype1902 = 0; + $xfer += $input->readListBegin($_etype1902, $_size1899); + for ($_i1903 = 0; $_i1903 < $_size1899; ++$_i1903) { + $elem1904 = null; + $xfer += $input->readString($elem1904); + $this->success []= $elem1904; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1891) { - $xfer += $output->writeString($iter1891); + foreach ($this->success as $iter1905) { + $xfer += $output->writeString($iter1905); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php index 1d879e447cda..8a9a245ee734 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1481 = 0; - $_etype1484 = 0; - $xfer += $input->readListBegin($_etype1484, $_size1481); - for ($_i1485 = 0; $_i1485 < $_size1481; ++$_i1485) { - $elem1486 = null; - $xfer += $input->readString($elem1486); - $this->success []= $elem1486; + $_size1495 = 0; + $_etype1498 = 0; + $xfer += $input->readListBegin($_etype1498, $_size1495); + for ($_i1499 = 0; $_i1499 < $_size1495; ++$_i1499) { + $elem1500 = null; + $xfer += $input->readString($elem1500); + $this->success []= $elem1500; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1487) { - $xfer += $output->writeString($iter1487); + foreach ($this->success as $iter1501) { + $xfer += $output->writeString($iter1501); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php index ec781a022f00..fd3bdbf40935 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php @@ -68,13 +68,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1841 = 0; - $_etype1844 = 0; - $xfer += $input->readListBegin($_etype1844, $_size1841); - for ($_i1845 = 0; $_i1845 < $_size1841; ++$_i1845) { - $elem1846 = null; - $xfer += $input->readString($elem1846); - $this->success []= $elem1846; + $_size1855 = 0; + $_etype1858 = 0; + $xfer += $input->readListBegin($_etype1858, $_size1855); + for ($_i1859 = 0; $_i1859 < $_size1855; ++$_i1859) { + $elem1860 = null; + $xfer += $input->readString($elem1860); + $this->success []= $elem1860; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1847) { - $xfer += $output->writeString($iter1847); + foreach ($this->success as $iter1861) { + $xfer += $output->writeString($iter1861); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php index f081fa6c6476..aa956ce7c99c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1899 = 0; - $_etype1902 = 0; - $xfer += $input->readListBegin($_etype1902, $_size1899); - for ($_i1903 = 0; $_i1903 < $_size1899; ++$_i1903) { - $elem1904 = null; - $elem1904 = new \metastore\WriteEventInfo(); - $xfer += $elem1904->read($input); - $this->success []= $elem1904; + $_size1913 = 0; + $_etype1916 = 0; + $xfer += $input->readListBegin($_etype1916, $_size1913); + for ($_i1917 = 0; $_i1917 < $_size1913; ++$_i1917) { + $elem1918 = null; + $elem1918 = new \metastore\WriteEventInfo(); + $xfer += $elem1918->read($input); + $this->success []= $elem1918; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1905) { - $xfer += $iter1905->write($output); + foreach ($this->success as $iter1919) { + $xfer += $iter1919->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php index ea626bf4838e..cf6afd2c2395 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1332 = 0; - $_etype1335 = 0; - $xfer += $input->readListBegin($_etype1335, $_size1332); - for ($_i1336 = 0; $_i1336 < $_size1332; ++$_i1336) { - $elem1337 = null; - $xfer += $input->readString($elem1337); - $this->success []= $elem1337; + $_size1346 = 0; + $_etype1349 = 0; + $xfer += $input->readListBegin($_etype1349, $_size1346); + for ($_i1350 = 0; $_i1350 < $_size1346; ++$_i1350) { + $elem1351 = null; + $xfer += $input->readString($elem1351); + $this->success []= $elem1351; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1338) { - $xfer += $output->writeString($iter1338); + foreach ($this->success as $iter1352) { + $xfer += $output->writeString($iter1352); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php index e844c403dcf1..41b8faec9aac 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1346 = 0; - $_etype1349 = 0; - $xfer += $input->readListBegin($_etype1349, $_size1346); - for ($_i1350 = 0; $_i1350 < $_size1346; ++$_i1350) { - $elem1351 = null; - $xfer += $input->readString($elem1351); - $this->success []= $elem1351; + $_size1360 = 0; + $_etype1363 = 0; + $xfer += $input->readListBegin($_etype1363, $_size1360); + for ($_i1364 = 0; $_i1364 < $_size1360; ++$_i1364) { + $elem1365 = null; + $xfer += $input->readString($elem1365); + $this->success []= $elem1365; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1352) { - $xfer += $output->writeString($iter1352); + foreach ($this->success as $iter1366) { + $xfer += $output->writeString($iter1366); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php index b6ed87a9e4ba..548467e4810e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1362 = 0; - $_etype1365 = 0; - $xfer += $input->readListBegin($_etype1365, $_size1362); - for ($_i1366 = 0; $_i1366 < $_size1362; ++$_i1366) { - $elem1367 = null; - $elem1367 = new \metastore\FieldSchema(); - $xfer += $elem1367->read($input); - $this->success []= $elem1367; + $_size1376 = 0; + $_etype1379 = 0; + $xfer += $input->readListBegin($_etype1379, $_size1376); + for ($_i1380 = 0; $_i1380 < $_size1376; ++$_i1380) { + $elem1381 = null; + $elem1381 = new \metastore\FieldSchema(); + $xfer += $elem1381->read($input); + $this->success []= $elem1381; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1368) { - $xfer += $iter1368->write($output); + foreach ($this->success as $iter1382) { + $xfer += $iter1382->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php index 61ca4baa48ab..6e8331417d50 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1369 = 0; - $_etype1372 = 0; - $xfer += $input->readListBegin($_etype1372, $_size1369); - for ($_i1373 = 0; $_i1373 < $_size1369; ++$_i1373) { - $elem1374 = null; - $elem1374 = new \metastore\FieldSchema(); - $xfer += $elem1374->read($input); - $this->success []= $elem1374; + $_size1383 = 0; + $_etype1386 = 0; + $xfer += $input->readListBegin($_etype1386, $_size1383); + for ($_i1387 = 0; $_i1387 < $_size1383; ++$_i1387) { + $elem1388 = null; + $elem1388 = new \metastore\FieldSchema(); + $xfer += $elem1388->read($input); + $this->success []= $elem1388; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1375) { - $xfer += $iter1375->write($output); + foreach ($this->success as $iter1389) { + $xfer += $iter1389->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php index 8268310cc589..7515fcd01b3c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1792 = 0; - $_etype1795 = 0; - $xfer += $input->readListBegin($_etype1795, $_size1792); - for ($_i1796 = 0; $_i1796 < $_size1792; ++$_i1796) { - $elem1797 = null; - $xfer += $input->readString($elem1797); - $this->success []= $elem1797; + $_size1806 = 0; + $_etype1809 = 0; + $xfer += $input->readListBegin($_etype1809, $_size1806); + for ($_i1810 = 0; $_i1810 < $_size1806; ++$_i1810) { + $elem1811 = null; + $xfer += $input->readString($elem1811); + $this->success []= $elem1811; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1798) { - $xfer += $output->writeString($iter1798); + foreach ($this->success as $iter1812) { + $xfer += $output->writeString($iter1812); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php index ee039e9fce9d..da066c60534b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php @@ -68,13 +68,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1848 = 0; - $_etype1851 = 0; - $xfer += $input->readListBegin($_etype1851, $_size1848); - for ($_i1852 = 0; $_i1852 < $_size1848; ++$_i1852) { - $elem1853 = null; - $xfer += $input->readString($elem1853); - $this->success []= $elem1853; + $_size1862 = 0; + $_etype1865 = 0; + $xfer += $input->readListBegin($_etype1865, $_size1862); + for ($_i1866 = 0; $_i1866 < $_size1862; ++$_i1866) { + $elem1867 = null; + $xfer += $input->readString($elem1867); + $this->success []= $elem1867; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1854) { - $xfer += $output->writeString($iter1854); + foreach ($this->success as $iter1868) { + $xfer += $output->writeString($iter1868); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php index 3f7d740c2613..e9c2964bc6a6 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1460 = 0; - $_etype1463 = 0; - $xfer += $input->readListBegin($_etype1463, $_size1460); - for ($_i1464 = 0; $_i1464 < $_size1460; ++$_i1464) { - $elem1465 = null; - $xfer += $input->readString($elem1465); - $this->success []= $elem1465; + $_size1474 = 0; + $_etype1477 = 0; + $xfer += $input->readListBegin($_etype1477, $_size1474); + for ($_i1478 = 0; $_i1478 < $_size1474; ++$_i1478) { + $elem1479 = null; + $xfer += $input->readString($elem1479); + $this->success []= $elem1479; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1466) { - $xfer += $output->writeString($iter1466); + foreach ($this->success as $iter1480) { + $xfer += $output->writeString($iter1480); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php index 65a92e10e262..db79d88731f6 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1709 = 0; - $_etype1712 = 0; - $xfer += $input->readListBegin($_etype1712, $_size1709); - for ($_i1713 = 0; $_i1713 < $_size1709; ++$_i1713) { - $elem1714 = null; - $elem1714 = new \metastore\PartitionSpec(); - $xfer += $elem1714->read($input); - $this->success []= $elem1714; + $_size1723 = 0; + $_etype1726 = 0; + $xfer += $input->readListBegin($_etype1726, $_size1723); + for ($_i1727 = 0; $_i1727 < $_size1723; ++$_i1727) { + $elem1728 = null; + $elem1728 = new \metastore\PartitionSpec(); + $xfer += $elem1728->read($input); + $this->success []= $elem1728; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1715) { - $xfer += $iter1715->write($output); + foreach ($this->success as $iter1729) { + $xfer += $iter1729->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php index ba8b05fc45ce..1d43357ad595 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1558 = 0; - $_etype1561 = 0; - $xfer += $input->readListBegin($_etype1561, $_size1558); - for ($_i1562 = 0; $_i1562 < $_size1558; ++$_i1562) { - $elem1563 = null; - $xfer += $input->readString($elem1563); - $this->part_vals []= $elem1563; + $_size1572 = 0; + $_etype1575 = 0; + $xfer += $input->readListBegin($_etype1575, $_size1572); + for ($_i1576 = 0; $_i1576 < $_size1572; ++$_i1576) { + $elem1577 = null; + $xfer += $input->readString($elem1577); + $this->part_vals []= $elem1577; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1564) { - $xfer += $output->writeString($iter1564); + foreach ($this->part_vals as $iter1578) { + $xfer += $output->writeString($iter1578); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php index 4a51b02c7fb0..5dfca92a5356 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1674 = 0; - $_etype1677 = 0; - $xfer += $input->readListBegin($_etype1677, $_size1674); - for ($_i1678 = 0; $_i1678 < $_size1674; ++$_i1678) { - $elem1679 = null; - $xfer += $input->readString($elem1679); - $this->part_vals []= $elem1679; + $_size1688 = 0; + $_etype1691 = 0; + $xfer += $input->readListBegin($_etype1691, $_size1688); + for ($_i1692 = 0; $_i1692 < $_size1688; ++$_i1692) { + $elem1693 = null; + $xfer += $input->readString($elem1693); + $this->part_vals []= $elem1693; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1680) { - $xfer += $output->writeString($iter1680); + foreach ($this->part_vals as $iter1694) { + $xfer += $output->writeString($iter1694); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php index 93dac9bd6bc4..f6cac7f439f5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php @@ -94,13 +94,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1681 = 0; - $_etype1684 = 0; - $xfer += $input->readListBegin($_etype1684, $_size1681); - for ($_i1685 = 0; $_i1685 < $_size1681; ++$_i1685) { - $elem1686 = null; - $xfer += $input->readString($elem1686); - $this->success []= $elem1686; + $_size1695 = 0; + $_etype1698 = 0; + $xfer += $input->readListBegin($_etype1698, $_size1695); + for ($_i1699 = 0; $_i1699 < $_size1695; ++$_i1699) { + $elem1700 = null; + $xfer += $input->readString($elem1700); + $this->success []= $elem1700; } $xfer += $input->readListEnd(); } else { @@ -143,8 +143,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1687) { - $xfer += $output->writeString($iter1687); + foreach ($this->success as $iter1701) { + $xfer += $output->writeString($iter1701); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php index 2ac83ae1a280..927ddd467d7b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php @@ -94,13 +94,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1688 = 0; - $_etype1691 = 0; - $xfer += $input->readListBegin($_etype1691, $_size1688); - for ($_i1692 = 0; $_i1692 < $_size1688; ++$_i1692) { - $elem1693 = null; - $xfer += $input->readString($elem1693); - $this->success []= $elem1693; + $_size1702 = 0; + $_etype1705 = 0; + $xfer += $input->readListBegin($_etype1705, $_size1702); + for ($_i1706 = 0; $_i1706 < $_size1702; ++$_i1706) { + $elem1707 = null; + $xfer += $input->readString($elem1707); + $this->success []= $elem1707; } $xfer += $input->readListEnd(); } else { @@ -143,8 +143,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1694) { - $xfer += $output->writeString($iter1694); + foreach ($this->success as $iter1708) { + $xfer += $output->writeString($iter1708); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php index 4c0b37b67d93..a3f8299255ba 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php @@ -94,13 +94,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1632 = 0; - $_etype1635 = 0; - $xfer += $input->readListBegin($_etype1635, $_size1632); - for ($_i1636 = 0; $_i1636 < $_size1632; ++$_i1636) { - $elem1637 = null; - $xfer += $input->readString($elem1637); - $this->success []= $elem1637; + $_size1646 = 0; + $_etype1649 = 0; + $xfer += $input->readListBegin($_etype1649, $_size1646); + for ($_i1650 = 0; $_i1650 < $_size1646; ++$_i1650) { + $elem1651 = null; + $xfer += $input->readString($elem1651); + $this->success []= $elem1651; } $xfer += $input->readListEnd(); } else { @@ -143,8 +143,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1638) { - $xfer += $output->writeString($iter1638); + foreach ($this->success as $iter1652) { + $xfer += $output->writeString($iter1652); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php index 9c850cede841..8e79331510ae 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php @@ -134,13 +134,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1590 = 0; - $_etype1593 = 0; - $xfer += $input->readListBegin($_etype1593, $_size1590); - for ($_i1594 = 0; $_i1594 < $_size1590; ++$_i1594) { - $elem1595 = null; - $xfer += $input->readString($elem1595); - $this->part_vals []= $elem1595; + $_size1604 = 0; + $_etype1607 = 0; + $xfer += $input->readListBegin($_etype1607, $_size1604); + for ($_i1608 = 0; $_i1608 < $_size1604; ++$_i1608) { + $elem1609 = null; + $xfer += $input->readString($elem1609); + $this->part_vals []= $elem1609; } $xfer += $input->readListEnd(); } else { @@ -157,13 +157,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1596 = 0; - $_etype1599 = 0; - $xfer += $input->readListBegin($_etype1599, $_size1596); - for ($_i1600 = 0; $_i1600 < $_size1596; ++$_i1600) { - $elem1601 = null; - $xfer += $input->readString($elem1601); - $this->group_names []= $elem1601; + $_size1610 = 0; + $_etype1613 = 0; + $xfer += $input->readListBegin($_etype1613, $_size1610); + for ($_i1614 = 0; $_i1614 < $_size1610; ++$_i1614) { + $elem1615 = null; + $xfer += $input->readString($elem1615); + $this->group_names []= $elem1615; } $xfer += $input->readListEnd(); } else { @@ -200,8 +200,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1602) { - $xfer += $output->writeString($iter1602); + foreach ($this->part_vals as $iter1616) { + $xfer += $output->writeString($iter1616); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -217,8 +217,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1603) { - $xfer += $output->writeString($iter1603); + foreach ($this->group_names as $iter1617) { + $xfer += $output->writeString($iter1617); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php index 893f588f3ff2..accd7d41558b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1702 = 0; - $_etype1705 = 0; - $xfer += $input->readListBegin($_etype1705, $_size1702); - for ($_i1706 = 0; $_i1706 < $_size1702; ++$_i1706) { - $elem1707 = null; - $elem1707 = new \metastore\Partition(); - $xfer += $elem1707->read($input); - $this->success []= $elem1707; + $_size1716 = 0; + $_etype1719 = 0; + $xfer += $input->readListBegin($_etype1719, $_size1716); + for ($_i1720 = 0; $_i1720 < $_size1716; ++$_i1720) { + $elem1721 = null; + $elem1721 = new \metastore\Partition(); + $xfer += $elem1721->read($input); + $this->success []= $elem1721; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1708) { - $xfer += $iter1708->write($output); + foreach ($this->success as $iter1722) { + $xfer += $iter1722->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php index f32b1c73a240..3479e5db697f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1695 = 0; - $_etype1698 = 0; - $xfer += $input->readListBegin($_etype1698, $_size1695); - for ($_i1699 = 0; $_i1699 < $_size1695; ++$_i1699) { - $elem1700 = null; - $elem1700 = new \metastore\Partition(); - $xfer += $elem1700->read($input); - $this->success []= $elem1700; + $_size1709 = 0; + $_etype1712 = 0; + $xfer += $input->readListBegin($_etype1712, $_size1709); + for ($_i1713 = 0; $_i1713 < $_size1709; ++$_i1713) { + $elem1714 = null; + $elem1714 = new \metastore\Partition(); + $xfer += $elem1714->read($input); + $this->success []= $elem1714; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1701) { - $xfer += $iter1701->write($output); + foreach ($this->success as $iter1715) { + $xfer += $iter1715->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php index e41efcb469ff..e49950967509 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->names = array(); - $_size1716 = 0; - $_etype1719 = 0; - $xfer += $input->readListBegin($_etype1719, $_size1716); - for ($_i1720 = 0; $_i1720 < $_size1716; ++$_i1720) { - $elem1721 = null; - $xfer += $input->readString($elem1721); - $this->names []= $elem1721; + $_size1730 = 0; + $_etype1733 = 0; + $xfer += $input->readListBegin($_etype1733, $_size1730); + for ($_i1734 = 0; $_i1734 < $_size1730; ++$_i1734) { + $elem1735 = null; + $xfer += $input->readString($elem1735); + $this->names []= $elem1735; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('names', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->names)); - foreach ($this->names as $iter1722) { - $xfer += $output->writeString($iter1722); + foreach ($this->names as $iter1736) { + $xfer += $output->writeString($iter1736); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php index bc019ce3916b..616ae5afaac1 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1723 = 0; - $_etype1726 = 0; - $xfer += $input->readListBegin($_etype1726, $_size1723); - for ($_i1727 = 0; $_i1727 < $_size1723; ++$_i1727) { - $elem1728 = null; - $elem1728 = new \metastore\Partition(); - $xfer += $elem1728->read($input); - $this->success []= $elem1728; + $_size1737 = 0; + $_etype1740 = 0; + $xfer += $input->readListBegin($_etype1740, $_size1737); + for ($_i1741 = 0; $_i1741 < $_size1737; ++$_i1741) { + $elem1742 = null; + $elem1742 = new \metastore\Partition(); + $xfer += $elem1742->read($input); + $this->success []= $elem1742; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1729) { - $xfer += $iter1729->write($output); + foreach ($this->success as $iter1743) { + $xfer += $iter1743->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php index a44b83e0f931..b51b393bf67c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1639 = 0; - $_etype1642 = 0; - $xfer += $input->readListBegin($_etype1642, $_size1639); - for ($_i1643 = 0; $_i1643 < $_size1639; ++$_i1643) { - $elem1644 = null; - $xfer += $input->readString($elem1644); - $this->part_vals []= $elem1644; + $_size1653 = 0; + $_etype1656 = 0; + $xfer += $input->readListBegin($_etype1656, $_size1653); + for ($_i1657 = 0; $_i1657 < $_size1653; ++$_i1657) { + $elem1658 = null; + $xfer += $input->readString($elem1658); + $this->part_vals []= $elem1658; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1645) { - $xfer += $output->writeString($iter1645); + foreach ($this->part_vals as $iter1659) { + $xfer += $output->writeString($iter1659); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php index 946da8be4145..a4751d3ba9f0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1646 = 0; - $_etype1649 = 0; - $xfer += $input->readListBegin($_etype1649, $_size1646); - for ($_i1650 = 0; $_i1650 < $_size1646; ++$_i1650) { - $elem1651 = null; - $elem1651 = new \metastore\Partition(); - $xfer += $elem1651->read($input); - $this->success []= $elem1651; + $_size1660 = 0; + $_etype1663 = 0; + $xfer += $input->readListBegin($_etype1663, $_size1660); + for ($_i1664 = 0; $_i1664 < $_size1660; ++$_i1664) { + $elem1665 = null; + $elem1665 = new \metastore\Partition(); + $xfer += $elem1665->read($input); + $this->success []= $elem1665; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1652) { - $xfer += $iter1652->write($output); + foreach ($this->success as $iter1666) { + $xfer += $iter1666->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php index ddf8b85ca026..fe977507604b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php @@ -146,13 +146,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1653 = 0; - $_etype1656 = 0; - $xfer += $input->readListBegin($_etype1656, $_size1653); - for ($_i1657 = 0; $_i1657 < $_size1653; ++$_i1657) { - $elem1658 = null; - $xfer += $input->readString($elem1658); - $this->part_vals []= $elem1658; + $_size1667 = 0; + $_etype1670 = 0; + $xfer += $input->readListBegin($_etype1670, $_size1667); + for ($_i1671 = 0; $_i1671 < $_size1667; ++$_i1671) { + $elem1672 = null; + $xfer += $input->readString($elem1672); + $this->part_vals []= $elem1672; } $xfer += $input->readListEnd(); } else { @@ -176,13 +176,13 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1659 = 0; - $_etype1662 = 0; - $xfer += $input->readListBegin($_etype1662, $_size1659); - for ($_i1663 = 0; $_i1663 < $_size1659; ++$_i1663) { - $elem1664 = null; - $xfer += $input->readString($elem1664); - $this->group_names []= $elem1664; + $_size1673 = 0; + $_etype1676 = 0; + $xfer += $input->readListBegin($_etype1676, $_size1673); + for ($_i1677 = 0; $_i1677 < $_size1673; ++$_i1677) { + $elem1678 = null; + $xfer += $input->readString($elem1678); + $this->group_names []= $elem1678; } $xfer += $input->readListEnd(); } else { @@ -219,8 +219,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1665) { - $xfer += $output->writeString($iter1665); + foreach ($this->part_vals as $iter1679) { + $xfer += $output->writeString($iter1679); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -241,8 +241,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 6); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1666) { - $xfer += $output->writeString($iter1666); + foreach ($this->group_names as $iter1680) { + $xfer += $output->writeString($iter1680); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php index 58929995a41b..cd3ea604870b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1667 = 0; - $_etype1670 = 0; - $xfer += $input->readListBegin($_etype1670, $_size1667); - for ($_i1671 = 0; $_i1671 < $_size1667; ++$_i1671) { - $elem1672 = null; - $elem1672 = new \metastore\Partition(); - $xfer += $elem1672->read($input); - $this->success []= $elem1672; + $_size1681 = 0; + $_etype1684 = 0; + $xfer += $input->readListBegin($_etype1684, $_size1681); + for ($_i1685 = 0; $_i1685 < $_size1681; ++$_i1685) { + $elem1686 = null; + $elem1686 = new \metastore\Partition(); + $xfer += $elem1686->read($input); + $this->success []= $elem1686; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1673) { - $xfer += $iter1673->write($output); + foreach ($this->success as $iter1687) { + $xfer += $iter1687->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php index b6922cbf912a..7e25f59da25c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1625 = 0; - $_etype1628 = 0; - $xfer += $input->readListBegin($_etype1628, $_size1625); - for ($_i1629 = 0; $_i1629 < $_size1625; ++$_i1629) { - $elem1630 = null; - $elem1630 = new \metastore\PartitionSpec(); - $xfer += $elem1630->read($input); - $this->success []= $elem1630; + $_size1639 = 0; + $_etype1642 = 0; + $xfer += $input->readListBegin($_etype1642, $_size1639); + for ($_i1643 = 0; $_i1643 < $_size1639; ++$_i1643) { + $elem1644 = null; + $elem1644 = new \metastore\PartitionSpec(); + $xfer += $elem1644->read($input); + $this->success []= $elem1644; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1631) { - $xfer += $iter1631->write($output); + foreach ($this->success as $iter1645) { + $xfer += $iter1645->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php index 0d128f5ab20b..42c9a65c494e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1604 = 0; - $_etype1607 = 0; - $xfer += $input->readListBegin($_etype1607, $_size1604); - for ($_i1608 = 0; $_i1608 < $_size1604; ++$_i1608) { - $elem1609 = null; - $elem1609 = new \metastore\Partition(); - $xfer += $elem1609->read($input); - $this->success []= $elem1609; + $_size1618 = 0; + $_etype1621 = 0; + $xfer += $input->readListBegin($_etype1621, $_size1618); + for ($_i1622 = 0; $_i1622 < $_size1618; ++$_i1622) { + $elem1623 = null; + $elem1623 = new \metastore\Partition(); + $xfer += $elem1623->read($input); + $this->success []= $elem1623; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1610) { - $xfer += $iter1610->write($output); + foreach ($this->success as $iter1624) { + $xfer += $iter1624->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php index 31ec4fb2a67b..b54451581882 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php @@ -144,13 +144,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1611 = 0; - $_etype1614 = 0; - $xfer += $input->readListBegin($_etype1614, $_size1611); - for ($_i1615 = 0; $_i1615 < $_size1611; ++$_i1615) { - $elem1616 = null; - $xfer += $input->readString($elem1616); - $this->group_names []= $elem1616; + $_size1625 = 0; + $_etype1628 = 0; + $xfer += $input->readListBegin($_etype1628, $_size1625); + for ($_i1629 = 0; $_i1629 < $_size1625; ++$_i1629) { + $elem1630 = null; + $xfer += $input->readString($elem1630); + $this->group_names []= $elem1630; } $xfer += $input->readListEnd(); } else { @@ -197,8 +197,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1617) { - $xfer += $output->writeString($iter1617); + foreach ($this->group_names as $iter1631) { + $xfer += $output->writeString($iter1631); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php index bfa9e9ca2eba..9f06b78f393e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1618 = 0; - $_etype1621 = 0; - $xfer += $input->readListBegin($_etype1621, $_size1618); - for ($_i1622 = 0; $_i1622 < $_size1618; ++$_i1622) { - $elem1623 = null; - $elem1623 = new \metastore\Partition(); - $xfer += $elem1623->read($input); - $this->success []= $elem1623; + $_size1632 = 0; + $_etype1635 = 0; + $xfer += $input->readListBegin($_etype1635, $_size1632); + for ($_i1636 = 0; $_i1636 < $_size1632; ++$_i1636) { + $elem1637 = null; + $elem1637 = new \metastore\Partition(); + $xfer += $elem1637->read($input); + $this->success []= $elem1637; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1624) { - $xfer += $iter1624->write($output); + foreach ($this->success as $iter1638) { + $xfer += $iter1638->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php index 62117af4baf8..ab1303b1f8ca 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php @@ -108,13 +108,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1813 = 0; - $_etype1816 = 0; - $xfer += $input->readListBegin($_etype1816, $_size1813); - for ($_i1817 = 0; $_i1817 < $_size1813; ++$_i1817) { - $elem1818 = null; - $xfer += $input->readString($elem1818); - $this->group_names []= $elem1818; + $_size1827 = 0; + $_etype1830 = 0; + $xfer += $input->readListBegin($_etype1830, $_size1827); + for ($_i1831 = 0; $_i1831 < $_size1827; ++$_i1831) { + $elem1832 = null; + $xfer += $input->readString($elem1832); + $this->group_names []= $elem1832; } $xfer += $input->readListEnd(); } else { @@ -154,8 +154,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1819) { - $xfer += $output->writeString($iter1819); + foreach ($this->group_names as $iter1833) { + $xfer += $output->writeString($iter1833); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php index 79c556d168da..7353970ddf4c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1799 = 0; - $_etype1802 = 0; - $xfer += $input->readListBegin($_etype1802, $_size1799); - for ($_i1803 = 0; $_i1803 < $_size1799; ++$_i1803) { - $elem1804 = null; - $xfer += $input->readString($elem1804); - $this->success []= $elem1804; + $_size1813 = 0; + $_etype1816 = 0; + $xfer += $input->readListBegin($_etype1816, $_size1813); + for ($_i1817 = 0; $_i1817 < $_size1813; ++$_i1817) { + $elem1818 = null; + $xfer += $input->readString($elem1818); + $this->success []= $elem1818; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1805) { - $xfer += $output->writeString($iter1805); + foreach ($this->success as $iter1819) { + $xfer += $output->writeString($iter1819); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php index e5caa75b0190..051d4230459a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1878 = 0; - $_etype1881 = 0; - $xfer += $input->readListBegin($_etype1881, $_size1878); - for ($_i1882 = 0; $_i1882 < $_size1878; ++$_i1882) { - $elem1883 = null; - $elem1883 = new \metastore\RuntimeStat(); - $xfer += $elem1883->read($input); - $this->success []= $elem1883; + $_size1892 = 0; + $_etype1895 = 0; + $xfer += $input->readListBegin($_etype1895, $_size1892); + for ($_i1896 = 0; $_i1896 < $_size1892; ++$_i1896) { + $elem1897 = null; + $elem1897 = new \metastore\RuntimeStat(); + $xfer += $elem1897->read($input); + $this->success []= $elem1897; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1884) { - $xfer += $iter1884->write($output); + foreach ($this->success as $iter1898) { + $xfer += $iter1898->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php index 208eb079e250..b3afca738165 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1871 = 0; - $_etype1874 = 0; - $xfer += $input->readListBegin($_etype1874, $_size1871); - for ($_i1875 = 0; $_i1875 < $_size1871; ++$_i1875) { - $elem1876 = null; - $elem1876 = new \metastore\SchemaVersion(); - $xfer += $elem1876->read($input); - $this->success []= $elem1876; + $_size1885 = 0; + $_etype1888 = 0; + $xfer += $input->readListBegin($_etype1888, $_size1885); + for ($_i1889 = 0; $_i1889 < $_size1885; ++$_i1889) { + $elem1890 = null; + $elem1890 = new \metastore\SchemaVersion(); + $xfer += $elem1890->read($input); + $this->success []= $elem1890; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1877) { - $xfer += $iter1877->write($output); + foreach ($this->success as $iter1891) { + $xfer += $iter1891->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php index 868089378eb0..f6ae9424cb74 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1376 = 0; - $_etype1379 = 0; - $xfer += $input->readListBegin($_etype1379, $_size1376); - for ($_i1380 = 0; $_i1380 < $_size1376; ++$_i1380) { - $elem1381 = null; - $elem1381 = new \metastore\FieldSchema(); - $xfer += $elem1381->read($input); - $this->success []= $elem1381; + $_size1390 = 0; + $_etype1393 = 0; + $xfer += $input->readListBegin($_etype1393, $_size1390); + for ($_i1394 = 0; $_i1394 < $_size1390; ++$_i1394) { + $elem1395 = null; + $elem1395 = new \metastore\FieldSchema(); + $xfer += $elem1395->read($input); + $this->success []= $elem1395; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1382) { - $xfer += $iter1382->write($output); + foreach ($this->success as $iter1396) { + $xfer += $iter1396->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php index 09801c089181..adc09cc0e92b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1383 = 0; - $_etype1386 = 0; - $xfer += $input->readListBegin($_etype1386, $_size1383); - for ($_i1387 = 0; $_i1387 < $_size1383; ++$_i1387) { - $elem1388 = null; - $elem1388 = new \metastore\FieldSchema(); - $xfer += $elem1388->read($input); - $this->success []= $elem1388; + $_size1397 = 0; + $_etype1400 = 0; + $xfer += $input->readListBegin($_etype1400, $_size1397); + for ($_i1401 = 0; $_i1401 < $_size1397; ++$_i1401) { + $elem1402 = null; + $elem1402 = new \metastore\FieldSchema(); + $xfer += $elem1402->read($input); + $this->success []= $elem1402; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1389) { - $xfer += $iter1389->write($output); + foreach ($this->success as $iter1403) { + $xfer += $iter1403->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php index 80db8401f838..eb41013a39aa 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->tbl_types = array(); - $_size1467 = 0; - $_etype1470 = 0; - $xfer += $input->readListBegin($_etype1470, $_size1467); - for ($_i1471 = 0; $_i1471 < $_size1467; ++$_i1471) { - $elem1472 = null; - $xfer += $input->readString($elem1472); - $this->tbl_types []= $elem1472; + $_size1481 = 0; + $_etype1484 = 0; + $xfer += $input->readListBegin($_etype1484, $_size1481); + for ($_i1485 = 0; $_i1485 < $_size1481; ++$_i1485) { + $elem1486 = null; + $xfer += $input->readString($elem1486); + $this->tbl_types []= $elem1486; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('tbl_types', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->tbl_types)); - foreach ($this->tbl_types as $iter1473) { - $xfer += $output->writeString($iter1473); + foreach ($this->tbl_types as $iter1487) { + $xfer += $output->writeString($iter1487); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php index 3ebf4e02ef24..fa85fda009df 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1474 = 0; - $_etype1477 = 0; - $xfer += $input->readListBegin($_etype1477, $_size1474); - for ($_i1478 = 0; $_i1478 < $_size1474; ++$_i1478) { - $elem1479 = null; - $elem1479 = new \metastore\TableMeta(); - $xfer += $elem1479->read($input); - $this->success []= $elem1479; + $_size1488 = 0; + $_etype1491 = 0; + $xfer += $input->readListBegin($_etype1491, $_size1488); + for ($_i1492 = 0; $_i1492 < $_size1488; ++$_i1492) { + $elem1493 = null; + $elem1493 = new \metastore\TableMeta(); + $xfer += $elem1493->read($input); + $this->success []= $elem1493; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1480) { - $xfer += $iter1480->write($output); + foreach ($this->success as $iter1494) { + $xfer += $iter1494->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php index 33662073a7ff..e125030e775a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php @@ -107,13 +107,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1509 = 0; - $_etype1512 = 0; - $xfer += $input->readListBegin($_etype1512, $_size1509); - for ($_i1513 = 0; $_i1513 < $_size1509; ++$_i1513) { - $elem1514 = null; - $xfer += $input->readString($elem1514); - $this->success []= $elem1514; + $_size1523 = 0; + $_etype1526 = 0; + $xfer += $input->readListBegin($_etype1526, $_size1523); + for ($_i1527 = 0; $_i1527 < $_size1523; ++$_i1527) { + $elem1528 = null; + $xfer += $input->readString($elem1528); + $this->success []= $elem1528; } $xfer += $input->readListEnd(); } else { @@ -164,8 +164,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1515) { - $xfer += $output->writeString($iter1515); + foreach ($this->success as $iter1529) { + $xfer += $output->writeString($iter1529); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php index 6b4878fc419b..8bdb9756a4b3 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php @@ -87,13 +87,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->tbl_names = array(); - $_size1488 = 0; - $_etype1491 = 0; - $xfer += $input->readListBegin($_etype1491, $_size1488); - for ($_i1492 = 0; $_i1492 < $_size1488; ++$_i1492) { - $elem1493 = null; - $xfer += $input->readString($elem1493); - $this->tbl_names []= $elem1493; + $_size1502 = 0; + $_etype1505 = 0; + $xfer += $input->readListBegin($_etype1505, $_size1502); + for ($_i1506 = 0; $_i1506 < $_size1502; ++$_i1506) { + $elem1507 = null; + $xfer += $input->readString($elem1507); + $this->tbl_names []= $elem1507; } $xfer += $input->readListEnd(); } else { @@ -125,8 +125,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->tbl_names)); - foreach ($this->tbl_names as $iter1494) { - $xfer += $output->writeString($iter1494); + foreach ($this->tbl_names as $iter1508) { + $xfer += $output->writeString($iter1508); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php index c23cd3ea05b9..fccbc17d1db8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php @@ -69,14 +69,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1495 = 0; - $_etype1498 = 0; - $xfer += $input->readListBegin($_etype1498, $_size1495); - for ($_i1499 = 0; $_i1499 < $_size1495; ++$_i1499) { - $elem1500 = null; - $elem1500 = new \metastore\Table(); - $xfer += $elem1500->read($input); - $this->success []= $elem1500; + $_size1509 = 0; + $_etype1512 = 0; + $xfer += $input->readListBegin($_etype1512, $_size1509); + for ($_i1513 = 0; $_i1513 < $_size1509; ++$_i1513) { + $elem1514 = null; + $elem1514 = new \metastore\Table(); + $xfer += $elem1514->read($input); + $this->success []= $elem1514; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1501) { - $xfer += $iter1501->write($output); + foreach ($this->success as $iter1515) { + $xfer += $iter1515->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php index 4fd26704fd0b..f5819a715114 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1446 = 0; - $_etype1449 = 0; - $xfer += $input->readListBegin($_etype1449, $_size1446); - for ($_i1450 = 0; $_i1450 < $_size1446; ++$_i1450) { - $elem1451 = null; - $xfer += $input->readString($elem1451); - $this->success []= $elem1451; + $_size1460 = 0; + $_etype1463 = 0; + $xfer += $input->readListBegin($_etype1463, $_size1460); + for ($_i1464 = 0; $_i1464 < $_size1460; ++$_i1464) { + $elem1465 = null; + $xfer += $input->readString($elem1465); + $this->success []= $elem1465; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1452) { - $xfer += $output->writeString($iter1452); + foreach ($this->success as $iter1466) { + $xfer += $output->writeString($iter1466); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php index 657966cbe1c6..6d23127f2712 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1502 = 0; - $_etype1505 = 0; - $xfer += $input->readListBegin($_etype1505, $_size1502); - for ($_i1506 = 0; $_i1506 < $_size1502; ++$_i1506) { - $elem1507 = null; - $elem1507 = new \metastore\ExtendedTableInfo(); - $xfer += $elem1507->read($input); - $this->success []= $elem1507; + $_size1516 = 0; + $_etype1519 = 0; + $xfer += $input->readListBegin($_etype1519, $_size1516); + for ($_i1520 = 0; $_i1520 < $_size1516; ++$_i1520) { + $elem1521 = null; + $elem1521 = new \metastore\ExtendedTableInfo(); + $xfer += $elem1521->read($input); + $this->success []= $elem1521; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1508) { - $xfer += $iter1508->write($output); + foreach ($this->success as $iter1522) { + $xfer += $iter1522->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php index 17d52499ecf8..1e24c0bae93a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1439 = 0; - $_etype1442 = 0; - $xfer += $input->readListBegin($_etype1442, $_size1439); - for ($_i1443 = 0; $_i1443 < $_size1439; ++$_i1443) { - $elem1444 = null; - $xfer += $input->readString($elem1444); - $this->success []= $elem1444; + $_size1453 = 0; + $_etype1456 = 0; + $xfer += $input->readListBegin($_etype1456, $_size1453); + for ($_i1457 = 0; $_i1457 < $_size1453; ++$_i1457) { + $elem1458 = null; + $xfer += $input->readString($elem1458); + $this->success []= $elem1458; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1445) { - $xfer += $output->writeString($iter1445); + foreach ($this->success as $iter1459) { + $xfer += $output->writeString($iter1459); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php index b4c33658cd7a..45daa41ab670 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php @@ -86,17 +86,17 @@ public function read($input) case 0: if ($ftype == TType::MAP) { $this->success = array(); - $_size1353 = 0; - $_ktype1354 = 0; - $_vtype1355 = 0; - $xfer += $input->readMapBegin($_ktype1354, $_vtype1355, $_size1353); - for ($_i1357 = 0; $_i1357 < $_size1353; ++$_i1357) { - $key1358 = ''; - $val1359 = new \metastore\Type(); - $xfer += $input->readString($key1358); - $val1359 = new \metastore\Type(); - $xfer += $val1359->read($input); - $this->success[$key1358] = $val1359; + $_size1367 = 0; + $_ktype1368 = 0; + $_vtype1369 = 0; + $xfer += $input->readMapBegin($_ktype1368, $_vtype1369, $_size1367); + for ($_i1371 = 0; $_i1371 < $_size1367; ++$_i1371) { + $key1372 = ''; + $val1373 = new \metastore\Type(); + $xfer += $input->readString($key1372); + $val1373 = new \metastore\Type(); + $xfer += $val1373->read($input); + $this->success[$key1372] = $val1373; } $xfer += $input->readMapEnd(); } else { @@ -131,9 +131,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::MAP, 0); $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success)); - foreach ($this->success as $kiter1360 => $viter1361) { - $xfer += $output->writeString($kiter1360); - $xfer += $viter1361->write($output); + foreach ($this->success as $kiter1374 => $viter1375) { + $xfer += $output->writeString($kiter1374); + $xfer += $viter1375->write($output); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php index e2a3a188693a..38c1c98781de 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php @@ -123,16 +123,16 @@ public function read($input) case 3: if ($ftype == TType::MAP) { $this->part_vals = array(); - $_size1783 = 0; - $_ktype1784 = 0; - $_vtype1785 = 0; - $xfer += $input->readMapBegin($_ktype1784, $_vtype1785, $_size1783); - for ($_i1787 = 0; $_i1787 < $_size1783; ++$_i1787) { - $key1788 = ''; - $val1789 = ''; - $xfer += $input->readString($key1788); - $xfer += $input->readString($val1789); - $this->part_vals[$key1788] = $val1789; + $_size1797 = 0; + $_ktype1798 = 0; + $_vtype1799 = 0; + $xfer += $input->readMapBegin($_ktype1798, $_vtype1799, $_size1797); + for ($_i1801 = 0; $_i1801 < $_size1797; ++$_i1801) { + $key1802 = ''; + $val1803 = ''; + $xfer += $input->readString($key1802); + $xfer += $input->readString($val1803); + $this->part_vals[$key1802] = $val1803; } $xfer += $input->readMapEnd(); } else { @@ -176,9 +176,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $kiter1790 => $viter1791) { - $xfer += $output->writeString($kiter1790); - $xfer += $output->writeString($viter1791); + foreach ($this->part_vals as $kiter1804 => $viter1805) { + $xfer += $output->writeString($kiter1804); + $xfer += $output->writeString($viter1805); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php index ed9668965608..0c04dbcb0aea 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1820 = 0; - $_etype1823 = 0; - $xfer += $input->readListBegin($_etype1823, $_size1820); - for ($_i1824 = 0; $_i1824 < $_size1820; ++$_i1824) { - $elem1825 = null; - $elem1825 = new \metastore\HiveObjectPrivilege(); - $xfer += $elem1825->read($input); - $this->success []= $elem1825; + $_size1834 = 0; + $_etype1837 = 0; + $xfer += $input->readListBegin($_etype1837, $_size1834); + for ($_i1838 = 0; $_i1838 < $_size1834; ++$_i1838) { + $elem1839 = null; + $elem1839 = new \metastore\HiveObjectPrivilege(); + $xfer += $elem1839->read($input); + $this->success []= $elem1839; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1826) { - $xfer += $iter1826->write($output); + foreach ($this->success as $iter1840) { + $xfer += $iter1840->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php index 8f45cdb166d6..860a92a023cc 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1806 = 0; - $_etype1809 = 0; - $xfer += $input->readListBegin($_etype1809, $_size1806); - for ($_i1810 = 0; $_i1810 < $_size1806; ++$_i1810) { - $elem1811 = null; - $elem1811 = new \metastore\Role(); - $xfer += $elem1811->read($input); - $this->success []= $elem1811; + $_size1820 = 0; + $_etype1823 = 0; + $xfer += $input->readListBegin($_etype1823, $_size1820); + for ($_i1824 = 0; $_i1824 < $_size1820; ++$_i1824) { + $elem1825 = null; + $elem1825 = new \metastore\Role(); + $xfer += $elem1825->read($input); + $this->success []= $elem1825; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1812) { - $xfer += $iter1812->write($output); + foreach ($this->success as $iter1826) { + $xfer += $iter1826->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php index dfe1d8895ef3..a335b9c9d02f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php @@ -123,16 +123,16 @@ public function read($input) case 3: if ($ftype == TType::MAP) { $this->part_vals = array(); - $_size1774 = 0; - $_ktype1775 = 0; - $_vtype1776 = 0; - $xfer += $input->readMapBegin($_ktype1775, $_vtype1776, $_size1774); - for ($_i1778 = 0; $_i1778 < $_size1774; ++$_i1778) { - $key1779 = ''; - $val1780 = ''; - $xfer += $input->readString($key1779); - $xfer += $input->readString($val1780); - $this->part_vals[$key1779] = $val1780; + $_size1788 = 0; + $_ktype1789 = 0; + $_vtype1790 = 0; + $xfer += $input->readMapBegin($_ktype1789, $_vtype1790, $_size1788); + for ($_i1792 = 0; $_i1792 < $_size1788; ++$_i1792) { + $key1793 = ''; + $val1794 = ''; + $xfer += $input->readString($key1793); + $xfer += $input->readString($val1794); + $this->part_vals[$key1793] = $val1794; } $xfer += $input->readMapEnd(); } else { @@ -176,9 +176,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $kiter1781 => $viter1782) { - $xfer += $output->writeString($kiter1781); - $xfer += $output->writeString($viter1782); + foreach ($this->part_vals as $kiter1795 => $viter1796) { + $xfer += $output->writeString($kiter1795); + $xfer += $output->writeString($viter1796); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php index 603b3c54fc86..32a792c46d56 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php @@ -80,13 +80,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1751 = 0; - $_etype1754 = 0; - $xfer += $input->readListBegin($_etype1754, $_size1751); - for ($_i1755 = 0; $_i1755 < $_size1751; ++$_i1755) { - $elem1756 = null; - $xfer += $input->readString($elem1756); - $this->part_vals []= $elem1756; + $_size1765 = 0; + $_etype1768 = 0; + $xfer += $input->readListBegin($_etype1768, $_size1765); + for ($_i1769 = 0; $_i1769 < $_size1765; ++$_i1769) { + $elem1770 = null; + $xfer += $input->readString($elem1770); + $this->part_vals []= $elem1770; } $xfer += $input->readListEnd(); } else { @@ -120,8 +120,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 1); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1757) { - $xfer += $output->writeString($iter1757); + foreach ($this->part_vals as $iter1771) { + $xfer += $output->writeString($iter1771); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php index 11d1e8bf2ab6..18263b34051d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php @@ -85,16 +85,16 @@ public function read($input) case 0: if ($ftype == TType::MAP) { $this->success = array(); - $_size1765 = 0; - $_ktype1766 = 0; - $_vtype1767 = 0; - $xfer += $input->readMapBegin($_ktype1766, $_vtype1767, $_size1765); - for ($_i1769 = 0; $_i1769 < $_size1765; ++$_i1769) { - $key1770 = ''; - $val1771 = ''; - $xfer += $input->readString($key1770); - $xfer += $input->readString($val1771); - $this->success[$key1770] = $val1771; + $_size1779 = 0; + $_ktype1780 = 0; + $_vtype1781 = 0; + $xfer += $input->readMapBegin($_ktype1780, $_vtype1781, $_size1779); + for ($_i1783 = 0; $_i1783 < $_size1779; ++$_i1783) { + $key1784 = ''; + $val1785 = ''; + $xfer += $input->readString($key1784); + $xfer += $input->readString($val1785); + $this->success[$key1784] = $val1785; } $xfer += $input->readMapEnd(); } else { @@ -129,9 +129,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::MAP, 0); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); - foreach ($this->success as $kiter1772 => $viter1773) { - $xfer += $output->writeString($kiter1772); - $xfer += $output->writeString($viter1773); + foreach ($this->success as $kiter1786 => $viter1787) { + $xfer += $output->writeString($kiter1786); + $xfer += $output->writeString($viter1787); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php index 807926c49dbc..bd6f8e46f7d0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1758 = 0; - $_etype1761 = 0; - $xfer += $input->readListBegin($_etype1761, $_size1758); - for ($_i1762 = 0; $_i1762 < $_size1758; ++$_i1762) { - $elem1763 = null; - $xfer += $input->readString($elem1763); - $this->success []= $elem1763; + $_size1772 = 0; + $_etype1775 = 0; + $xfer += $input->readListBegin($_etype1775, $_size1772); + for ($_i1776 = 0; $_i1776 < $_size1772; ++$_i1776) { + $elem1777 = null; + $xfer += $input->readString($elem1777); + $this->success []= $elem1777; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1764) { - $xfer += $output->writeString($iter1764); + foreach ($this->success as $iter1778) { + $xfer += $output->writeString($iter1778); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php index e0102b445466..a0ebcef960cd 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php @@ -119,13 +119,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1744 = 0; - $_etype1747 = 0; - $xfer += $input->readListBegin($_etype1747, $_size1744); - for ($_i1748 = 0; $_i1748 < $_size1744; ++$_i1748) { - $elem1749 = null; - $xfer += $input->readString($elem1749); - $this->part_vals []= $elem1749; + $_size1758 = 0; + $_etype1761 = 0; + $xfer += $input->readListBegin($_etype1761, $_size1758); + for ($_i1762 = 0; $_i1762 < $_size1758; ++$_i1762) { + $elem1763 = null; + $xfer += $input->readString($elem1763); + $this->part_vals []= $elem1763; } $xfer += $input->readListEnd(); } else { @@ -170,8 +170,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1750) { - $xfer += $output->writeString($iter1750); + foreach ($this->part_vals as $iter1764) { + $xfer += $output->writeString($iter1764); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php index b1aef3bb1581..6ce4ee0b957d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php @@ -87,13 +87,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1827 = 0; - $_etype1830 = 0; - $xfer += $input->readListBegin($_etype1830, $_size1827); - for ($_i1831 = 0; $_i1831 < $_size1827; ++$_i1831) { - $elem1832 = null; - $xfer += $input->readString($elem1832); - $this->group_names []= $elem1832; + $_size1841 = 0; + $_etype1844 = 0; + $xfer += $input->readListBegin($_etype1844, $_size1841); + for ($_i1845 = 0; $_i1845 < $_size1841; ++$_i1845) { + $elem1846 = null; + $xfer += $input->readString($elem1846); + $this->group_names []= $elem1846; } $xfer += $input->readListEnd(); } else { @@ -125,8 +125,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1833) { - $xfer += $output->writeString($iter1833); + foreach ($this->group_names as $iter1847) { + $xfer += $output->writeString($iter1847); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php index 8b258d78576a..7436e4e22bb7 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1834 = 0; - $_etype1837 = 0; - $xfer += $input->readListBegin($_etype1837, $_size1834); - for ($_i1838 = 0; $_i1838 < $_size1834; ++$_i1838) { - $elem1839 = null; - $xfer += $input->readString($elem1839); - $this->success []= $elem1839; + $_size1848 = 0; + $_etype1851 = 0; + $xfer += $input->readListBegin($_etype1851, $_size1848); + for ($_i1852 = 0; $_i1852 < $_size1848; ++$_i1852) { + $elem1853 = null; + $xfer += $input->readString($elem1853); + $this->success []= $elem1853; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1840) { - $xfer += $output->writeString($iter1840); + foreach ($this->success as $iter1854) { + $xfer += $output->writeString($iter1854); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php index cc99395b90d7..9b895253caeb 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->partNames = array(); - $_size1432 = 0; - $_etype1435 = 0; - $xfer += $input->readListBegin($_etype1435, $_size1432); - for ($_i1436 = 0; $_i1436 < $_size1432; ++$_i1436) { - $elem1437 = null; - $xfer += $input->readString($elem1437); - $this->partNames []= $elem1437; + $_size1446 = 0; + $_etype1449 = 0; + $xfer += $input->readListBegin($_etype1449, $_size1446); + for ($_i1450 = 0; $_i1450 < $_size1446; ++$_i1450) { + $elem1451 = null; + $xfer += $input->readString($elem1451); + $this->partNames []= $elem1451; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partNames', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->partNames)); - foreach ($this->partNames as $iter1438) { - $xfer += $output->writeString($iter1438); + foreach ($this->partNames as $iter1452) { + $xfer += $output->writeString($iter1452); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMFullResourcePlan.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMFullResourcePlan.php index ee5d57a12795..7f9a416a5fd7 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMFullResourcePlan.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMFullResourcePlan.php @@ -141,14 +141,14 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->pools = array(); - $_size1071 = 0; - $_etype1074 = 0; - $xfer += $input->readListBegin($_etype1074, $_size1071); - for ($_i1075 = 0; $_i1075 < $_size1071; ++$_i1075) { - $elem1076 = null; - $elem1076 = new \metastore\WMPool(); - $xfer += $elem1076->read($input); - $this->pools []= $elem1076; + $_size1085 = 0; + $_etype1088 = 0; + $xfer += $input->readListBegin($_etype1088, $_size1085); + for ($_i1089 = 0; $_i1089 < $_size1085; ++$_i1089) { + $elem1090 = null; + $elem1090 = new \metastore\WMPool(); + $xfer += $elem1090->read($input); + $this->pools []= $elem1090; } $xfer += $input->readListEnd(); } else { @@ -158,14 +158,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->mappings = array(); - $_size1077 = 0; - $_etype1080 = 0; - $xfer += $input->readListBegin($_etype1080, $_size1077); - for ($_i1081 = 0; $_i1081 < $_size1077; ++$_i1081) { - $elem1082 = null; - $elem1082 = new \metastore\WMMapping(); - $xfer += $elem1082->read($input); - $this->mappings []= $elem1082; + $_size1091 = 0; + $_etype1094 = 0; + $xfer += $input->readListBegin($_etype1094, $_size1091); + for ($_i1095 = 0; $_i1095 < $_size1091; ++$_i1095) { + $elem1096 = null; + $elem1096 = new \metastore\WMMapping(); + $xfer += $elem1096->read($input); + $this->mappings []= $elem1096; } $xfer += $input->readListEnd(); } else { @@ -175,14 +175,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->triggers = array(); - $_size1083 = 0; - $_etype1086 = 0; - $xfer += $input->readListBegin($_etype1086, $_size1083); - for ($_i1087 = 0; $_i1087 < $_size1083; ++$_i1087) { - $elem1088 = null; - $elem1088 = new \metastore\WMTrigger(); - $xfer += $elem1088->read($input); - $this->triggers []= $elem1088; + $_size1097 = 0; + $_etype1100 = 0; + $xfer += $input->readListBegin($_etype1100, $_size1097); + for ($_i1101 = 0; $_i1101 < $_size1097; ++$_i1101) { + $elem1102 = null; + $elem1102 = new \metastore\WMTrigger(); + $xfer += $elem1102->read($input); + $this->triggers []= $elem1102; } $xfer += $input->readListEnd(); } else { @@ -192,14 +192,14 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->poolTriggers = array(); - $_size1089 = 0; - $_etype1092 = 0; - $xfer += $input->readListBegin($_etype1092, $_size1089); - for ($_i1093 = 0; $_i1093 < $_size1089; ++$_i1093) { - $elem1094 = null; - $elem1094 = new \metastore\WMPoolTrigger(); - $xfer += $elem1094->read($input); - $this->poolTriggers []= $elem1094; + $_size1103 = 0; + $_etype1106 = 0; + $xfer += $input->readListBegin($_etype1106, $_size1103); + for ($_i1107 = 0; $_i1107 < $_size1103; ++$_i1107) { + $elem1108 = null; + $elem1108 = new \metastore\WMPoolTrigger(); + $xfer += $elem1108->read($input); + $this->poolTriggers []= $elem1108; } $xfer += $input->readListEnd(); } else { @@ -234,8 +234,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('pools', TType::LST, 2); $output->writeListBegin(TType::STRUCT, count($this->pools)); - foreach ($this->pools as $iter1095) { - $xfer += $iter1095->write($output); + foreach ($this->pools as $iter1109) { + $xfer += $iter1109->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -246,8 +246,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('mappings', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->mappings)); - foreach ($this->mappings as $iter1096) { - $xfer += $iter1096->write($output); + foreach ($this->mappings as $iter1110) { + $xfer += $iter1110->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -258,8 +258,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('triggers', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->triggers)); - foreach ($this->triggers as $iter1097) { - $xfer += $iter1097->write($output); + foreach ($this->triggers as $iter1111) { + $xfer += $iter1111->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -270,8 +270,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('poolTriggers', TType::LST, 5); $output->writeListBegin(TType::STRUCT, count($this->poolTriggers)); - foreach ($this->poolTriggers as $iter1098) { - $xfer += $iter1098->write($output); + foreach ($this->poolTriggers as $iter1112) { + $xfer += $iter1112->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetAllResourcePlanResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetAllResourcePlanResponse.php index 9e6573bff1be..0cfb8c311daa 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetAllResourcePlanResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetAllResourcePlanResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->resourcePlans = array(); - $_size1099 = 0; - $_etype1102 = 0; - $xfer += $input->readListBegin($_etype1102, $_size1099); - for ($_i1103 = 0; $_i1103 < $_size1099; ++$_i1103) { - $elem1104 = null; - $elem1104 = new \metastore\WMResourcePlan(); - $xfer += $elem1104->read($input); - $this->resourcePlans []= $elem1104; + $_size1113 = 0; + $_etype1116 = 0; + $xfer += $input->readListBegin($_etype1116, $_size1113); + for ($_i1117 = 0; $_i1117 < $_size1113; ++$_i1117) { + $elem1118 = null; + $elem1118 = new \metastore\WMResourcePlan(); + $xfer += $elem1118->read($input); + $this->resourcePlans []= $elem1118; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('resourcePlans', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->resourcePlans)); - foreach ($this->resourcePlans as $iter1105) { - $xfer += $iter1105->write($output); + foreach ($this->resourcePlans as $iter1119) { + $xfer += $iter1119->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetTriggersForResourePlanResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetTriggersForResourePlanResponse.php index a77e72da03b5..e3312777d7f8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetTriggersForResourePlanResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMGetTriggersForResourePlanResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->triggers = array(); - $_size1120 = 0; - $_etype1123 = 0; - $xfer += $input->readListBegin($_etype1123, $_size1120); - for ($_i1124 = 0; $_i1124 < $_size1120; ++$_i1124) { - $elem1125 = null; - $elem1125 = new \metastore\WMTrigger(); - $xfer += $elem1125->read($input); - $this->triggers []= $elem1125; + $_size1134 = 0; + $_etype1137 = 0; + $xfer += $input->readListBegin($_etype1137, $_size1134); + for ($_i1138 = 0; $_i1138 < $_size1134; ++$_i1138) { + $elem1139 = null; + $elem1139 = new \metastore\WMTrigger(); + $xfer += $elem1139->read($input); + $this->triggers []= $elem1139; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('triggers', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->triggers)); - foreach ($this->triggers as $iter1126) { - $xfer += $iter1126->write($output); + foreach ($this->triggers as $iter1140) { + $xfer += $iter1140->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMValidateResourcePlanResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMValidateResourcePlanResponse.php index 2ccf020b49be..76b6474e8ecc 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMValidateResourcePlanResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WMValidateResourcePlanResponse.php @@ -84,13 +84,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->errors = array(); - $_size1106 = 0; - $_etype1109 = 0; - $xfer += $input->readListBegin($_etype1109, $_size1106); - for ($_i1110 = 0; $_i1110 < $_size1106; ++$_i1110) { - $elem1111 = null; - $xfer += $input->readString($elem1111); - $this->errors []= $elem1111; + $_size1120 = 0; + $_etype1123 = 0; + $xfer += $input->readListBegin($_etype1123, $_size1120); + for ($_i1124 = 0; $_i1124 < $_size1120; ++$_i1124) { + $elem1125 = null; + $xfer += $input->readString($elem1125); + $this->errors []= $elem1125; } $xfer += $input->readListEnd(); } else { @@ -100,13 +100,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->warnings = array(); - $_size1112 = 0; - $_etype1115 = 0; - $xfer += $input->readListBegin($_etype1115, $_size1112); - for ($_i1116 = 0; $_i1116 < $_size1112; ++$_i1116) { - $elem1117 = null; - $xfer += $input->readString($elem1117); - $this->warnings []= $elem1117; + $_size1126 = 0; + $_etype1129 = 0; + $xfer += $input->readListBegin($_etype1129, $_size1126); + for ($_i1130 = 0; $_i1130 < $_size1126; ++$_i1130) { + $elem1131 = null; + $xfer += $input->readString($elem1131); + $this->warnings []= $elem1131; } $xfer += $input->readListEnd(); } else { @@ -133,8 +133,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('errors', TType::LST, 1); $output->writeListBegin(TType::STRING, count($this->errors)); - foreach ($this->errors as $iter1118) { - $xfer += $output->writeString($iter1118); + foreach ($this->errors as $iter1132) { + $xfer += $output->writeString($iter1132); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('warnings', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->warnings)); - foreach ($this->warnings as $iter1119) { - $xfer += $output->writeString($iter1119); + foreach ($this->warnings as $iter1133) { + $xfer += $output->writeString($iter1133); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogBatchRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogBatchRequest.php index d07ff4d62af4..09aa9f382c1a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogBatchRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogBatchRequest.php @@ -126,14 +126,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->requestList = array(); - $_size934 = 0; - $_etype937 = 0; - $xfer += $input->readListBegin($_etype937, $_size934); - for ($_i938 = 0; $_i938 < $_size934; ++$_i938) { - $elem939 = null; - $elem939 = new \metastore\WriteNotificationLogRequest(); - $xfer += $elem939->read($input); - $this->requestList []= $elem939; + $_size948 = 0; + $_etype951 = 0; + $xfer += $input->readListBegin($_etype951, $_size948); + for ($_i952 = 0; $_i952 < $_size948; ++$_i952) { + $elem953 = null; + $elem953 = new \metastore\WriteNotificationLogRequest(); + $xfer += $elem953->read($input); + $this->requestList []= $elem953; } $xfer += $input->readListEnd(); } else { @@ -175,8 +175,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('requestList', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->requestList)); - foreach ($this->requestList as $iter940) { - $xfer += $iter940->write($output); + foreach ($this->requestList as $iter954) { + $xfer += $iter954->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogRequest.php index fe7910b34465..9d043c0ce917 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/WriteNotificationLogRequest.php @@ -165,13 +165,13 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->partitionVals = array(); - $_size927 = 0; - $_etype930 = 0; - $xfer += $input->readListBegin($_etype930, $_size927); - for ($_i931 = 0; $_i931 < $_size927; ++$_i931) { - $elem932 = null; - $xfer += $input->readString($elem932); - $this->partitionVals []= $elem932; + $_size941 = 0; + $_etype944 = 0; + $xfer += $input->readListBegin($_etype944, $_size941); + for ($_i945 = 0; $_i945 < $_size941; ++$_i945) { + $elem946 = null; + $xfer += $input->readString($elem946); + $this->partitionVals []= $elem946; } $xfer += $input->readListEnd(); } else { @@ -226,8 +226,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 6); $output->writeListBegin(TType::STRING, count($this->partitionVals)); - foreach ($this->partitionVals as $iter933) { - $xfer += $output->writeString($iter933); + foreach ($this->partitionVals as $iter947) { + $xfer += $output->writeString($iter947); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py index 5ebe814fee78..fd7009c45ca5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py @@ -22459,10 +22459,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1332, _size1329) = iprot.readListBegin() - for _i1333 in range(_size1329): - _elem1334 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1334) + (_etype1346, _size1343) = iprot.readListBegin() + for _i1347 in range(_size1343): + _elem1348 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1348) iprot.readListEnd() else: iprot.skip(ftype) @@ -22484,8 +22484,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1335 in self.success: - oprot.writeString(iter1335.encode('utf-8') if sys.version_info[0] == 2 else iter1335) + for iter1349 in self.success: + oprot.writeString(iter1349.encode('utf-8') if sys.version_info[0] == 2 else iter1349) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -22583,10 +22583,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1339, _size1336) = iprot.readListBegin() - for _i1340 in range(_size1336): - _elem1341 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1341) + (_etype1353, _size1350) = iprot.readListBegin() + for _i1354 in range(_size1350): + _elem1355 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1355) iprot.readListEnd() else: iprot.skip(ftype) @@ -22608,8 +22608,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1342 in self.success: - oprot.writeString(iter1342.encode('utf-8') if sys.version_info[0] == 2 else iter1342) + for iter1356 in self.success: + oprot.writeString(iter1356.encode('utf-8') if sys.version_info[0] == 2 else iter1356) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -23326,10 +23326,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1346, _size1343) = iprot.readListBegin() - for _i1347 in range(_size1343): - _elem1348 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1348) + (_etype1360, _size1357) = iprot.readListBegin() + for _i1361 in range(_size1357): + _elem1362 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1362) iprot.readListEnd() else: iprot.skip(ftype) @@ -23351,8 +23351,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1349 in self.success: - oprot.writeString(iter1349.encode('utf-8') if sys.version_info[0] == 2 else iter1349) + for iter1363 in self.success: + oprot.writeString(iter1363.encode('utf-8') if sys.version_info[0] == 2 else iter1363) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24073,12 +24073,12 @@ def read(self, iprot): if fid == 0: if ftype == TType.MAP: self.success = {} - (_ktype1351, _vtype1352, _size1350) = iprot.readMapBegin() - for _i1354 in range(_size1350): - _key1355 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1356 = Type() - _val1356.read(iprot) - self.success[_key1355] = _val1356 + (_ktype1365, _vtype1366, _size1364) = iprot.readMapBegin() + for _i1368 in range(_size1364): + _key1369 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1370 = Type() + _val1370.read(iprot) + self.success[_key1369] = _val1370 iprot.readMapEnd() else: iprot.skip(ftype) @@ -24100,9 +24100,9 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.MAP, 0) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success)) - for kiter1357, viter1358 in self.success.items(): - oprot.writeString(kiter1357.encode('utf-8') if sys.version_info[0] == 2 else kiter1357) - viter1358.write(oprot) + for kiter1371, viter1372 in self.success.items(): + oprot.writeString(kiter1371.encode('utf-8') if sys.version_info[0] == 2 else kiter1371) + viter1372.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.o2 is not None: @@ -24235,11 +24235,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1362, _size1359) = iprot.readListBegin() - for _i1363 in range(_size1359): - _elem1364 = FieldSchema() - _elem1364.read(iprot) - self.success.append(_elem1364) + (_etype1376, _size1373) = iprot.readListBegin() + for _i1377 in range(_size1373): + _elem1378 = FieldSchema() + _elem1378.read(iprot) + self.success.append(_elem1378) iprot.readListEnd() else: iprot.skip(ftype) @@ -24271,8 +24271,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1365 in self.success: - iter1365.write(oprot) + for iter1379 in self.success: + iter1379.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24428,11 +24428,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1369, _size1366) = iprot.readListBegin() - for _i1370 in range(_size1366): - _elem1371 = FieldSchema() - _elem1371.read(iprot) - self.success.append(_elem1371) + (_etype1383, _size1380) = iprot.readListBegin() + for _i1384 in range(_size1380): + _elem1385 = FieldSchema() + _elem1385.read(iprot) + self.success.append(_elem1385) iprot.readListEnd() else: iprot.skip(ftype) @@ -24464,8 +24464,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1372 in self.success: - iter1372.write(oprot) + for iter1386 in self.success: + iter1386.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24769,11 +24769,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1376, _size1373) = iprot.readListBegin() - for _i1377 in range(_size1373): - _elem1378 = FieldSchema() - _elem1378.read(iprot) - self.success.append(_elem1378) + (_etype1390, _size1387) = iprot.readListBegin() + for _i1391 in range(_size1387): + _elem1392 = FieldSchema() + _elem1392.read(iprot) + self.success.append(_elem1392) iprot.readListEnd() else: iprot.skip(ftype) @@ -24805,8 +24805,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1379 in self.success: - iter1379.write(oprot) + for iter1393 in self.success: + iter1393.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24962,11 +24962,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1383, _size1380) = iprot.readListBegin() - for _i1384 in range(_size1380): - _elem1385 = FieldSchema() - _elem1385.read(iprot) - self.success.append(_elem1385) + (_etype1397, _size1394) = iprot.readListBegin() + for _i1398 in range(_size1394): + _elem1399 = FieldSchema() + _elem1399.read(iprot) + self.success.append(_elem1399) iprot.readListEnd() else: iprot.skip(ftype) @@ -24998,8 +24998,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1386 in self.success: - iter1386.write(oprot) + for iter1400 in self.success: + iter1400.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -25576,66 +25576,66 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.primaryKeys = [] - (_etype1390, _size1387) = iprot.readListBegin() - for _i1391 in range(_size1387): - _elem1392 = SQLPrimaryKey() - _elem1392.read(iprot) - self.primaryKeys.append(_elem1392) + (_etype1404, _size1401) = iprot.readListBegin() + for _i1405 in range(_size1401): + _elem1406 = SQLPrimaryKey() + _elem1406.read(iprot) + self.primaryKeys.append(_elem1406) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: self.foreignKeys = [] - (_etype1396, _size1393) = iprot.readListBegin() - for _i1397 in range(_size1393): - _elem1398 = SQLForeignKey() - _elem1398.read(iprot) - self.foreignKeys.append(_elem1398) + (_etype1410, _size1407) = iprot.readListBegin() + for _i1411 in range(_size1407): + _elem1412 = SQLForeignKey() + _elem1412.read(iprot) + self.foreignKeys.append(_elem1412) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.uniqueConstraints = [] - (_etype1402, _size1399) = iprot.readListBegin() - for _i1403 in range(_size1399): - _elem1404 = SQLUniqueConstraint() - _elem1404.read(iprot) - self.uniqueConstraints.append(_elem1404) + (_etype1416, _size1413) = iprot.readListBegin() + for _i1417 in range(_size1413): + _elem1418 = SQLUniqueConstraint() + _elem1418.read(iprot) + self.uniqueConstraints.append(_elem1418) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.LIST: self.notNullConstraints = [] - (_etype1408, _size1405) = iprot.readListBegin() - for _i1409 in range(_size1405): - _elem1410 = SQLNotNullConstraint() - _elem1410.read(iprot) - self.notNullConstraints.append(_elem1410) + (_etype1422, _size1419) = iprot.readListBegin() + for _i1423 in range(_size1419): + _elem1424 = SQLNotNullConstraint() + _elem1424.read(iprot) + self.notNullConstraints.append(_elem1424) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 6: if ftype == TType.LIST: self.defaultConstraints = [] - (_etype1414, _size1411) = iprot.readListBegin() - for _i1415 in range(_size1411): - _elem1416 = SQLDefaultConstraint() - _elem1416.read(iprot) - self.defaultConstraints.append(_elem1416) + (_etype1428, _size1425) = iprot.readListBegin() + for _i1429 in range(_size1425): + _elem1430 = SQLDefaultConstraint() + _elem1430.read(iprot) + self.defaultConstraints.append(_elem1430) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 7: if ftype == TType.LIST: self.checkConstraints = [] - (_etype1420, _size1417) = iprot.readListBegin() - for _i1421 in range(_size1417): - _elem1422 = SQLCheckConstraint() - _elem1422.read(iprot) - self.checkConstraints.append(_elem1422) + (_etype1434, _size1431) = iprot.readListBegin() + for _i1435 in range(_size1431): + _elem1436 = SQLCheckConstraint() + _elem1436.read(iprot) + self.checkConstraints.append(_elem1436) iprot.readListEnd() else: iprot.skip(ftype) @@ -25656,43 +25656,43 @@ def write(self, oprot): if self.primaryKeys is not None: oprot.writeFieldBegin('primaryKeys', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys)) - for iter1423 in self.primaryKeys: - iter1423.write(oprot) + for iter1437 in self.primaryKeys: + iter1437.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.foreignKeys is not None: oprot.writeFieldBegin('foreignKeys', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys)) - for iter1424 in self.foreignKeys: - iter1424.write(oprot) + for iter1438 in self.foreignKeys: + iter1438.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.uniqueConstraints is not None: oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints)) - for iter1425 in self.uniqueConstraints: - iter1425.write(oprot) + for iter1439 in self.uniqueConstraints: + iter1439.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.notNullConstraints is not None: oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5) oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints)) - for iter1426 in self.notNullConstraints: - iter1426.write(oprot) + for iter1440 in self.notNullConstraints: + iter1440.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.defaultConstraints is not None: oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6) oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints)) - for iter1427 in self.defaultConstraints: - iter1427.write(oprot) + for iter1441 in self.defaultConstraints: + iter1441.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.checkConstraints is not None: oprot.writeFieldBegin('checkConstraints', TType.LIST, 7) oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints)) - for iter1428 in self.checkConstraints: - iter1428.write(oprot) + for iter1442 in self.checkConstraints: + iter1442.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27485,10 +27485,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.partNames = [] - (_etype1432, _size1429) = iprot.readListBegin() - for _i1433 in range(_size1429): - _elem1434 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partNames.append(_elem1434) + (_etype1446, _size1443) = iprot.readListBegin() + for _i1447 in range(_size1443): + _elem1448 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partNames.append(_elem1448) iprot.readListEnd() else: iprot.skip(ftype) @@ -27513,8 +27513,8 @@ def write(self, oprot): if self.partNames is not None: oprot.writeFieldBegin('partNames', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.partNames)) - for iter1435 in self.partNames: - oprot.writeString(iter1435.encode('utf-8') if sys.version_info[0] == 2 else iter1435) + for iter1449 in self.partNames: + oprot.writeString(iter1449.encode('utf-8') if sys.version_info[0] == 2 else iter1449) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27840,10 +27840,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1439, _size1436) = iprot.readListBegin() - for _i1440 in range(_size1436): - _elem1441 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1441) + (_etype1453, _size1450) = iprot.readListBegin() + for _i1454 in range(_size1450): + _elem1455 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1455) iprot.readListEnd() else: iprot.skip(ftype) @@ -27865,8 +27865,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1442 in self.success: - oprot.writeString(iter1442.encode('utf-8') if sys.version_info[0] == 2 else iter1442) + for iter1456 in self.success: + oprot.writeString(iter1456.encode('utf-8') if sys.version_info[0] == 2 else iter1456) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28007,10 +28007,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1446, _size1443) = iprot.readListBegin() - for _i1447 in range(_size1443): - _elem1448 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1448) + (_etype1460, _size1457) = iprot.readListBegin() + for _i1461 in range(_size1457): + _elem1462 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1462) iprot.readListEnd() else: iprot.skip(ftype) @@ -28032,8 +28032,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1449 in self.success: - oprot.writeString(iter1449.encode('utf-8') if sys.version_info[0] == 2 else iter1449) + for iter1463 in self.success: + oprot.writeString(iter1463.encode('utf-8') if sys.version_info[0] == 2 else iter1463) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28131,11 +28131,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1453, _size1450) = iprot.readListBegin() - for _i1454 in range(_size1450): - _elem1455 = Table() - _elem1455.read(iprot) - self.success.append(_elem1455) + (_etype1467, _size1464) = iprot.readListBegin() + for _i1468 in range(_size1464): + _elem1469 = Table() + _elem1469.read(iprot) + self.success.append(_elem1469) iprot.readListEnd() else: iprot.skip(ftype) @@ -28157,8 +28157,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1456 in self.success: - iter1456.write(oprot) + for iter1470 in self.success: + iter1470.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28275,10 +28275,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1460, _size1457) = iprot.readListBegin() - for _i1461 in range(_size1457): - _elem1462 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1462) + (_etype1474, _size1471) = iprot.readListBegin() + for _i1475 in range(_size1471): + _elem1476 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1476) iprot.readListEnd() else: iprot.skip(ftype) @@ -28300,8 +28300,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1463 in self.success: - oprot.writeString(iter1463.encode('utf-8') if sys.version_info[0] == 2 else iter1463) + for iter1477 in self.success: + oprot.writeString(iter1477.encode('utf-8') if sys.version_info[0] == 2 else iter1477) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28368,10 +28368,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.tbl_types = [] - (_etype1467, _size1464) = iprot.readListBegin() - for _i1468 in range(_size1464): - _elem1469 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.tbl_types.append(_elem1469) + (_etype1481, _size1478) = iprot.readListBegin() + for _i1482 in range(_size1478): + _elem1483 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tbl_types.append(_elem1483) iprot.readListEnd() else: iprot.skip(ftype) @@ -28396,8 +28396,8 @@ def write(self, oprot): if self.tbl_types is not None: oprot.writeFieldBegin('tbl_types', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.tbl_types)) - for iter1470 in self.tbl_types: - oprot.writeString(iter1470.encode('utf-8') if sys.version_info[0] == 2 else iter1470) + for iter1484 in self.tbl_types: + oprot.writeString(iter1484.encode('utf-8') if sys.version_info[0] == 2 else iter1484) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28450,11 +28450,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1474, _size1471) = iprot.readListBegin() - for _i1475 in range(_size1471): - _elem1476 = TableMeta() - _elem1476.read(iprot) - self.success.append(_elem1476) + (_etype1488, _size1485) = iprot.readListBegin() + for _i1489 in range(_size1485): + _elem1490 = TableMeta() + _elem1490.read(iprot) + self.success.append(_elem1490) iprot.readListEnd() else: iprot.skip(ftype) @@ -28476,8 +28476,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1477 in self.success: - iter1477.write(oprot) + for iter1491 in self.success: + iter1491.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28594,10 +28594,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1481, _size1478) = iprot.readListBegin() - for _i1482 in range(_size1478): - _elem1483 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1483) + (_etype1495, _size1492) = iprot.readListBegin() + for _i1496 in range(_size1492): + _elem1497 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1497) iprot.readListEnd() else: iprot.skip(ftype) @@ -28619,8 +28619,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1484 in self.success: - oprot.writeString(iter1484.encode('utf-8') if sys.version_info[0] == 2 else iter1484) + for iter1498 in self.success: + oprot.writeString(iter1498.encode('utf-8') if sys.version_info[0] == 2 else iter1498) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28840,10 +28840,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.tbl_names = [] - (_etype1488, _size1485) = iprot.readListBegin() - for _i1489 in range(_size1485): - _elem1490 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.tbl_names.append(_elem1490) + (_etype1502, _size1499) = iprot.readListBegin() + for _i1503 in range(_size1499): + _elem1504 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tbl_names.append(_elem1504) iprot.readListEnd() else: iprot.skip(ftype) @@ -28864,8 +28864,8 @@ def write(self, oprot): if self.tbl_names is not None: oprot.writeFieldBegin('tbl_names', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.tbl_names)) - for iter1491 in self.tbl_names: - oprot.writeString(iter1491.encode('utf-8') if sys.version_info[0] == 2 else iter1491) + for iter1505 in self.tbl_names: + oprot.writeString(iter1505.encode('utf-8') if sys.version_info[0] == 2 else iter1505) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28915,11 +28915,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1495, _size1492) = iprot.readListBegin() - for _i1496 in range(_size1492): - _elem1497 = Table() - _elem1497.read(iprot) - self.success.append(_elem1497) + (_etype1509, _size1506) = iprot.readListBegin() + for _i1510 in range(_size1506): + _elem1511 = Table() + _elem1511.read(iprot) + self.success.append(_elem1511) iprot.readListEnd() else: iprot.skip(ftype) @@ -28936,8 +28936,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1498 in self.success: - iter1498.write(oprot) + for iter1512 in self.success: + iter1512.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -29050,11 +29050,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1502, _size1499) = iprot.readListBegin() - for _i1503 in range(_size1499): - _elem1504 = ExtendedTableInfo() - _elem1504.read(iprot) - self.success.append(_elem1504) + (_etype1516, _size1513) = iprot.readListBegin() + for _i1517 in range(_size1513): + _elem1518 = ExtendedTableInfo() + _elem1518.read(iprot) + self.success.append(_elem1518) iprot.readListEnd() else: iprot.skip(ftype) @@ -29076,8 +29076,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1505 in self.success: - iter1505.write(oprot) + for iter1519 in self.success: + iter1519.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -29890,10 +29890,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1509, _size1506) = iprot.readListBegin() - for _i1510 in range(_size1506): - _elem1511 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1511) + (_etype1523, _size1520) = iprot.readListBegin() + for _i1524 in range(_size1520): + _elem1525 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1525) iprot.readListEnd() else: iprot.skip(ftype) @@ -29925,8 +29925,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1512 in self.success: - oprot.writeString(iter1512.encode('utf-8') if sys.version_info[0] == 2 else iter1512) + for iter1526 in self.success: + oprot.writeString(iter1526.encode('utf-8') if sys.version_info[0] == 2 else iter1526) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -30981,11 +30981,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.new_parts = [] - (_etype1516, _size1513) = iprot.readListBegin() - for _i1517 in range(_size1513): - _elem1518 = Partition() - _elem1518.read(iprot) - self.new_parts.append(_elem1518) + (_etype1530, _size1527) = iprot.readListBegin() + for _i1531 in range(_size1527): + _elem1532 = Partition() + _elem1532.read(iprot) + self.new_parts.append(_elem1532) iprot.readListEnd() else: iprot.skip(ftype) @@ -31002,8 +31002,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1519 in self.new_parts: - iter1519.write(oprot) + for iter1533 in self.new_parts: + iter1533.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -31149,11 +31149,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.new_parts = [] - (_etype1523, _size1520) = iprot.readListBegin() - for _i1524 in range(_size1520): - _elem1525 = PartitionSpec() - _elem1525.read(iprot) - self.new_parts.append(_elem1525) + (_etype1537, _size1534) = iprot.readListBegin() + for _i1538 in range(_size1534): + _elem1539 = PartitionSpec() + _elem1539.read(iprot) + self.new_parts.append(_elem1539) iprot.readListEnd() else: iprot.skip(ftype) @@ -31170,8 +31170,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1526 in self.new_parts: - iter1526.write(oprot) + for iter1540 in self.new_parts: + iter1540.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -31331,10 +31331,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1530, _size1527) = iprot.readListBegin() - for _i1531 in range(_size1527): - _elem1532 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1532) + (_etype1544, _size1541) = iprot.readListBegin() + for _i1545 in range(_size1541): + _elem1546 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1546) iprot.readListEnd() else: iprot.skip(ftype) @@ -31359,8 +31359,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1533 in self.part_vals: - oprot.writeString(iter1533.encode('utf-8') if sys.version_info[0] == 2 else iter1533) + for iter1547 in self.part_vals: + oprot.writeString(iter1547.encode('utf-8') if sys.version_info[0] == 2 else iter1547) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -31686,10 +31686,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1537, _size1534) = iprot.readListBegin() - for _i1538 in range(_size1534): - _elem1539 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1539) + (_etype1551, _size1548) = iprot.readListBegin() + for _i1552 in range(_size1548): + _elem1553 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1553) iprot.readListEnd() else: iprot.skip(ftype) @@ -31720,8 +31720,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1540 in self.part_vals: - oprot.writeString(iter1540.encode('utf-8') if sys.version_info[0] == 2 else iter1540) + for iter1554 in self.part_vals: + oprot.writeString(iter1554.encode('utf-8') if sys.version_info[0] == 2 else iter1554) oprot.writeListEnd() oprot.writeFieldEnd() if self.environment_context is not None: @@ -32272,10 +32272,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1544, _size1541) = iprot.readListBegin() - for _i1545 in range(_size1541): - _elem1546 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1546) + (_etype1558, _size1555) = iprot.readListBegin() + for _i1559 in range(_size1555): + _elem1560 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1560) iprot.readListEnd() else: iprot.skip(ftype) @@ -32305,8 +32305,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1547 in self.part_vals: - oprot.writeString(iter1547.encode('utf-8') if sys.version_info[0] == 2 else iter1547) + for iter1561 in self.part_vals: + oprot.writeString(iter1561.encode('utf-8') if sys.version_info[0] == 2 else iter1561) oprot.writeListEnd() oprot.writeFieldEnd() if self.deleteData is not None: @@ -32465,10 +32465,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1551, _size1548) = iprot.readListBegin() - for _i1552 in range(_size1548): - _elem1553 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1553) + (_etype1565, _size1562) = iprot.readListBegin() + for _i1566 in range(_size1562): + _elem1567 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1567) iprot.readListEnd() else: iprot.skip(ftype) @@ -32504,8 +32504,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1554 in self.part_vals: - oprot.writeString(iter1554.encode('utf-8') if sys.version_info[0] == 2 else iter1554) + for iter1568 in self.part_vals: + oprot.writeString(iter1568.encode('utf-8') if sys.version_info[0] == 2 else iter1568) oprot.writeListEnd() oprot.writeFieldEnd() if self.deleteData is not None: @@ -33193,10 +33193,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1558, _size1555) = iprot.readListBegin() - for _i1559 in range(_size1555): - _elem1560 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1560) + (_etype1572, _size1569) = iprot.readListBegin() + for _i1573 in range(_size1569): + _elem1574 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1574) iprot.readListEnd() else: iprot.skip(ftype) @@ -33221,8 +33221,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1561 in self.part_vals: - oprot.writeString(iter1561.encode('utf-8') if sys.version_info[0] == 2 else iter1561) + for iter1575 in self.part_vals: + oprot.writeString(iter1575.encode('utf-8') if sys.version_info[0] == 2 else iter1575) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -33516,11 +33516,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.partitionSpecs = {} - (_ktype1563, _vtype1564, _size1562) = iprot.readMapBegin() - for _i1566 in range(_size1562): - _key1567 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1568 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionSpecs[_key1567] = _val1568 + (_ktype1577, _vtype1578, _size1576) = iprot.readMapBegin() + for _i1580 in range(_size1576): + _key1581 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1582 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionSpecs[_key1581] = _val1582 iprot.readMapEnd() else: iprot.skip(ftype) @@ -33557,9 +33557,9 @@ def write(self, oprot): if self.partitionSpecs is not None: oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) - for kiter1569, viter1570 in self.partitionSpecs.items(): - oprot.writeString(kiter1569.encode('utf-8') if sys.version_info[0] == 2 else kiter1569) - oprot.writeString(viter1570.encode('utf-8') if sys.version_info[0] == 2 else viter1570) + for kiter1583, viter1584 in self.partitionSpecs.items(): + oprot.writeString(kiter1583.encode('utf-8') if sys.version_info[0] == 2 else kiter1583) + oprot.writeString(viter1584.encode('utf-8') if sys.version_info[0] == 2 else viter1584) oprot.writeMapEnd() oprot.writeFieldEnd() if self.source_db is not None: @@ -33746,11 +33746,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.partitionSpecs = {} - (_ktype1572, _vtype1573, _size1571) = iprot.readMapBegin() - for _i1575 in range(_size1571): - _key1576 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1577 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionSpecs[_key1576] = _val1577 + (_ktype1586, _vtype1587, _size1585) = iprot.readMapBegin() + for _i1589 in range(_size1585): + _key1590 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1591 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionSpecs[_key1590] = _val1591 iprot.readMapEnd() else: iprot.skip(ftype) @@ -33787,9 +33787,9 @@ def write(self, oprot): if self.partitionSpecs is not None: oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) - for kiter1578, viter1579 in self.partitionSpecs.items(): - oprot.writeString(kiter1578.encode('utf-8') if sys.version_info[0] == 2 else kiter1578) - oprot.writeString(viter1579.encode('utf-8') if sys.version_info[0] == 2 else viter1579) + for kiter1592, viter1593 in self.partitionSpecs.items(): + oprot.writeString(kiter1592.encode('utf-8') if sys.version_info[0] == 2 else kiter1592) + oprot.writeString(viter1593.encode('utf-8') if sys.version_info[0] == 2 else viter1593) oprot.writeMapEnd() oprot.writeFieldEnd() if self.source_db is not None: @@ -33866,11 +33866,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1583, _size1580) = iprot.readListBegin() - for _i1584 in range(_size1580): - _elem1585 = Partition() - _elem1585.read(iprot) - self.success.append(_elem1585) + (_etype1597, _size1594) = iprot.readListBegin() + for _i1598 in range(_size1594): + _elem1599 = Partition() + _elem1599.read(iprot) + self.success.append(_elem1599) iprot.readListEnd() else: iprot.skip(ftype) @@ -33907,8 +33907,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1586 in self.success: - iter1586.write(oprot) + for iter1600 in self.success: + iter1600.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -33994,10 +33994,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1590, _size1587) = iprot.readListBegin() - for _i1591 in range(_size1587): - _elem1592 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1592) + (_etype1604, _size1601) = iprot.readListBegin() + for _i1605 in range(_size1601): + _elem1606 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1606) iprot.readListEnd() else: iprot.skip(ftype) @@ -34009,10 +34009,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.group_names = [] - (_etype1596, _size1593) = iprot.readListBegin() - for _i1597 in range(_size1593): - _elem1598 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1598) + (_etype1610, _size1607) = iprot.readListBegin() + for _i1611 in range(_size1607): + _elem1612 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1612) iprot.readListEnd() else: iprot.skip(ftype) @@ -34037,8 +34037,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1599 in self.part_vals: - oprot.writeString(iter1599.encode('utf-8') if sys.version_info[0] == 2 else iter1599) + for iter1613 in self.part_vals: + oprot.writeString(iter1613.encode('utf-8') if sys.version_info[0] == 2 else iter1613) oprot.writeListEnd() oprot.writeFieldEnd() if self.user_name is not None: @@ -34048,8 +34048,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1600 in self.group_names: - oprot.writeString(iter1600.encode('utf-8') if sys.version_info[0] == 2 else iter1600) + for iter1614 in self.group_names: + oprot.writeString(iter1614.encode('utf-8') if sys.version_info[0] == 2 else iter1614) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -34450,11 +34450,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1604, _size1601) = iprot.readListBegin() - for _i1605 in range(_size1601): - _elem1606 = Partition() - _elem1606.read(iprot) - self.success.append(_elem1606) + (_etype1618, _size1615) = iprot.readListBegin() + for _i1619 in range(_size1615): + _elem1620 = Partition() + _elem1620.read(iprot) + self.success.append(_elem1620) iprot.readListEnd() else: iprot.skip(ftype) @@ -34481,8 +34481,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1607 in self.success: - iter1607.write(oprot) + for iter1621 in self.success: + iter1621.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -34717,10 +34717,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.group_names = [] - (_etype1611, _size1608) = iprot.readListBegin() - for _i1612 in range(_size1608): - _elem1613 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1613) + (_etype1625, _size1622) = iprot.readListBegin() + for _i1626 in range(_size1622): + _elem1627 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1627) iprot.readListEnd() else: iprot.skip(ftype) @@ -34753,8 +34753,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1614 in self.group_names: - oprot.writeString(iter1614.encode('utf-8') if sys.version_info[0] == 2 else iter1614) + for iter1628 in self.group_names: + oprot.writeString(iter1628.encode('utf-8') if sys.version_info[0] == 2 else iter1628) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -34811,11 +34811,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1618, _size1615) = iprot.readListBegin() - for _i1619 in range(_size1615): - _elem1620 = Partition() - _elem1620.read(iprot) - self.success.append(_elem1620) + (_etype1632, _size1629) = iprot.readListBegin() + for _i1633 in range(_size1629): + _elem1634 = Partition() + _elem1634.read(iprot) + self.success.append(_elem1634) iprot.readListEnd() else: iprot.skip(ftype) @@ -34842,8 +34842,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1621 in self.success: - iter1621.write(oprot) + for iter1635 in self.success: + iter1635.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -34991,11 +34991,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1625, _size1622) = iprot.readListBegin() - for _i1626 in range(_size1622): - _elem1627 = PartitionSpec() - _elem1627.read(iprot) - self.success.append(_elem1627) + (_etype1639, _size1636) = iprot.readListBegin() + for _i1640 in range(_size1636): + _elem1641 = PartitionSpec() + _elem1641.read(iprot) + self.success.append(_elem1641) iprot.readListEnd() else: iprot.skip(ftype) @@ -35022,8 +35022,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1628 in self.success: - iter1628.write(oprot) + for iter1642 in self.success: + iter1642.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -35171,10 +35171,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1632, _size1629) = iprot.readListBegin() - for _i1633 in range(_size1629): - _elem1634 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1634) + (_etype1646, _size1643) = iprot.readListBegin() + for _i1647 in range(_size1643): + _elem1648 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1648) iprot.readListEnd() else: iprot.skip(ftype) @@ -35201,8 +35201,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1635 in self.success: - oprot.writeString(iter1635.encode('utf-8') if sys.version_info[0] == 2 else iter1635) + for iter1649 in self.success: + oprot.writeString(iter1649.encode('utf-8') if sys.version_info[0] == 2 else iter1649) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -35425,10 +35425,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1639, _size1636) = iprot.readListBegin() - for _i1640 in range(_size1636): - _elem1641 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1641) + (_etype1653, _size1650) = iprot.readListBegin() + for _i1654 in range(_size1650): + _elem1655 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1655) iprot.readListEnd() else: iprot.skip(ftype) @@ -35458,8 +35458,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1642 in self.part_vals: - oprot.writeString(iter1642.encode('utf-8') if sys.version_info[0] == 2 else iter1642) + for iter1656 in self.part_vals: + oprot.writeString(iter1656.encode('utf-8') if sys.version_info[0] == 2 else iter1656) oprot.writeListEnd() oprot.writeFieldEnd() if self.max_parts is not None: @@ -35519,11 +35519,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1646, _size1643) = iprot.readListBegin() - for _i1647 in range(_size1643): - _elem1648 = Partition() - _elem1648.read(iprot) - self.success.append(_elem1648) + (_etype1660, _size1657) = iprot.readListBegin() + for _i1661 in range(_size1657): + _elem1662 = Partition() + _elem1662.read(iprot) + self.success.append(_elem1662) iprot.readListEnd() else: iprot.skip(ftype) @@ -35550,8 +35550,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1649 in self.success: - iter1649.write(oprot) + for iter1663 in self.success: + iter1663.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -35629,10 +35629,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1653, _size1650) = iprot.readListBegin() - for _i1654 in range(_size1650): - _elem1655 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1655) + (_etype1667, _size1664) = iprot.readListBegin() + for _i1668 in range(_size1664): + _elem1669 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1669) iprot.readListEnd() else: iprot.skip(ftype) @@ -35649,10 +35649,10 @@ def read(self, iprot): elif fid == 6: if ftype == TType.LIST: self.group_names = [] - (_etype1659, _size1656) = iprot.readListBegin() - for _i1660 in range(_size1656): - _elem1661 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1661) + (_etype1673, _size1670) = iprot.readListBegin() + for _i1674 in range(_size1670): + _elem1675 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1675) iprot.readListEnd() else: iprot.skip(ftype) @@ -35677,8 +35677,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1662 in self.part_vals: - oprot.writeString(iter1662.encode('utf-8') if sys.version_info[0] == 2 else iter1662) + for iter1676 in self.part_vals: + oprot.writeString(iter1676.encode('utf-8') if sys.version_info[0] == 2 else iter1676) oprot.writeListEnd() oprot.writeFieldEnd() if self.max_parts is not None: @@ -35692,8 +35692,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 6) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1663 in self.group_names: - oprot.writeString(iter1663.encode('utf-8') if sys.version_info[0] == 2 else iter1663) + for iter1677 in self.group_names: + oprot.writeString(iter1677.encode('utf-8') if sys.version_info[0] == 2 else iter1677) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -35751,11 +35751,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1667, _size1664) = iprot.readListBegin() - for _i1668 in range(_size1664): - _elem1669 = Partition() - _elem1669.read(iprot) - self.success.append(_elem1669) + (_etype1681, _size1678) = iprot.readListBegin() + for _i1682 in range(_size1678): + _elem1683 = Partition() + _elem1683.read(iprot) + self.success.append(_elem1683) iprot.readListEnd() else: iprot.skip(ftype) @@ -35782,8 +35782,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1670 in self.success: - iter1670.write(oprot) + for iter1684 in self.success: + iter1684.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36006,10 +36006,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1674, _size1671) = iprot.readListBegin() - for _i1675 in range(_size1671): - _elem1676 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1676) + (_etype1688, _size1685) = iprot.readListBegin() + for _i1689 in range(_size1685): + _elem1690 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1690) iprot.readListEnd() else: iprot.skip(ftype) @@ -36039,8 +36039,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1677 in self.part_vals: - oprot.writeString(iter1677.encode('utf-8') if sys.version_info[0] == 2 else iter1677) + for iter1691 in self.part_vals: + oprot.writeString(iter1691.encode('utf-8') if sys.version_info[0] == 2 else iter1691) oprot.writeListEnd() oprot.writeFieldEnd() if self.max_parts is not None: @@ -36100,10 +36100,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1681, _size1678) = iprot.readListBegin() - for _i1682 in range(_size1678): - _elem1683 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1683) + (_etype1695, _size1692) = iprot.readListBegin() + for _i1696 in range(_size1692): + _elem1697 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1697) iprot.readListEnd() else: iprot.skip(ftype) @@ -36130,8 +36130,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1684 in self.success: - oprot.writeString(iter1684.encode('utf-8') if sys.version_info[0] == 2 else iter1684) + for iter1698 in self.success: + oprot.writeString(iter1698.encode('utf-8') if sys.version_info[0] == 2 else iter1698) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36405,10 +36405,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1688, _size1685) = iprot.readListBegin() - for _i1689 in range(_size1685): - _elem1690 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1690) + (_etype1702, _size1699) = iprot.readListBegin() + for _i1703 in range(_size1699): + _elem1704 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1704) iprot.readListEnd() else: iprot.skip(ftype) @@ -36435,8 +36435,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1691 in self.success: - oprot.writeString(iter1691.encode('utf-8') if sys.version_info[0] == 2 else iter1691) + for iter1705 in self.success: + oprot.writeString(iter1705.encode('utf-8') if sys.version_info[0] == 2 else iter1705) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36596,11 +36596,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1695, _size1692) = iprot.readListBegin() - for _i1696 in range(_size1692): - _elem1697 = Partition() - _elem1697.read(iprot) - self.success.append(_elem1697) + (_etype1709, _size1706) = iprot.readListBegin() + for _i1710 in range(_size1706): + _elem1711 = Partition() + _elem1711.read(iprot) + self.success.append(_elem1711) iprot.readListEnd() else: iprot.skip(ftype) @@ -36627,8 +36627,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1698 in self.success: - iter1698.write(oprot) + for iter1712 in self.success: + iter1712.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36753,11 +36753,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1702, _size1699) = iprot.readListBegin() - for _i1703 in range(_size1699): - _elem1704 = Partition() - _elem1704.read(iprot) - self.success.append(_elem1704) + (_etype1716, _size1713) = iprot.readListBegin() + for _i1717 in range(_size1713): + _elem1718 = Partition() + _elem1718.read(iprot) + self.success.append(_elem1718) iprot.readListEnd() else: iprot.skip(ftype) @@ -36784,8 +36784,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1705 in self.success: - iter1705.write(oprot) + for iter1719 in self.success: + iter1719.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36945,11 +36945,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1709, _size1706) = iprot.readListBegin() - for _i1710 in range(_size1706): - _elem1711 = PartitionSpec() - _elem1711.read(iprot) - self.success.append(_elem1711) + (_etype1723, _size1720) = iprot.readListBegin() + for _i1724 in range(_size1720): + _elem1725 = PartitionSpec() + _elem1725.read(iprot) + self.success.append(_elem1725) iprot.readListEnd() else: iprot.skip(ftype) @@ -36976,8 +36976,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1712 in self.success: - iter1712.write(oprot) + for iter1726 in self.success: + iter1726.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -37518,10 +37518,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.names = [] - (_etype1716, _size1713) = iprot.readListBegin() - for _i1717 in range(_size1713): - _elem1718 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.names.append(_elem1718) + (_etype1730, _size1727) = iprot.readListBegin() + for _i1731 in range(_size1727): + _elem1732 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.names.append(_elem1732) iprot.readListEnd() else: iprot.skip(ftype) @@ -37546,8 +37546,8 @@ def write(self, oprot): if self.names is not None: oprot.writeFieldBegin('names', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.names)) - for iter1719 in self.names: - oprot.writeString(iter1719.encode('utf-8') if sys.version_info[0] == 2 else iter1719) + for iter1733 in self.names: + oprot.writeString(iter1733.encode('utf-8') if sys.version_info[0] == 2 else iter1733) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -37604,11 +37604,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1723, _size1720) = iprot.readListBegin() - for _i1724 in range(_size1720): - _elem1725 = Partition() - _elem1725.read(iprot) - self.success.append(_elem1725) + (_etype1737, _size1734) = iprot.readListBegin() + for _i1738 in range(_size1734): + _elem1739 = Partition() + _elem1739.read(iprot) + self.success.append(_elem1739) iprot.readListEnd() else: iprot.skip(ftype) @@ -37640,8 +37640,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1726 in self.success: - iter1726.write(oprot) + for iter1740 in self.success: + iter1740.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -38337,11 +38337,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.new_parts = [] - (_etype1730, _size1727) = iprot.readListBegin() - for _i1731 in range(_size1727): - _elem1732 = Partition() - _elem1732.read(iprot) - self.new_parts.append(_elem1732) + (_etype1744, _size1741) = iprot.readListBegin() + for _i1745 in range(_size1741): + _elem1746 = Partition() + _elem1746.read(iprot) + self.new_parts.append(_elem1746) iprot.readListEnd() else: iprot.skip(ftype) @@ -38366,8 +38366,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1733 in self.new_parts: - iter1733.write(oprot) + for iter1747 in self.new_parts: + iter1747.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -38508,11 +38508,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.new_parts = [] - (_etype1737, _size1734) = iprot.readListBegin() - for _i1738 in range(_size1734): - _elem1739 = Partition() - _elem1739.read(iprot) - self.new_parts.append(_elem1739) + (_etype1751, _size1748) = iprot.readListBegin() + for _i1752 in range(_size1748): + _elem1753 = Partition() + _elem1753.read(iprot) + self.new_parts.append(_elem1753) iprot.readListEnd() else: iprot.skip(ftype) @@ -38543,8 +38543,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1740 in self.new_parts: - iter1740.write(oprot) + for iter1754 in self.new_parts: + iter1754.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.environment_context is not None: @@ -39013,10 +39013,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1744, _size1741) = iprot.readListBegin() - for _i1745 in range(_size1741): - _elem1746 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1746) + (_etype1758, _size1755) = iprot.readListBegin() + for _i1759 in range(_size1755): + _elem1760 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1760) iprot.readListEnd() else: iprot.skip(ftype) @@ -39047,8 +39047,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1747 in self.part_vals: - oprot.writeString(iter1747.encode('utf-8') if sys.version_info[0] == 2 else iter1747) + for iter1761 in self.part_vals: + oprot.writeString(iter1761.encode('utf-8') if sys.version_info[0] == 2 else iter1761) oprot.writeListEnd() oprot.writeFieldEnd() if self.new_part is not None: @@ -39329,10 +39329,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.part_vals = [] - (_etype1751, _size1748) = iprot.readListBegin() - for _i1752 in range(_size1748): - _elem1753 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1753) + (_etype1765, _size1762) = iprot.readListBegin() + for _i1766 in range(_size1762): + _elem1767 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1767) iprot.readListEnd() else: iprot.skip(ftype) @@ -39354,8 +39354,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1754 in self.part_vals: - oprot.writeString(iter1754.encode('utf-8') if sys.version_info[0] == 2 else iter1754) + for iter1768 in self.part_vals: + oprot.writeString(iter1768.encode('utf-8') if sys.version_info[0] == 2 else iter1768) oprot.writeListEnd() oprot.writeFieldEnd() if self.throw_exception is not None: @@ -39693,10 +39693,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1758, _size1755) = iprot.readListBegin() - for _i1759 in range(_size1755): - _elem1760 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1760) + (_etype1772, _size1769) = iprot.readListBegin() + for _i1773 in range(_size1769): + _elem1774 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1774) iprot.readListEnd() else: iprot.skip(ftype) @@ -39718,8 +39718,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1761 in self.success: - oprot.writeString(iter1761.encode('utf-8') if sys.version_info[0] == 2 else iter1761) + for iter1775 in self.success: + oprot.writeString(iter1775.encode('utf-8') if sys.version_info[0] == 2 else iter1775) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -39836,11 +39836,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.MAP: self.success = {} - (_ktype1763, _vtype1764, _size1762) = iprot.readMapBegin() - for _i1766 in range(_size1762): - _key1767 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1768 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success[_key1767] = _val1768 + (_ktype1777, _vtype1778, _size1776) = iprot.readMapBegin() + for _i1780 in range(_size1776): + _key1781 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1782 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success[_key1781] = _val1782 iprot.readMapEnd() else: iprot.skip(ftype) @@ -39862,9 +39862,9 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.MAP, 0) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) - for kiter1769, viter1770 in self.success.items(): - oprot.writeString(kiter1769.encode('utf-8') if sys.version_info[0] == 2 else kiter1769) - oprot.writeString(viter1770.encode('utf-8') if sys.version_info[0] == 2 else viter1770) + for kiter1783, viter1784 in self.success.items(): + oprot.writeString(kiter1783.encode('utf-8') if sys.version_info[0] == 2 else kiter1783) + oprot.writeString(viter1784.encode('utf-8') if sys.version_info[0] == 2 else viter1784) oprot.writeMapEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -39933,11 +39933,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.MAP: self.part_vals = {} - (_ktype1772, _vtype1773, _size1771) = iprot.readMapBegin() - for _i1775 in range(_size1771): - _key1776 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1777 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals[_key1776] = _val1777 + (_ktype1786, _vtype1787, _size1785) = iprot.readMapBegin() + for _i1789 in range(_size1785): + _key1790 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1791 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals[_key1790] = _val1791 iprot.readMapEnd() else: iprot.skip(ftype) @@ -39967,9 +39967,9 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter1778, viter1779 in self.part_vals.items(): - oprot.writeString(kiter1778.encode('utf-8') if sys.version_info[0] == 2 else kiter1778) - oprot.writeString(viter1779.encode('utf-8') if sys.version_info[0] == 2 else viter1779) + for kiter1792, viter1793 in self.part_vals.items(): + oprot.writeString(kiter1792.encode('utf-8') if sys.version_info[0] == 2 else kiter1792) + oprot.writeString(viter1793.encode('utf-8') if sys.version_info[0] == 2 else viter1793) oprot.writeMapEnd() oprot.writeFieldEnd() if self.eventType is not None: @@ -40163,11 +40163,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.MAP: self.part_vals = {} - (_ktype1781, _vtype1782, _size1780) = iprot.readMapBegin() - for _i1784 in range(_size1780): - _key1785 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1786 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals[_key1785] = _val1786 + (_ktype1795, _vtype1796, _size1794) = iprot.readMapBegin() + for _i1798 in range(_size1794): + _key1799 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1800 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals[_key1799] = _val1800 iprot.readMapEnd() else: iprot.skip(ftype) @@ -40197,9 +40197,9 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter1787, viter1788 in self.part_vals.items(): - oprot.writeString(kiter1787.encode('utf-8') if sys.version_info[0] == 2 else kiter1787) - oprot.writeString(viter1788.encode('utf-8') if sys.version_info[0] == 2 else viter1788) + for kiter1801, viter1802 in self.part_vals.items(): + oprot.writeString(kiter1801.encode('utf-8') if sys.version_info[0] == 2 else kiter1801) + oprot.writeString(viter1802.encode('utf-8') if sys.version_info[0] == 2 else viter1802) oprot.writeMapEnd() oprot.writeFieldEnd() if self.eventType is not None: @@ -44241,10 +44241,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1792, _size1789) = iprot.readListBegin() - for _i1793 in range(_size1789): - _elem1794 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1794) + (_etype1806, _size1803) = iprot.readListBegin() + for _i1807 in range(_size1803): + _elem1808 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1808) iprot.readListEnd() else: iprot.skip(ftype) @@ -44266,8 +44266,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1795 in self.success: - oprot.writeString(iter1795.encode('utf-8') if sys.version_info[0] == 2 else iter1795) + for iter1809 in self.success: + oprot.writeString(iter1809.encode('utf-8') if sys.version_info[0] == 2 else iter1809) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -44913,10 +44913,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1799, _size1796) = iprot.readListBegin() - for _i1800 in range(_size1796): - _elem1801 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1801) + (_etype1813, _size1810) = iprot.readListBegin() + for _i1814 in range(_size1810): + _elem1815 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1815) iprot.readListEnd() else: iprot.skip(ftype) @@ -44938,8 +44938,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1802 in self.success: - oprot.writeString(iter1802.encode('utf-8') if sys.version_info[0] == 2 else iter1802) + for iter1816 in self.success: + oprot.writeString(iter1816.encode('utf-8') if sys.version_info[0] == 2 else iter1816) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -45422,11 +45422,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1806, _size1803) = iprot.readListBegin() - for _i1807 in range(_size1803): - _elem1808 = Role() - _elem1808.read(iprot) - self.success.append(_elem1808) + (_etype1820, _size1817) = iprot.readListBegin() + for _i1821 in range(_size1817): + _elem1822 = Role() + _elem1822.read(iprot) + self.success.append(_elem1822) iprot.readListEnd() else: iprot.skip(ftype) @@ -45448,8 +45448,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1809 in self.success: - iter1809.write(oprot) + for iter1823 in self.success: + iter1823.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -45928,10 +45928,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.group_names = [] - (_etype1813, _size1810) = iprot.readListBegin() - for _i1814 in range(_size1810): - _elem1815 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1815) + (_etype1827, _size1824) = iprot.readListBegin() + for _i1828 in range(_size1824): + _elem1829 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1829) iprot.readListEnd() else: iprot.skip(ftype) @@ -45956,8 +45956,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1816 in self.group_names: - oprot.writeString(iter1816.encode('utf-8') if sys.version_info[0] == 2 else iter1816) + for iter1830 in self.group_names: + oprot.writeString(iter1830.encode('utf-8') if sys.version_info[0] == 2 else iter1830) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -46171,11 +46171,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1820, _size1817) = iprot.readListBegin() - for _i1821 in range(_size1817): - _elem1822 = HiveObjectPrivilege() - _elem1822.read(iprot) - self.success.append(_elem1822) + (_etype1834, _size1831) = iprot.readListBegin() + for _i1835 in range(_size1831): + _elem1836 = HiveObjectPrivilege() + _elem1836.read(iprot) + self.success.append(_elem1836) iprot.readListEnd() else: iprot.skip(ftype) @@ -46197,8 +46197,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1823 in self.success: - iter1823.write(oprot) + for iter1837 in self.success: + iter1837.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -46829,10 +46829,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.group_names = [] - (_etype1827, _size1824) = iprot.readListBegin() - for _i1828 in range(_size1824): - _elem1829 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1829) + (_etype1841, _size1838) = iprot.readListBegin() + for _i1842 in range(_size1838): + _elem1843 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1843) iprot.readListEnd() else: iprot.skip(ftype) @@ -46853,8 +46853,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1830 in self.group_names: - oprot.writeString(iter1830.encode('utf-8') if sys.version_info[0] == 2 else iter1830) + for iter1844 in self.group_names: + oprot.writeString(iter1844.encode('utf-8') if sys.version_info[0] == 2 else iter1844) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -46906,10 +46906,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1834, _size1831) = iprot.readListBegin() - for _i1835 in range(_size1831): - _elem1836 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1836) + (_etype1848, _size1845) = iprot.readListBegin() + for _i1849 in range(_size1845): + _elem1850 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1850) iprot.readListEnd() else: iprot.skip(ftype) @@ -46931,8 +46931,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1837 in self.success: - oprot.writeString(iter1837.encode('utf-8') if sys.version_info[0] == 2 else iter1837) + for iter1851 in self.success: + oprot.writeString(iter1851.encode('utf-8') if sys.version_info[0] == 2 else iter1851) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -47815,10 +47815,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1841, _size1838) = iprot.readListBegin() - for _i1842 in range(_size1838): - _elem1843 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1843) + (_etype1855, _size1852) = iprot.readListBegin() + for _i1856 in range(_size1852): + _elem1857 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1857) iprot.readListEnd() else: iprot.skip(ftype) @@ -47835,8 +47835,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1844 in self.success: - oprot.writeString(iter1844.encode('utf-8') if sys.version_info[0] == 2 else iter1844) + for iter1858 in self.success: + oprot.writeString(iter1858.encode('utf-8') if sys.version_info[0] == 2 else iter1858) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -48333,10 +48333,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1848, _size1845) = iprot.readListBegin() - for _i1849 in range(_size1845): - _elem1850 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1850) + (_etype1862, _size1859) = iprot.readListBegin() + for _i1863 in range(_size1859): + _elem1864 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1864) iprot.readListEnd() else: iprot.skip(ftype) @@ -48353,8 +48353,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1851 in self.success: - oprot.writeString(iter1851.encode('utf-8') if sys.version_info[0] == 2 else iter1851) + for iter1865 in self.success: + oprot.writeString(iter1865.encode('utf-8') if sys.version_info[0] == 2 else iter1865) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -49521,11 +49521,11 @@ def read(self, iprot): elif fid == 2: if ftype == TType.MAP: self.writeIds = {} - (_ktype1853, _vtype1854, _size1852) = iprot.readMapBegin() - for _i1856 in range(_size1852): - _key1857 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1858 = iprot.readI64() - self.writeIds[_key1857] = _val1858 + (_ktype1867, _vtype1868, _size1866) = iprot.readMapBegin() + for _i1870 in range(_size1866): + _key1871 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1872 = iprot.readI64() + self.writeIds[_key1871] = _val1872 iprot.readMapEnd() else: iprot.skip(ftype) @@ -49546,9 +49546,9 @@ def write(self, oprot): if self.writeIds is not None: oprot.writeFieldBegin('writeIds', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.I64, len(self.writeIds)) - for kiter1859, viter1860 in self.writeIds.items(): - oprot.writeString(kiter1859.encode('utf-8') if sys.version_info[0] == 2 else kiter1859) - oprot.writeI64(viter1860) + for kiter1873, viter1874 in self.writeIds.items(): + oprot.writeString(kiter1873.encode('utf-8') if sys.version_info[0] == 2 else kiter1873) + oprot.writeI64(viter1874) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -52161,10 +52161,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1864, _size1861) = iprot.readListBegin() - for _i1865 in range(_size1861): - _elem1866 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1866) + (_etype1878, _size1875) = iprot.readListBegin() + for _i1879 in range(_size1875): + _elem1880 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1880) iprot.readListEnd() else: iprot.skip(ftype) @@ -52181,8 +52181,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1867 in self.success: - oprot.writeString(iter1867.encode('utf-8') if sys.version_info[0] == 2 else iter1867) + for iter1881 in self.success: + oprot.writeString(iter1881.encode('utf-8') if sys.version_info[0] == 2 else iter1881) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -58738,11 +58738,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1871, _size1868) = iprot.readListBegin() - for _i1872 in range(_size1868): - _elem1873 = SchemaVersion() - _elem1873.read(iprot) - self.success.append(_elem1873) + (_etype1885, _size1882) = iprot.readListBegin() + for _i1886 in range(_size1882): + _elem1887 = SchemaVersion() + _elem1887.read(iprot) + self.success.append(_elem1887) iprot.readListEnd() else: iprot.skip(ftype) @@ -58769,8 +58769,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1874 in self.success: - iter1874.write(oprot) + for iter1888 in self.success: + iter1888.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -60159,11 +60159,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1878, _size1875) = iprot.readListBegin() - for _i1879 in range(_size1875): - _elem1880 = RuntimeStat() - _elem1880.read(iprot) - self.success.append(_elem1880) + (_etype1892, _size1889) = iprot.readListBegin() + for _i1893 in range(_size1889): + _elem1894 = RuntimeStat() + _elem1894.read(iprot) + self.success.append(_elem1894) iprot.readListEnd() else: iprot.skip(ftype) @@ -60185,8 +60185,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1881 in self.success: - iter1881.write(oprot) + for iter1895 in self.success: + iter1895.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -61823,10 +61823,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1885, _size1882) = iprot.readListBegin() - for _i1886 in range(_size1882): - _elem1887 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1887) + (_etype1899, _size1896) = iprot.readListBegin() + for _i1900 in range(_size1896): + _elem1901 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1901) iprot.readListEnd() else: iprot.skip(ftype) @@ -61848,8 +61848,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1888 in self.success: - oprot.writeString(iter1888.encode('utf-8') if sys.version_info[0] == 2 else iter1888) + for iter1902 in self.success: + oprot.writeString(iter1902.encode('utf-8') if sys.version_info[0] == 2 else iter1902) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -62241,10 +62241,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1892, _size1889) = iprot.readListBegin() - for _i1893 in range(_size1889): - _elem1894 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1894) + (_etype1906, _size1903) = iprot.readListBegin() + for _i1907 in range(_size1903): + _elem1908 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1908) iprot.readListEnd() else: iprot.skip(ftype) @@ -62266,8 +62266,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1895 in self.success: - oprot.writeString(iter1895.encode('utf-8') if sys.version_info[0] == 2 else iter1895) + for iter1909 in self.success: + oprot.writeString(iter1909.encode('utf-8') if sys.version_info[0] == 2 else iter1909) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -62510,11 +62510,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1899, _size1896) = iprot.readListBegin() - for _i1900 in range(_size1896): - _elem1901 = WriteEventInfo() - _elem1901.read(iprot) - self.success.append(_elem1901) + (_etype1913, _size1910) = iprot.readListBegin() + for _i1914 in range(_size1910): + _elem1915 = WriteEventInfo() + _elem1915.read(iprot) + self.success.append(_elem1915) iprot.readListEnd() else: iprot.skip(ftype) @@ -62536,8 +62536,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1902 in self.success: - iter1902.write(oprot) + for iter1916 in self.success: + iter1916.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py index 434c81d9bc38..a06a33c405b2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -17848,14 +17848,20 @@ class NotificationEventRequest(object): - lastEvent - maxEvents - eventTypeSkipList + - catName + - dbName + - tableNames """ - def __init__(self, lastEvent=None, maxEvents=None, eventTypeSkipList=None,): + def __init__(self, lastEvent=None, maxEvents=None, eventTypeSkipList=None, catName=None, dbName=None, tableNames=None,): self.lastEvent = lastEvent self.maxEvents = maxEvents self.eventTypeSkipList = eventTypeSkipList + self.catName = catName + self.dbName = dbName + self.tableNames = tableNames def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -17886,6 +17892,26 @@ def read(self, iprot): iprot.readListEnd() else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.catName = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.dbName = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.LIST: + self.tableNames = [] + (_etype861, _size858) = iprot.readListBegin() + for _i862 in range(_size858): + _elem863 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tableNames.append(_elem863) + iprot.readListEnd() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -17907,8 +17933,23 @@ def write(self, oprot): if self.eventTypeSkipList is not None: oprot.writeFieldBegin('eventTypeSkipList', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.eventTypeSkipList)) - for iter858 in self.eventTypeSkipList: - oprot.writeString(iter858.encode('utf-8') if sys.version_info[0] == 2 else iter858) + for iter864 in self.eventTypeSkipList: + oprot.writeString(iter864.encode('utf-8') if sys.version_info[0] == 2 else iter864) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.catName is not None: + oprot.writeFieldBegin('catName', TType.STRING, 4) + oprot.writeString(self.catName.encode('utf-8') if sys.version_info[0] == 2 else self.catName) + oprot.writeFieldEnd() + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 5) + oprot.writeString(self.dbName.encode('utf-8') if sys.version_info[0] == 2 else self.dbName) + oprot.writeFieldEnd() + if self.tableNames is not None: + oprot.writeFieldBegin('tableNames', TType.LIST, 6) + oprot.writeListBegin(TType.STRING, len(self.tableNames)) + for iter865 in self.tableNames: + oprot.writeString(iter865.encode('utf-8') if sys.version_info[0] == 2 else iter865) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18096,11 +18137,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.events = [] - (_etype862, _size859) = iprot.readListBegin() - for _i863 in range(_size859): - _elem864 = NotificationEvent() - _elem864.read(iprot) - self.events.append(_elem864) + (_etype869, _size866) = iprot.readListBegin() + for _i870 in range(_size866): + _elem871 = NotificationEvent() + _elem871.read(iprot) + self.events.append(_elem871) iprot.readListEnd() else: iprot.skip(ftype) @@ -18117,8 +18158,8 @@ def write(self, oprot): if self.events is not None: oprot.writeFieldBegin('events', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.events)) - for iter865 in self.events: - iter865.write(oprot) + for iter872 in self.events: + iter872.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18208,16 +18249,18 @@ class NotificationEventsCountRequest(object): - catName - toEventId - limit + - tableNames """ - def __init__(self, fromEventId=None, dbName=None, catName=None, toEventId=None, limit=None,): + def __init__(self, fromEventId=None, dbName=None, catName=None, toEventId=None, limit=None, tableNames=None,): self.fromEventId = fromEventId self.dbName = dbName self.catName = catName self.toEventId = toEventId self.limit = limit + self.tableNames = tableNames def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -18253,6 +18296,16 @@ def read(self, iprot): self.limit = iprot.readI64() else: iprot.skip(ftype) + elif fid == 6: + if ftype == TType.LIST: + self.tableNames = [] + (_etype876, _size873) = iprot.readListBegin() + for _i877 in range(_size873): + _elem878 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tableNames.append(_elem878) + iprot.readListEnd() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -18283,6 +18336,13 @@ def write(self, oprot): oprot.writeFieldBegin('limit', TType.I64, 5) oprot.writeI64(self.limit) oprot.writeFieldEnd() + if self.tableNames is not None: + oprot.writeFieldBegin('tableNames', TType.LIST, 6) + oprot.writeListBegin(TType.STRING, len(self.tableNames)) + for iter879 in self.tableNames: + oprot.writeString(iter879.encode('utf-8') if sys.version_info[0] == 2 else iter879) + oprot.writeListEnd() + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18400,40 +18460,40 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.filesAdded = [] - (_etype869, _size866) = iprot.readListBegin() - for _i870 in range(_size866): - _elem871 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.filesAdded.append(_elem871) + (_etype883, _size880) = iprot.readListBegin() + for _i884 in range(_size880): + _elem885 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.filesAdded.append(_elem885) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: self.filesAddedChecksum = [] - (_etype875, _size872) = iprot.readListBegin() - for _i876 in range(_size872): - _elem877 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.filesAddedChecksum.append(_elem877) + (_etype889, _size886) = iprot.readListBegin() + for _i890 in range(_size886): + _elem891 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.filesAddedChecksum.append(_elem891) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.subDirectoryList = [] - (_etype881, _size878) = iprot.readListBegin() - for _i882 in range(_size878): - _elem883 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.subDirectoryList.append(_elem883) + (_etype895, _size892) = iprot.readListBegin() + for _i896 in range(_size892): + _elem897 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.subDirectoryList.append(_elem897) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.LIST: self.partitionVal = [] - (_etype887, _size884) = iprot.readListBegin() - for _i888 in range(_size884): - _elem889 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionVal.append(_elem889) + (_etype901, _size898) = iprot.readListBegin() + for _i902 in range(_size898): + _elem903 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionVal.append(_elem903) iprot.readListEnd() else: iprot.skip(ftype) @@ -18454,29 +18514,29 @@ def write(self, oprot): if self.filesAdded is not None: oprot.writeFieldBegin('filesAdded', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.filesAdded)) - for iter890 in self.filesAdded: - oprot.writeString(iter890.encode('utf-8') if sys.version_info[0] == 2 else iter890) + for iter904 in self.filesAdded: + oprot.writeString(iter904.encode('utf-8') if sys.version_info[0] == 2 else iter904) oprot.writeListEnd() oprot.writeFieldEnd() if self.filesAddedChecksum is not None: oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum)) - for iter891 in self.filesAddedChecksum: - oprot.writeString(iter891.encode('utf-8') if sys.version_info[0] == 2 else iter891) + for iter905 in self.filesAddedChecksum: + oprot.writeString(iter905.encode('utf-8') if sys.version_info[0] == 2 else iter905) oprot.writeListEnd() oprot.writeFieldEnd() if self.subDirectoryList is not None: oprot.writeFieldBegin('subDirectoryList', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.subDirectoryList)) - for iter892 in self.subDirectoryList: - oprot.writeString(iter892.encode('utf-8') if sys.version_info[0] == 2 else iter892) + for iter906 in self.subDirectoryList: + oprot.writeString(iter906.encode('utf-8') if sys.version_info[0] == 2 else iter906) oprot.writeListEnd() oprot.writeFieldEnd() if self.partitionVal is not None: oprot.writeFieldBegin('partitionVal', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.partitionVal)) - for iter893 in self.partitionVal: - oprot.writeString(iter893.encode('utf-8') if sys.version_info[0] == 2 else iter893) + for iter907 in self.partitionVal: + oprot.writeString(iter907.encode('utf-8') if sys.version_info[0] == 2 else iter907) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18532,11 +18592,11 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.insertDatas = [] - (_etype897, _size894) = iprot.readListBegin() - for _i898 in range(_size894): - _elem899 = InsertEventRequestData() - _elem899.read(iprot) - self.insertDatas.append(_elem899) + (_etype911, _size908) = iprot.readListBegin() + for _i912 in range(_size908): + _elem913 = InsertEventRequestData() + _elem913.read(iprot) + self.insertDatas.append(_elem913) iprot.readListEnd() else: iprot.skip(ftype) @@ -18562,8 +18622,8 @@ def write(self, oprot): if self.insertDatas is not None: oprot.writeFieldBegin('insertDatas', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.insertDatas)) - for iter900 in self.insertDatas: - iter900.write(oprot) + for iter914 in self.insertDatas: + iter914.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.refreshEvent is not None: @@ -18644,10 +18704,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.partitionVals = [] - (_etype904, _size901) = iprot.readListBegin() - for _i905 in range(_size901): - _elem906 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionVals.append(_elem906) + (_etype918, _size915) = iprot.readListBegin() + for _i919 in range(_size915): + _elem920 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionVals.append(_elem920) iprot.readListEnd() else: iprot.skip(ftype) @@ -18659,11 +18719,11 @@ def read(self, iprot): elif fid == 7: if ftype == TType.MAP: self.tblParams = {} - (_ktype908, _vtype909, _size907) = iprot.readMapBegin() - for _i911 in range(_size907): - _key912 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val913 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.tblParams[_key912] = _val913 + (_ktype922, _vtype923, _size921) = iprot.readMapBegin() + for _i925 in range(_size921): + _key926 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val927 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tblParams[_key926] = _val927 iprot.readMapEnd() else: iprot.skip(ftype) @@ -18696,8 +18756,8 @@ def write(self, oprot): if self.partitionVals is not None: oprot.writeFieldBegin('partitionVals', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.partitionVals)) - for iter914 in self.partitionVals: - oprot.writeString(iter914.encode('utf-8') if sys.version_info[0] == 2 else iter914) + for iter928 in self.partitionVals: + oprot.writeString(iter928.encode('utf-8') if sys.version_info[0] == 2 else iter928) oprot.writeListEnd() oprot.writeFieldEnd() if self.catName is not None: @@ -18707,9 +18767,9 @@ def write(self, oprot): if self.tblParams is not None: oprot.writeFieldBegin('tblParams', TType.MAP, 7) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.tblParams)) - for kiter915, viter916 in self.tblParams.items(): - oprot.writeString(kiter915.encode('utf-8') if sys.version_info[0] == 2 else kiter915) - oprot.writeString(viter916.encode('utf-8') if sys.version_info[0] == 2 else viter916) + for kiter929, viter930 in self.tblParams.items(): + oprot.writeString(kiter929.encode('utf-8') if sys.version_info[0] == 2 else kiter929) + oprot.writeString(viter930.encode('utf-8') if sys.version_info[0] == 2 else viter930) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18757,10 +18817,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.eventIds = [] - (_etype920, _size917) = iprot.readListBegin() - for _i921 in range(_size917): - _elem922 = iprot.readI64() - self.eventIds.append(_elem922) + (_etype934, _size931) = iprot.readListBegin() + for _i935 in range(_size931): + _elem936 = iprot.readI64() + self.eventIds.append(_elem936) iprot.readListEnd() else: iprot.skip(ftype) @@ -18777,8 +18837,8 @@ def write(self, oprot): if self.eventIds is not None: oprot.writeFieldBegin('eventIds', TType.LIST, 1) oprot.writeListBegin(TType.I64, len(self.eventIds)) - for iter923 in self.eventIds: - oprot.writeI64(iter923) + for iter937 in self.eventIds: + oprot.writeI64(iter937) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -18858,10 +18918,10 @@ def read(self, iprot): elif fid == 6: if ftype == TType.LIST: self.partitionVals = [] - (_etype927, _size924) = iprot.readListBegin() - for _i928 in range(_size924): - _elem929 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionVals.append(_elem929) + (_etype941, _size938) = iprot.readListBegin() + for _i942 in range(_size938): + _elem943 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionVals.append(_elem943) iprot.readListEnd() else: iprot.skip(ftype) @@ -18898,8 +18958,8 @@ def write(self, oprot): if self.partitionVals is not None: oprot.writeFieldBegin('partitionVals', TType.LIST, 6) oprot.writeListBegin(TType.STRING, len(self.partitionVals)) - for iter930 in self.partitionVals: - oprot.writeString(iter930.encode('utf-8') if sys.version_info[0] == 2 else iter930) + for iter944 in self.partitionVals: + oprot.writeString(iter944.encode('utf-8') if sys.version_info[0] == 2 else iter944) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -19014,11 +19074,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.requestList = [] - (_etype934, _size931) = iprot.readListBegin() - for _i935 in range(_size931): - _elem936 = WriteNotificationLogRequest() - _elem936.read(iprot) - self.requestList.append(_elem936) + (_etype948, _size945) = iprot.readListBegin() + for _i949 in range(_size945): + _elem950 = WriteNotificationLogRequest() + _elem950.read(iprot) + self.requestList.append(_elem950) iprot.readListEnd() else: iprot.skip(ftype) @@ -19047,8 +19107,8 @@ def write(self, oprot): if self.requestList is not None: oprot.writeFieldBegin('requestList', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.requestList)) - for iter937 in self.requestList: - iter937.write(oprot) + for iter951 in self.requestList: + iter951.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -19210,12 +19270,12 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.metadata = {} - (_ktype939, _vtype940, _size938) = iprot.readMapBegin() - for _i942 in range(_size938): - _key943 = iprot.readI64() - _val944 = MetadataPpdResult() - _val944.read(iprot) - self.metadata[_key943] = _val944 + (_ktype953, _vtype954, _size952) = iprot.readMapBegin() + for _i956 in range(_size952): + _key957 = iprot.readI64() + _val958 = MetadataPpdResult() + _val958.read(iprot) + self.metadata[_key957] = _val958 iprot.readMapEnd() else: iprot.skip(ftype) @@ -19237,9 +19297,9 @@ def write(self, oprot): if self.metadata is not None: oprot.writeFieldBegin('metadata', TType.MAP, 1) oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata)) - for kiter945, viter946 in self.metadata.items(): - oprot.writeI64(kiter945) - viter946.write(oprot) + for kiter959, viter960 in self.metadata.items(): + oprot.writeI64(kiter959) + viter960.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.isSupported is not None: @@ -19297,10 +19357,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fileIds = [] - (_etype950, _size947) = iprot.readListBegin() - for _i951 in range(_size947): - _elem952 = iprot.readI64() - self.fileIds.append(_elem952) + (_etype964, _size961) = iprot.readListBegin() + for _i965 in range(_size961): + _elem966 = iprot.readI64() + self.fileIds.append(_elem966) iprot.readListEnd() else: iprot.skip(ftype) @@ -19332,8 +19392,8 @@ def write(self, oprot): if self.fileIds is not None: oprot.writeFieldBegin('fileIds', TType.LIST, 1) oprot.writeListBegin(TType.I64, len(self.fileIds)) - for iter953 in self.fileIds: - oprot.writeI64(iter953) + for iter967 in self.fileIds: + oprot.writeI64(iter967) oprot.writeListEnd() oprot.writeFieldEnd() if self.expr is not None: @@ -19395,11 +19455,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.metadata = {} - (_ktype955, _vtype956, _size954) = iprot.readMapBegin() - for _i958 in range(_size954): - _key959 = iprot.readI64() - _val960 = iprot.readBinary() - self.metadata[_key959] = _val960 + (_ktype969, _vtype970, _size968) = iprot.readMapBegin() + for _i972 in range(_size968): + _key973 = iprot.readI64() + _val974 = iprot.readBinary() + self.metadata[_key973] = _val974 iprot.readMapEnd() else: iprot.skip(ftype) @@ -19421,9 +19481,9 @@ def write(self, oprot): if self.metadata is not None: oprot.writeFieldBegin('metadata', TType.MAP, 1) oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata)) - for kiter961, viter962 in self.metadata.items(): - oprot.writeI64(kiter961) - oprot.writeBinary(viter962) + for kiter975, viter976 in self.metadata.items(): + oprot.writeI64(kiter975) + oprot.writeBinary(viter976) oprot.writeMapEnd() oprot.writeFieldEnd() if self.isSupported is not None: @@ -19475,10 +19535,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fileIds = [] - (_etype966, _size963) = iprot.readListBegin() - for _i967 in range(_size963): - _elem968 = iprot.readI64() - self.fileIds.append(_elem968) + (_etype980, _size977) = iprot.readListBegin() + for _i981 in range(_size977): + _elem982 = iprot.readI64() + self.fileIds.append(_elem982) iprot.readListEnd() else: iprot.skip(ftype) @@ -19495,8 +19555,8 @@ def write(self, oprot): if self.fileIds is not None: oprot.writeFieldBegin('fileIds', TType.LIST, 1) oprot.writeListBegin(TType.I64, len(self.fileIds)) - for iter969 in self.fileIds: - oprot.writeI64(iter969) + for iter983 in self.fileIds: + oprot.writeI64(iter983) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -19586,20 +19646,20 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fileIds = [] - (_etype973, _size970) = iprot.readListBegin() - for _i974 in range(_size970): - _elem975 = iprot.readI64() - self.fileIds.append(_elem975) + (_etype987, _size984) = iprot.readListBegin() + for _i988 in range(_size984): + _elem989 = iprot.readI64() + self.fileIds.append(_elem989) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.LIST: self.metadata = [] - (_etype979, _size976) = iprot.readListBegin() - for _i980 in range(_size976): - _elem981 = iprot.readBinary() - self.metadata.append(_elem981) + (_etype993, _size990) = iprot.readListBegin() + for _i994 in range(_size990): + _elem995 = iprot.readBinary() + self.metadata.append(_elem995) iprot.readListEnd() else: iprot.skip(ftype) @@ -19621,15 +19681,15 @@ def write(self, oprot): if self.fileIds is not None: oprot.writeFieldBegin('fileIds', TType.LIST, 1) oprot.writeListBegin(TType.I64, len(self.fileIds)) - for iter982 in self.fileIds: - oprot.writeI64(iter982) + for iter996 in self.fileIds: + oprot.writeI64(iter996) oprot.writeListEnd() oprot.writeFieldEnd() if self.metadata is not None: oprot.writeFieldBegin('metadata', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.metadata)) - for iter983 in self.metadata: - oprot.writeBinary(iter983) + for iter997 in self.metadata: + oprot.writeBinary(iter997) oprot.writeListEnd() oprot.writeFieldEnd() if self.type is not None: @@ -19721,10 +19781,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fileIds = [] - (_etype987, _size984) = iprot.readListBegin() - for _i988 in range(_size984): - _elem989 = iprot.readI64() - self.fileIds.append(_elem989) + (_etype1001, _size998) = iprot.readListBegin() + for _i1002 in range(_size998): + _elem1003 = iprot.readI64() + self.fileIds.append(_elem1003) iprot.readListEnd() else: iprot.skip(ftype) @@ -19741,8 +19801,8 @@ def write(self, oprot): if self.fileIds is not None: oprot.writeFieldBegin('fileIds', TType.LIST, 1) oprot.writeListBegin(TType.I64, len(self.fileIds)) - for iter990 in self.fileIds: - oprot.writeI64(iter990) + for iter1004 in self.fileIds: + oprot.writeI64(iter1004) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -19941,11 +20001,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.functions = [] - (_etype994, _size991) = iprot.readListBegin() - for _i995 in range(_size991): - _elem996 = Function() - _elem996.read(iprot) - self.functions.append(_elem996) + (_etype1008, _size1005) = iprot.readListBegin() + for _i1009 in range(_size1005): + _elem1010 = Function() + _elem1010.read(iprot) + self.functions.append(_elem1010) iprot.readListEnd() else: iprot.skip(ftype) @@ -19962,8 +20022,8 @@ def write(self, oprot): if self.functions is not None: oprot.writeFieldBegin('functions', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.functions)) - for iter997 in self.functions: - iter997.write(oprot) + for iter1011 in self.functions: + iter1011.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -20007,10 +20067,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.values = [] - (_etype1001, _size998) = iprot.readListBegin() - for _i1002 in range(_size998): - _elem1003 = iprot.readI32() - self.values.append(_elem1003) + (_etype1015, _size1012) = iprot.readListBegin() + for _i1016 in range(_size1012): + _elem1017 = iprot.readI32() + self.values.append(_elem1017) iprot.readListEnd() else: iprot.skip(ftype) @@ -20027,8 +20087,8 @@ def write(self, oprot): if self.values is not None: oprot.writeFieldBegin('values', TType.LIST, 1) oprot.writeListBegin(TType.I32, len(self.values)) - for iter1004 in self.values: - oprot.writeI32(iter1004) + for iter1018 in self.values: + oprot.writeI32(iter1018) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -20078,10 +20138,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fieldList = [] - (_etype1008, _size1005) = iprot.readListBegin() - for _i1009 in range(_size1005): - _elem1010 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.fieldList.append(_elem1010) + (_etype1022, _size1019) = iprot.readListBegin() + for _i1023 in range(_size1019): + _elem1024 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.fieldList.append(_elem1024) iprot.readListEnd() else: iprot.skip(ftype) @@ -20108,8 +20168,8 @@ def write(self, oprot): if self.fieldList is not None: oprot.writeFieldBegin('fieldList', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.fieldList)) - for iter1011 in self.fieldList: - oprot.writeString(iter1011.encode('utf-8') if sys.version_info[0] == 2 else iter1011) + for iter1025 in self.fieldList: + oprot.writeString(iter1025.encode('utf-8') if sys.version_info[0] == 2 else iter1025) oprot.writeListEnd() oprot.writeFieldEnd() if self.includeParamKeyPattern is not None: @@ -20210,10 +20270,10 @@ def read(self, iprot): elif fid == 8: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1015, _size1012) = iprot.readListBegin() - for _i1016 in range(_size1012): - _elem1017 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1017) + (_etype1029, _size1026) = iprot.readListBegin() + for _i1030 in range(_size1026): + _elem1031 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1031) iprot.readListEnd() else: iprot.skip(ftype) @@ -20269,8 +20329,8 @@ def write(self, oprot): if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 8) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1018 in self.processorCapabilities: - oprot.writeString(iter1018.encode('utf-8') if sys.version_info[0] == 2 else iter1018) + for iter1032 in self.processorCapabilities: + oprot.writeString(iter1032.encode('utf-8') if sys.version_info[0] == 2 else iter1032) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -20420,10 +20480,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.tblNames = [] - (_etype1022, _size1019) = iprot.readListBegin() - for _i1023 in range(_size1019): - _elem1024 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.tblNames.append(_elem1024) + (_etype1036, _size1033) = iprot.readListBegin() + for _i1037 in range(_size1033): + _elem1038 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tblNames.append(_elem1038) iprot.readListEnd() else: iprot.skip(ftype) @@ -20441,10 +20501,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1028, _size1025) = iprot.readListBegin() - for _i1029 in range(_size1025): - _elem1030 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1030) + (_etype1042, _size1039) = iprot.readListBegin() + for _i1043 in range(_size1039): + _elem1044 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1044) iprot.readListEnd() else: iprot.skip(ftype) @@ -20481,8 +20541,8 @@ def write(self, oprot): if self.tblNames is not None: oprot.writeFieldBegin('tblNames', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.tblNames)) - for iter1031 in self.tblNames: - oprot.writeString(iter1031.encode('utf-8') if sys.version_info[0] == 2 else iter1031) + for iter1045 in self.tblNames: + oprot.writeString(iter1045.encode('utf-8') if sys.version_info[0] == 2 else iter1045) oprot.writeListEnd() oprot.writeFieldEnd() if self.capabilities is not None: @@ -20496,8 +20556,8 @@ def write(self, oprot): if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1032 in self.processorCapabilities: - oprot.writeString(iter1032.encode('utf-8') if sys.version_info[0] == 2 else iter1032) + for iter1046 in self.processorCapabilities: + oprot.writeString(iter1046.encode('utf-8') if sys.version_info[0] == 2 else iter1046) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -20555,11 +20615,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.tables = [] - (_etype1036, _size1033) = iprot.readListBegin() - for _i1037 in range(_size1033): - _elem1038 = Table() - _elem1038.read(iprot) - self.tables.append(_elem1038) + (_etype1050, _size1047) = iprot.readListBegin() + for _i1051 in range(_size1047): + _elem1052 = Table() + _elem1052.read(iprot) + self.tables.append(_elem1052) iprot.readListEnd() else: iprot.skip(ftype) @@ -20576,8 +20636,8 @@ def write(self, oprot): if self.tables is not None: oprot.writeFieldBegin('tables', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.tables)) - for iter1039 in self.tables: - iter1039.write(oprot) + for iter1053 in self.tables: + iter1053.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -20660,10 +20720,10 @@ def read(self, iprot): elif fid == 6: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1043, _size1040) = iprot.readListBegin() - for _i1044 in range(_size1040): - _elem1045 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1045) + (_etype1057, _size1054) = iprot.readListBegin() + for _i1058 in range(_size1054): + _elem1059 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1059) iprot.readListEnd() else: iprot.skip(ftype) @@ -20705,8 +20765,8 @@ def write(self, oprot): if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 6) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1046 in self.processorCapabilities: - oprot.writeString(iter1046.encode('utf-8') if sys.version_info[0] == 2 else iter1046) + for iter1060 in self.processorCapabilities: + oprot.writeString(iter1060.encode('utf-8') if sys.version_info[0] == 2 else iter1060) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -20778,20 +20838,20 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.requiredReadCapabilities = [] - (_etype1050, _size1047) = iprot.readListBegin() - for _i1051 in range(_size1047): - _elem1052 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.requiredReadCapabilities.append(_elem1052) + (_etype1064, _size1061) = iprot.readListBegin() + for _i1065 in range(_size1061): + _elem1066 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.requiredReadCapabilities.append(_elem1066) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.requiredWriteCapabilities = [] - (_etype1056, _size1053) = iprot.readListBegin() - for _i1057 in range(_size1053): - _elem1058 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.requiredWriteCapabilities.append(_elem1058) + (_etype1070, _size1067) = iprot.readListBegin() + for _i1071 in range(_size1067): + _elem1072 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.requiredWriteCapabilities.append(_elem1072) iprot.readListEnd() else: iprot.skip(ftype) @@ -20816,15 +20876,15 @@ def write(self, oprot): if self.requiredReadCapabilities is not None: oprot.writeFieldBegin('requiredReadCapabilities', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.requiredReadCapabilities)) - for iter1059 in self.requiredReadCapabilities: - oprot.writeString(iter1059.encode('utf-8') if sys.version_info[0] == 2 else iter1059) + for iter1073 in self.requiredReadCapabilities: + oprot.writeString(iter1073.encode('utf-8') if sys.version_info[0] == 2 else iter1073) oprot.writeListEnd() oprot.writeFieldEnd() if self.requiredWriteCapabilities is not None: oprot.writeFieldBegin('requiredWriteCapabilities', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.requiredWriteCapabilities)) - for iter1060 in self.requiredWriteCapabilities: - oprot.writeString(iter1060.encode('utf-8') if sys.version_info[0] == 2 else iter1060) + for iter1074 in self.requiredWriteCapabilities: + oprot.writeString(iter1074.encode('utf-8') if sys.version_info[0] == 2 else iter1074) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -20886,10 +20946,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1064, _size1061) = iprot.readListBegin() - for _i1065 in range(_size1061): - _elem1066 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1066) + (_etype1078, _size1075) = iprot.readListBegin() + for _i1079 in range(_size1075): + _elem1080 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1080) iprot.readListEnd() else: iprot.skip(ftype) @@ -20919,8 +20979,8 @@ def write(self, oprot): if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1067 in self.processorCapabilities: - oprot.writeString(iter1067.encode('utf-8') if sys.version_info[0] == 2 else iter1067) + for iter1081 in self.processorCapabilities: + oprot.writeString(iter1081.encode('utf-8') if sys.version_info[0] == 2 else iter1081) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -22223,44 +22283,44 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.pools = [] - (_etype1071, _size1068) = iprot.readListBegin() - for _i1072 in range(_size1068): - _elem1073 = WMPool() - _elem1073.read(iprot) - self.pools.append(_elem1073) + (_etype1085, _size1082) = iprot.readListBegin() + for _i1086 in range(_size1082): + _elem1087 = WMPool() + _elem1087.read(iprot) + self.pools.append(_elem1087) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: self.mappings = [] - (_etype1077, _size1074) = iprot.readListBegin() - for _i1078 in range(_size1074): - _elem1079 = WMMapping() - _elem1079.read(iprot) - self.mappings.append(_elem1079) + (_etype1091, _size1088) = iprot.readListBegin() + for _i1092 in range(_size1088): + _elem1093 = WMMapping() + _elem1093.read(iprot) + self.mappings.append(_elem1093) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.triggers = [] - (_etype1083, _size1080) = iprot.readListBegin() - for _i1084 in range(_size1080): - _elem1085 = WMTrigger() - _elem1085.read(iprot) - self.triggers.append(_elem1085) + (_etype1097, _size1094) = iprot.readListBegin() + for _i1098 in range(_size1094): + _elem1099 = WMTrigger() + _elem1099.read(iprot) + self.triggers.append(_elem1099) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.LIST: self.poolTriggers = [] - (_etype1089, _size1086) = iprot.readListBegin() - for _i1090 in range(_size1086): - _elem1091 = WMPoolTrigger() - _elem1091.read(iprot) - self.poolTriggers.append(_elem1091) + (_etype1103, _size1100) = iprot.readListBegin() + for _i1104 in range(_size1100): + _elem1105 = WMPoolTrigger() + _elem1105.read(iprot) + self.poolTriggers.append(_elem1105) iprot.readListEnd() else: iprot.skip(ftype) @@ -22281,29 +22341,29 @@ def write(self, oprot): if self.pools is not None: oprot.writeFieldBegin('pools', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.pools)) - for iter1092 in self.pools: - iter1092.write(oprot) + for iter1106 in self.pools: + iter1106.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.mappings is not None: oprot.writeFieldBegin('mappings', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.mappings)) - for iter1093 in self.mappings: - iter1093.write(oprot) + for iter1107 in self.mappings: + iter1107.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.triggers is not None: oprot.writeFieldBegin('triggers', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.triggers)) - for iter1094 in self.triggers: - iter1094.write(oprot) + for iter1108 in self.triggers: + iter1108.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.poolTriggers is not None: oprot.writeFieldBegin('poolTriggers', TType.LIST, 5) oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers)) - for iter1095 in self.poolTriggers: - iter1095.write(oprot) + for iter1109 in self.poolTriggers: + iter1109.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -22758,11 +22818,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.resourcePlans = [] - (_etype1099, _size1096) = iprot.readListBegin() - for _i1100 in range(_size1096): - _elem1101 = WMResourcePlan() - _elem1101.read(iprot) - self.resourcePlans.append(_elem1101) + (_etype1113, _size1110) = iprot.readListBegin() + for _i1114 in range(_size1110): + _elem1115 = WMResourcePlan() + _elem1115.read(iprot) + self.resourcePlans.append(_elem1115) iprot.readListEnd() else: iprot.skip(ftype) @@ -22779,8 +22839,8 @@ def write(self, oprot): if self.resourcePlans is not None: oprot.writeFieldBegin('resourcePlans', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans)) - for iter1102 in self.resourcePlans: - iter1102.write(oprot) + for iter1116 in self.resourcePlans: + iter1116.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -23065,20 +23125,20 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.errors = [] - (_etype1106, _size1103) = iprot.readListBegin() - for _i1107 in range(_size1103): - _elem1108 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.errors.append(_elem1108) + (_etype1120, _size1117) = iprot.readListBegin() + for _i1121 in range(_size1117): + _elem1122 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.errors.append(_elem1122) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 2: if ftype == TType.LIST: self.warnings = [] - (_etype1112, _size1109) = iprot.readListBegin() - for _i1113 in range(_size1109): - _elem1114 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.warnings.append(_elem1114) + (_etype1126, _size1123) = iprot.readListBegin() + for _i1127 in range(_size1123): + _elem1128 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.warnings.append(_elem1128) iprot.readListEnd() else: iprot.skip(ftype) @@ -23095,15 +23155,15 @@ def write(self, oprot): if self.errors is not None: oprot.writeFieldBegin('errors', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.errors)) - for iter1115 in self.errors: - oprot.writeString(iter1115.encode('utf-8') if sys.version_info[0] == 2 else iter1115) + for iter1129 in self.errors: + oprot.writeString(iter1129.encode('utf-8') if sys.version_info[0] == 2 else iter1129) oprot.writeListEnd() oprot.writeFieldEnd() if self.warnings is not None: oprot.writeFieldBegin('warnings', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.warnings)) - for iter1116 in self.warnings: - oprot.writeString(iter1116.encode('utf-8') if sys.version_info[0] == 2 else iter1116) + for iter1130 in self.warnings: + oprot.writeString(iter1130.encode('utf-8') if sys.version_info[0] == 2 else iter1130) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -23638,11 +23698,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.triggers = [] - (_etype1120, _size1117) = iprot.readListBegin() - for _i1121 in range(_size1117): - _elem1122 = WMTrigger() - _elem1122.read(iprot) - self.triggers.append(_elem1122) + (_etype1134, _size1131) = iprot.readListBegin() + for _i1135 in range(_size1131): + _elem1136 = WMTrigger() + _elem1136.read(iprot) + self.triggers.append(_elem1136) iprot.readListEnd() else: iprot.skip(ftype) @@ -23659,8 +23719,8 @@ def write(self, oprot): if self.triggers is not None: oprot.writeFieldBegin('triggers', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.triggers)) - for iter1123 in self.triggers: - iter1123.write(oprot) + for iter1137 in self.triggers: + iter1137.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -24706,11 +24766,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.cols = [] - (_etype1127, _size1124) = iprot.readListBegin() - for _i1128 in range(_size1124): - _elem1129 = FieldSchema() - _elem1129.read(iprot) - self.cols.append(_elem1129) + (_etype1141, _size1138) = iprot.readListBegin() + for _i1142 in range(_size1138): + _elem1143 = FieldSchema() + _elem1143.read(iprot) + self.cols.append(_elem1143) iprot.readListEnd() else: iprot.skip(ftype) @@ -24770,8 +24830,8 @@ def write(self, oprot): if self.cols is not None: oprot.writeFieldBegin('cols', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.cols)) - for iter1130 in self.cols: - iter1130.write(oprot) + for iter1144 in self.cols: + iter1144.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.state is not None: @@ -24987,11 +25047,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.schemaVersions = [] - (_etype1134, _size1131) = iprot.readListBegin() - for _i1135 in range(_size1131): - _elem1136 = SchemaVersionDescriptor() - _elem1136.read(iprot) - self.schemaVersions.append(_elem1136) + (_etype1148, _size1145) = iprot.readListBegin() + for _i1149 in range(_size1145): + _elem1150 = SchemaVersionDescriptor() + _elem1150.read(iprot) + self.schemaVersions.append(_elem1150) iprot.readListEnd() else: iprot.skip(ftype) @@ -25008,8 +25068,8 @@ def write(self, oprot): if self.schemaVersions is not None: oprot.writeFieldBegin('schemaVersions', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions)) - for iter1137 in self.schemaVersions: - iter1137.write(oprot) + for iter1151 in self.schemaVersions: + iter1151.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -25433,76 +25493,76 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.primaryKeys = [] - (_etype1141, _size1138) = iprot.readListBegin() - for _i1142 in range(_size1138): - _elem1143 = SQLPrimaryKey() - _elem1143.read(iprot) - self.primaryKeys.append(_elem1143) + (_etype1155, _size1152) = iprot.readListBegin() + for _i1156 in range(_size1152): + _elem1157 = SQLPrimaryKey() + _elem1157.read(iprot) + self.primaryKeys.append(_elem1157) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.foreignKeys = [] - (_etype1147, _size1144) = iprot.readListBegin() - for _i1148 in range(_size1144): - _elem1149 = SQLForeignKey() - _elem1149.read(iprot) - self.foreignKeys.append(_elem1149) + (_etype1161, _size1158) = iprot.readListBegin() + for _i1162 in range(_size1158): + _elem1163 = SQLForeignKey() + _elem1163.read(iprot) + self.foreignKeys.append(_elem1163) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.LIST: self.uniqueConstraints = [] - (_etype1153, _size1150) = iprot.readListBegin() - for _i1154 in range(_size1150): - _elem1155 = SQLUniqueConstraint() - _elem1155.read(iprot) - self.uniqueConstraints.append(_elem1155) + (_etype1167, _size1164) = iprot.readListBegin() + for _i1168 in range(_size1164): + _elem1169 = SQLUniqueConstraint() + _elem1169.read(iprot) + self.uniqueConstraints.append(_elem1169) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 6: if ftype == TType.LIST: self.notNullConstraints = [] - (_etype1159, _size1156) = iprot.readListBegin() - for _i1160 in range(_size1156): - _elem1161 = SQLNotNullConstraint() - _elem1161.read(iprot) - self.notNullConstraints.append(_elem1161) + (_etype1173, _size1170) = iprot.readListBegin() + for _i1174 in range(_size1170): + _elem1175 = SQLNotNullConstraint() + _elem1175.read(iprot) + self.notNullConstraints.append(_elem1175) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 7: if ftype == TType.LIST: self.defaultConstraints = [] - (_etype1165, _size1162) = iprot.readListBegin() - for _i1166 in range(_size1162): - _elem1167 = SQLDefaultConstraint() - _elem1167.read(iprot) - self.defaultConstraints.append(_elem1167) + (_etype1179, _size1176) = iprot.readListBegin() + for _i1180 in range(_size1176): + _elem1181 = SQLDefaultConstraint() + _elem1181.read(iprot) + self.defaultConstraints.append(_elem1181) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 8: if ftype == TType.LIST: self.checkConstraints = [] - (_etype1171, _size1168) = iprot.readListBegin() - for _i1172 in range(_size1168): - _elem1173 = SQLCheckConstraint() - _elem1173.read(iprot) - self.checkConstraints.append(_elem1173) + (_etype1185, _size1182) = iprot.readListBegin() + for _i1186 in range(_size1182): + _elem1187 = SQLCheckConstraint() + _elem1187.read(iprot) + self.checkConstraints.append(_elem1187) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 9: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1177, _size1174) = iprot.readListBegin() - for _i1178 in range(_size1174): - _elem1179 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1179) + (_etype1191, _size1188) = iprot.readListBegin() + for _i1192 in range(_size1188): + _elem1193 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1193) iprot.readListEnd() else: iprot.skip(ftype) @@ -25532,50 +25592,50 @@ def write(self, oprot): if self.primaryKeys is not None: oprot.writeFieldBegin('primaryKeys', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys)) - for iter1180 in self.primaryKeys: - iter1180.write(oprot) + for iter1194 in self.primaryKeys: + iter1194.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.foreignKeys is not None: oprot.writeFieldBegin('foreignKeys', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys)) - for iter1181 in self.foreignKeys: - iter1181.write(oprot) + for iter1195 in self.foreignKeys: + iter1195.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.uniqueConstraints is not None: oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 5) oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints)) - for iter1182 in self.uniqueConstraints: - iter1182.write(oprot) + for iter1196 in self.uniqueConstraints: + iter1196.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.notNullConstraints is not None: oprot.writeFieldBegin('notNullConstraints', TType.LIST, 6) oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints)) - for iter1183 in self.notNullConstraints: - iter1183.write(oprot) + for iter1197 in self.notNullConstraints: + iter1197.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.defaultConstraints is not None: oprot.writeFieldBegin('defaultConstraints', TType.LIST, 7) oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints)) - for iter1184 in self.defaultConstraints: - iter1184.write(oprot) + for iter1198 in self.defaultConstraints: + iter1198.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.checkConstraints is not None: oprot.writeFieldBegin('checkConstraints', TType.LIST, 8) oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints)) - for iter1185 in self.checkConstraints: - iter1185.write(oprot) + for iter1199 in self.checkConstraints: + iter1199.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 9) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1186 in self.processorCapabilities: - oprot.writeString(iter1186.encode('utf-8') if sys.version_info[0] == 2 else iter1186) + for iter1200 in self.processorCapabilities: + oprot.writeString(iter1200.encode('utf-8') if sys.version_info[0] == 2 else iter1200) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -25662,11 +25722,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.MAP: self.parameters = {} - (_ktype1188, _vtype1189, _size1187) = iprot.readMapBegin() - for _i1191 in range(_size1187): - _key1192 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1193 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.parameters[_key1192] = _val1193 + (_ktype1202, _vtype1203, _size1201) = iprot.readMapBegin() + for _i1205 in range(_size1201): + _key1206 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1207 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.parameters[_key1206] = _val1207 iprot.readMapEnd() else: iprot.skip(ftype) @@ -25736,9 +25796,9 @@ def write(self, oprot): if self.parameters is not None: oprot.writeFieldBegin('parameters', TType.MAP, 4) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters)) - for kiter1194, viter1195 in self.parameters.items(): - oprot.writeString(kiter1194.encode('utf-8') if sys.version_info[0] == 2 else kiter1194) - oprot.writeString(viter1195.encode('utf-8') if sys.version_info[0] == 2 else viter1195) + for kiter1208, viter1209 in self.parameters.items(): + oprot.writeString(kiter1208.encode('utf-8') if sys.version_info[0] == 2 else kiter1208) + oprot.writeString(viter1209.encode('utf-8') if sys.version_info[0] == 2 else viter1209) oprot.writeMapEnd() oprot.writeFieldEnd() if self.privileges is not None: @@ -26470,11 +26530,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.partitions = [] - (_etype1199, _size1196) = iprot.readListBegin() - for _i1200 in range(_size1196): - _elem1201 = Partition() - _elem1201.read(iprot) - self.partitions.append(_elem1201) + (_etype1213, _size1210) = iprot.readListBegin() + for _i1214 in range(_size1210): + _elem1215 = Partition() + _elem1215.read(iprot) + self.partitions.append(_elem1215) iprot.readListEnd() else: iprot.skip(ftype) @@ -26502,11 +26562,11 @@ def read(self, iprot): elif fid == 9: if ftype == TType.LIST: self.partitionColSchema = [] - (_etype1205, _size1202) = iprot.readListBegin() - for _i1206 in range(_size1202): - _elem1207 = FieldSchema() - _elem1207.read(iprot) - self.partitionColSchema.append(_elem1207) + (_etype1219, _size1216) = iprot.readListBegin() + for _i1220 in range(_size1216): + _elem1221 = FieldSchema() + _elem1221.read(iprot) + self.partitionColSchema.append(_elem1221) iprot.readListEnd() else: iprot.skip(ftype) @@ -26535,8 +26595,8 @@ def write(self, oprot): if self.partitions is not None: oprot.writeFieldBegin('partitions', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.partitions)) - for iter1208 in self.partitions: - iter1208.write(oprot) + for iter1222 in self.partitions: + iter1222.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.environmentContext is not None: @@ -26558,8 +26618,8 @@ def write(self, oprot): if self.partitionColSchema is not None: oprot.writeFieldBegin('partitionColSchema', TType.LIST, 9) oprot.writeListBegin(TType.STRUCT, len(self.partitionColSchema)) - for iter1209 in self.partitionColSchema: - iter1209.write(oprot) + for iter1223 in self.partitionColSchema: + iter1223.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -26678,10 +26738,10 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.partVals = [] - (_etype1213, _size1210) = iprot.readListBegin() - for _i1214 in range(_size1210): - _elem1215 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partVals.append(_elem1215) + (_etype1227, _size1224) = iprot.readListBegin() + for _i1228 in range(_size1224): + _elem1229 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partVals.append(_elem1229) iprot.readListEnd() else: iprot.skip(ftype) @@ -26731,8 +26791,8 @@ def write(self, oprot): if self.partVals is not None: oprot.writeFieldBegin('partVals', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.partVals)) - for iter1216 in self.partVals: - oprot.writeString(iter1216.encode('utf-8') if sys.version_info[0] == 2 else iter1216) + for iter1230 in self.partVals: + oprot.writeString(iter1230.encode('utf-8') if sys.version_info[0] == 2 else iter1230) oprot.writeListEnd() oprot.writeFieldEnd() if self.newPart is not None: @@ -26897,10 +26957,10 @@ def read(self, iprot): elif fid == 8: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1220, _size1217) = iprot.readListBegin() - for _i1221 in range(_size1217): - _elem1222 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1222) + (_etype1234, _size1231) = iprot.readListBegin() + for _i1235 in range(_size1231): + _elem1236 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1236) iprot.readListEnd() else: iprot.skip(ftype) @@ -26960,8 +27020,8 @@ def write(self, oprot): if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 8) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1223 in self.processorCapabilities: - oprot.writeString(iter1223.encode('utf-8') if sys.version_info[0] == 2 else iter1223) + for iter1237 in self.processorCapabilities: + oprot.writeString(iter1237.encode('utf-8') if sys.version_info[0] == 2 else iter1237) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -27070,10 +27130,10 @@ def read(self, iprot): elif fid == 8: if ftype == TType.LIST: self.filters = [] - (_etype1227, _size1224) = iprot.readListBegin() - for _i1228 in range(_size1224): - _elem1229 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.filters.append(_elem1229) + (_etype1241, _size1238) = iprot.readListBegin() + for _i1242 in range(_size1238): + _elem1243 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.filters.append(_elem1243) iprot.readListEnd() else: iprot.skip(ftype) @@ -27094,8 +27154,8 @@ def write(self, oprot): if self.filters is not None: oprot.writeFieldBegin('filters', TType.LIST, 8) oprot.writeListBegin(TType.STRING, len(self.filters)) - for iter1230 in self.filters: - oprot.writeString(iter1230.encode('utf-8') if sys.version_info[0] == 2 else iter1230) + for iter1244 in self.filters: + oprot.writeString(iter1244.encode('utf-8') if sys.version_info[0] == 2 else iter1244) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27139,11 +27199,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.partitionSpec = [] - (_etype1234, _size1231) = iprot.readListBegin() - for _i1235 in range(_size1231): - _elem1236 = PartitionSpec() - _elem1236.read(iprot) - self.partitionSpec.append(_elem1236) + (_etype1248, _size1245) = iprot.readListBegin() + for _i1249 in range(_size1245): + _elem1250 = PartitionSpec() + _elem1250.read(iprot) + self.partitionSpec.append(_elem1250) iprot.readListEnd() else: iprot.skip(ftype) @@ -27160,8 +27220,8 @@ def write(self, oprot): if self.partitionSpec is not None: oprot.writeFieldBegin('partitionSpec', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.partitionSpec)) - for iter1237 in self.partitionSpec: - iter1237.write(oprot) + for iter1251 in self.partitionSpec: + iter1251.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27250,10 +27310,10 @@ def read(self, iprot): elif fid == 6: if ftype == TType.LIST: self.groupNames = [] - (_etype1241, _size1238) = iprot.readListBegin() - for _i1242 in range(_size1238): - _elem1243 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.groupNames.append(_elem1243) + (_etype1255, _size1252) = iprot.readListBegin() + for _i1256 in range(_size1252): + _elem1257 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.groupNames.append(_elem1257) iprot.readListEnd() else: iprot.skip(ftype) @@ -27272,10 +27332,10 @@ def read(self, iprot): elif fid == 9: if ftype == TType.LIST: self.processorCapabilities = [] - (_etype1247, _size1244) = iprot.readListBegin() - for _i1248 in range(_size1244): - _elem1249 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.processorCapabilities.append(_elem1249) + (_etype1261, _size1258) = iprot.readListBegin() + for _i1262 in range(_size1258): + _elem1263 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.processorCapabilities.append(_elem1263) iprot.readListEnd() else: iprot.skip(ftype) @@ -27322,8 +27382,8 @@ def write(self, oprot): if self.groupNames is not None: oprot.writeFieldBegin('groupNames', TType.LIST, 6) oprot.writeListBegin(TType.STRING, len(self.groupNames)) - for iter1250 in self.groupNames: - oprot.writeString(iter1250.encode('utf-8') if sys.version_info[0] == 2 else iter1250) + for iter1264 in self.groupNames: + oprot.writeString(iter1264.encode('utf-8') if sys.version_info[0] == 2 else iter1264) oprot.writeListEnd() oprot.writeFieldEnd() if self.projectionSpec is not None: @@ -27337,8 +27397,8 @@ def write(self, oprot): if self.processorCapabilities is not None: oprot.writeFieldBegin('processorCapabilities', TType.LIST, 9) oprot.writeListBegin(TType.STRING, len(self.processorCapabilities)) - for iter1251 in self.processorCapabilities: - oprot.writeString(iter1251.encode('utf-8') if sys.version_info[0] == 2 else iter1251) + for iter1265 in self.processorCapabilities: + oprot.writeString(iter1265.encode('utf-8') if sys.version_info[0] == 2 else iter1265) oprot.writeListEnd() oprot.writeFieldEnd() if self.processorIdentifier is not None: @@ -27507,11 +27567,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fields = [] - (_etype1255, _size1252) = iprot.readListBegin() - for _i1256 in range(_size1252): - _elem1257 = FieldSchema() - _elem1257.read(iprot) - self.fields.append(_elem1257) + (_etype1269, _size1266) = iprot.readListBegin() + for _i1270 in range(_size1266): + _elem1271 = FieldSchema() + _elem1271.read(iprot) + self.fields.append(_elem1271) iprot.readListEnd() else: iprot.skip(ftype) @@ -27528,8 +27588,8 @@ def write(self, oprot): if self.fields is not None: oprot.writeFieldBegin('fields', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.fields)) - for iter1258 in self.fields: - iter1258.write(oprot) + for iter1272 in self.fields: + iter1272.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27692,11 +27752,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.fields = [] - (_etype1262, _size1259) = iprot.readListBegin() - for _i1263 in range(_size1259): - _elem1264 = FieldSchema() - _elem1264.read(iprot) - self.fields.append(_elem1264) + (_etype1276, _size1273) = iprot.readListBegin() + for _i1277 in range(_size1273): + _elem1278 = FieldSchema() + _elem1278.read(iprot) + self.fields.append(_elem1278) iprot.readListEnd() else: iprot.skip(ftype) @@ -27713,8 +27773,8 @@ def write(self, oprot): if self.fields is not None: oprot.writeFieldBegin('fields', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.fields)) - for iter1265 in self.fields: - iter1265.write(oprot) + for iter1279 in self.fields: + iter1279.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27785,10 +27845,10 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.partVals = [] - (_etype1269, _size1266) = iprot.readListBegin() - for _i1270 in range(_size1266): - _elem1271 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partVals.append(_elem1271) + (_etype1283, _size1280) = iprot.readListBegin() + for _i1284 in range(_size1280): + _elem1285 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partVals.append(_elem1285) iprot.readListEnd() else: iprot.skip(ftype) @@ -27827,8 +27887,8 @@ def write(self, oprot): if self.partVals is not None: oprot.writeFieldBegin('partVals', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.partVals)) - for iter1272 in self.partVals: - oprot.writeString(iter1272.encode('utf-8') if sys.version_info[0] == 2 else iter1272) + for iter1286 in self.partVals: + oprot.writeString(iter1286.encode('utf-8') if sys.version_info[0] == 2 else iter1286) oprot.writeListEnd() oprot.writeFieldEnd() if self.validWriteIdList is not None: @@ -28073,11 +28133,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.partitions = [] - (_etype1276, _size1273) = iprot.readListBegin() - for _i1277 in range(_size1273): - _elem1278 = Partition() - _elem1278.read(iprot) - self.partitions.append(_elem1278) + (_etype1290, _size1287) = iprot.readListBegin() + for _i1291 in range(_size1287): + _elem1292 = Partition() + _elem1292.read(iprot) + self.partitions.append(_elem1292) iprot.readListEnd() else: iprot.skip(ftype) @@ -28094,8 +28154,8 @@ def write(self, oprot): if self.partitions is not None: oprot.writeFieldBegin('partitions', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.partitions)) - for iter1279 in self.partitions: - iter1279.write(oprot) + for iter1293 in self.partitions: + iter1293.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28280,10 +28340,10 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.partValues = [] - (_etype1283, _size1280) = iprot.readListBegin() - for _i1284 in range(_size1280): - _elem1285 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partValues.append(_elem1285) + (_etype1297, _size1294) = iprot.readListBegin() + for _i1298 in range(_size1294): + _elem1299 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partValues.append(_elem1299) iprot.readListEnd() else: iprot.skip(ftype) @@ -28327,8 +28387,8 @@ def write(self, oprot): if self.partValues is not None: oprot.writeFieldBegin('partValues', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.partValues)) - for iter1286 in self.partValues: - oprot.writeString(iter1286.encode('utf-8') if sys.version_info[0] == 2 else iter1286) + for iter1300 in self.partValues: + oprot.writeString(iter1300.encode('utf-8') if sys.version_info[0] == 2 else iter1300) oprot.writeListEnd() oprot.writeFieldEnd() if self.maxParts is not None: @@ -28388,10 +28448,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.names = [] - (_etype1290, _size1287) = iprot.readListBegin() - for _i1291 in range(_size1287): - _elem1292 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.names.append(_elem1292) + (_etype1304, _size1301) = iprot.readListBegin() + for _i1305 in range(_size1301): + _elem1306 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.names.append(_elem1306) iprot.readListEnd() else: iprot.skip(ftype) @@ -28408,8 +28468,8 @@ def write(self, oprot): if self.names is not None: oprot.writeFieldBegin('names', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.names)) - for iter1293 in self.names: - oprot.writeString(iter1293.encode('utf-8') if sys.version_info[0] == 2 else iter1293) + for iter1307 in self.names: + oprot.writeString(iter1307.encode('utf-8') if sys.version_info[0] == 2 else iter1307) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28488,10 +28548,10 @@ def read(self, iprot): elif fid == 4: if ftype == TType.LIST: self.partVals = [] - (_etype1297, _size1294) = iprot.readListBegin() - for _i1298 in range(_size1294): - _elem1299 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partVals.append(_elem1299) + (_etype1311, _size1308) = iprot.readListBegin() + for _i1312 in range(_size1308): + _elem1313 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partVals.append(_elem1313) iprot.readListEnd() else: iprot.skip(ftype) @@ -28508,10 +28568,10 @@ def read(self, iprot): elif fid == 7: if ftype == TType.LIST: self.groupNames = [] - (_etype1303, _size1300) = iprot.readListBegin() - for _i1304 in range(_size1300): - _elem1305 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.groupNames.append(_elem1305) + (_etype1317, _size1314) = iprot.readListBegin() + for _i1318 in range(_size1314): + _elem1319 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.groupNames.append(_elem1319) iprot.readListEnd() else: iprot.skip(ftype) @@ -28555,8 +28615,8 @@ def write(self, oprot): if self.partVals is not None: oprot.writeFieldBegin('partVals', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.partVals)) - for iter1306 in self.partVals: - oprot.writeString(iter1306.encode('utf-8') if sys.version_info[0] == 2 else iter1306) + for iter1320 in self.partVals: + oprot.writeString(iter1320.encode('utf-8') if sys.version_info[0] == 2 else iter1320) oprot.writeListEnd() oprot.writeFieldEnd() if self.maxParts is not None: @@ -28570,8 +28630,8 @@ def write(self, oprot): if self.groupNames is not None: oprot.writeFieldBegin('groupNames', TType.LIST, 7) oprot.writeListBegin(TType.STRING, len(self.groupNames)) - for iter1307 in self.groupNames: - oprot.writeString(iter1307.encode('utf-8') if sys.version_info[0] == 2 else iter1307) + for iter1321 in self.groupNames: + oprot.writeString(iter1321.encode('utf-8') if sys.version_info[0] == 2 else iter1321) oprot.writeListEnd() oprot.writeFieldEnd() if self.validWriteIdList is not None: @@ -28631,11 +28691,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.partitions = [] - (_etype1311, _size1308) = iprot.readListBegin() - for _i1312 in range(_size1308): - _elem1313 = Partition() - _elem1313.read(iprot) - self.partitions.append(_elem1313) + (_etype1325, _size1322) = iprot.readListBegin() + for _i1326 in range(_size1322): + _elem1327 = Partition() + _elem1327.read(iprot) + self.partitions.append(_elem1327) iprot.readListEnd() else: iprot.skip(ftype) @@ -28652,8 +28712,8 @@ def write(self, oprot): if self.partitions is not None: oprot.writeFieldBegin('partitions', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.partitions)) - for iter1314 in self.partitions: - iter1314.write(oprot) + for iter1328 in self.partitions: + iter1328.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28817,11 +28877,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.replicationMetricList = [] - (_etype1318, _size1315) = iprot.readListBegin() - for _i1319 in range(_size1315): - _elem1320 = ReplicationMetrics() - _elem1320.read(iprot) - self.replicationMetricList.append(_elem1320) + (_etype1332, _size1329) = iprot.readListBegin() + for _i1333 in range(_size1329): + _elem1334 = ReplicationMetrics() + _elem1334.read(iprot) + self.replicationMetricList.append(_elem1334) iprot.readListEnd() else: iprot.skip(ftype) @@ -28838,8 +28898,8 @@ def write(self, oprot): if self.replicationMetricList is not None: oprot.writeFieldBegin('replicationMetricList', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.replicationMetricList)) - for iter1321 in self.replicationMetricList: - iter1321.write(oprot) + for iter1335 in self.replicationMetricList: + iter1335.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28964,10 +29024,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.excludeTxnTypes = [] - (_etype1325, _size1322) = iprot.readListBegin() - for _i1326 in range(_size1322): - _elem1327 = iprot.readI32() - self.excludeTxnTypes.append(_elem1327) + (_etype1339, _size1336) = iprot.readListBegin() + for _i1340 in range(_size1336): + _elem1341 = iprot.readI32() + self.excludeTxnTypes.append(_elem1341) iprot.readListEnd() else: iprot.skip(ftype) @@ -28984,8 +29044,8 @@ def write(self, oprot): if self.excludeTxnTypes is not None: oprot.writeFieldBegin('excludeTxnTypes', TType.LIST, 1) oprot.writeListBegin(TType.I32, len(self.excludeTxnTypes)) - for iter1328 in self.excludeTxnTypes: - oprot.writeI32(iter1328) + for iter1342 in self.excludeTxnTypes: + oprot.writeI32(iter1342) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -32453,6 +32513,9 @@ def __ne__(self, other): (1, TType.I64, 'lastEvent', None, None, ), # 1 (2, TType.I32, 'maxEvents', None, None, ), # 2 (3, TType.LIST, 'eventTypeSkipList', (TType.STRING, 'UTF8', False), None, ), # 3 + (4, TType.STRING, 'catName', 'UTF8', None, ), # 4 + (5, TType.STRING, 'dbName', 'UTF8', None, ), # 5 + (6, TType.LIST, 'tableNames', (TType.STRING, 'UTF8', False), None, ), # 6 ) all_structs.append(NotificationEvent) NotificationEvent.thrift_spec = ( @@ -32484,6 +32547,7 @@ def __ne__(self, other): (3, TType.STRING, 'catName', 'UTF8', None, ), # 3 (4, TType.I64, 'toEventId', None, None, ), # 4 (5, TType.I64, 'limit', None, None, ), # 5 + (6, TType.LIST, 'tableNames', (TType.STRING, 'UTF8', False), None, ), # 6 ) all_structs.append(NotificationEventsCountResponse) NotificationEventsCountResponse.thrift_spec = ( diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb index 91eae5981642..72595ed42688 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -5079,11 +5079,17 @@ class NotificationEventRequest LASTEVENT = 1 MAXEVENTS = 2 EVENTTYPESKIPLIST = 3 + CATNAME = 4 + DBNAME = 5 + TABLENAMES = 6 FIELDS = { LASTEVENT => {:type => ::Thrift::Types::I64, :name => 'lastEvent'}, MAXEVENTS => {:type => ::Thrift::Types::I32, :name => 'maxEvents', :optional => true}, - EVENTTYPESKIPLIST => {:type => ::Thrift::Types::LIST, :name => 'eventTypeSkipList', :element => {:type => ::Thrift::Types::STRING}, :optional => true} + EVENTTYPESKIPLIST => {:type => ::Thrift::Types::LIST, :name => 'eventTypeSkipList', :element => {:type => ::Thrift::Types::STRING}, :optional => true}, + CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName', :optional => true}, + TABLENAMES => {:type => ::Thrift::Types::LIST, :name => 'tableNames', :element => {:type => ::Thrift::Types::STRING}, :optional => true} } def struct_fields; FIELDS; end @@ -5170,13 +5176,15 @@ class NotificationEventsCountRequest CATNAME = 3 TOEVENTID = 4 LIMIT = 5 + TABLENAMES = 6 FIELDS = { FROMEVENTID => {:type => ::Thrift::Types::I64, :name => 'fromEventId'}, DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}, CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, TOEVENTID => {:type => ::Thrift::Types::I64, :name => 'toEventId', :optional => true}, - LIMIT => {:type => ::Thrift::Types::I64, :name => 'limit', :optional => true} + LIMIT => {:type => ::Thrift::Types::I64, :name => 'limit', :optional => true}, + TABLENAMES => {:type => ::Thrift::Types::LIST, :name => 'tableNames', :element => {:type => ::Thrift::Types::STRING}, :optional => true} } def struct_fields; FIELDS; end diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift index cd56758873a5..935e286c85d4 100644 --- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift +++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift @@ -1483,6 +1483,9 @@ struct NotificationEventRequest { 1: required i64 lastEvent, 2: optional i32 maxEvents, 3: optional list eventTypeSkipList, + 4: optional string catName, + 5: optional string dbName, + 6: optional list tableNames } struct NotificationEvent { @@ -1509,7 +1512,8 @@ struct NotificationEventsCountRequest { 2: required string dbName, 3: optional string catName, 4: optional i64 toEventId, - 5: optional i64 limit + 5: optional i64 limit, + 6: optional list tableNames } struct NotificationEventsCountResponse { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index b3e8bbb82c24..05a421e93fa8 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -11845,8 +11845,36 @@ public NotificationEventResponse getNextNotification(NotificationEventRequest rq long lastEvent = rqst.getLastEvent(); List parameterVals = new ArrayList<>(); parameterVals.add(lastEvent); + // filterBuilder parameter is used for construction of conditional clause in the select query StringBuilder filterBuilder = new StringBuilder("eventId > para" + parameterVals.size()); + // parameterBuilder parameter is used for specify what types of parameters will go into the filterBuilder StringBuilder parameterBuilder = new StringBuilder("java.lang.Long para" + parameterVals.size()); + /* A fully constructed query would like: + -> filterBuilder: eventId > para0 && catalogName == para1 && dbName == para2 && (tableName == para3 + || tableName == para4) && eventType != para5 + -> parameterBuilder: java.lang.Long para0, java.lang.String para1, java.lang.String para2 + , java.lang.String para3, java.lang.String para4, java.lang.String para5 + */ + if (rqst.isSetCatName()) { + parameterVals.add(normalizeIdentifier(rqst.getCatName())); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append(" && catalogName == para" + parameterVals.size()); + } + if (rqst.isSetDbName()) { + parameterVals.add(normalizeIdentifier(rqst.getDbName())); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append(" && dbName == para" + parameterVals.size()); + } + if (rqst.isSetTableNames() && !rqst.getTableNames().isEmpty()) { + filterBuilder.append(" && ("); + for (String tableName : rqst.getTableNames()) { + parameterVals.add(normalizeIdentifier(tableName)); + parameterBuilder.append(", java.lang.String para" + parameterVals.size()); + filterBuilder.append("tableName == para" + parameterVals.size()+ " || "); + } + filterBuilder.setLength(filterBuilder.length() - 4); // remove the last " || " + filterBuilder.append(") "); + } if (rqst.isSetEventTypeSkipList()) { for (String eventType : rqst.getEventTypeSkipList()) { parameterVals.add(eventType); @@ -12212,6 +12240,17 @@ public NotificationEventsCountResponse getNotificationEventsCount(NotificationEv paramSpecs = paramSpecs + ", java.lang.Long toEventId"; paramVals.add(Long.valueOf(toEventId)); } + // Specify list of table names in the query string and parameter types + if (rqst.isSetTableNames() && !rqst.getTableNames().isEmpty()) { + queryStr = queryStr + " && ("; + for (String tableName : rqst.getTableNames()) { + paramVals.add(tableName.toLowerCase()); + queryStr = queryStr + "tableName == tableName" + paramVals.size() + " || "; + paramSpecs = paramSpecs + ", java.lang.String tableName" + paramVals.size(); + } + queryStr = queryStr.substring(0, queryStr.length() - 4); // remove the last " || " + queryStr += ")"; + } query = pm.newQuery(queryStr); query.declareParameters(paramSpecs); From f27320eb3f394f939b693d960b1eb619289671b9 Mon Sep 17 00:00:00 2001 From: rtrivedi12 <32664785+rtrivedi12@users.noreply.github.com> Date: Fri, 3 Nov 2023 05:21:08 +0530 Subject: [PATCH 035/179] HIVE-24815: Remove IDXS Table from Metastore Schema (#4370) * HIVE-24815: Remove IDXS Table from Metastore Schema --- .../hadoop/hive/metastore/model/MIndex.java | 200 ------------------ .../schematool/SchemaToolTaskValidate.java | 1 - .../src/main/resources/package.jdo | 47 ---- .../derby/hive-schema-4.0.0-beta-2.derby.sql | 19 -- ...ade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql | 4 + .../mssql/hive-schema-4.0.0-beta-2.mssql.sql | 49 ----- ...ade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql | 4 + .../mysql/hive-schema-4.0.0-beta-2.mysql.sql | 43 ---- ...ade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql | 4 + .../hive-schema-4.0.0-beta-2.oracle.sql | 50 ----- ...de-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql | 5 + .../hive-schema-4.0.0-beta-2.postgres.sql | 112 ---------- ...-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql | 5 + 13 files changed, 22 insertions(+), 521 deletions(-) delete mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MIndex.java diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MIndex.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MIndex.java deleted file mode 100644 index 1d679619d039..000000000000 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MIndex.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.metastore.model; - -import java.util.Map; - -/** - * Represents hive's index definition. - */ -public class MIndex { - - private String indexName; - private MTable origTable; - private int createTime; - private int lastAccessTime; - private Map parameters; - private MTable indexTable; - private MStorageDescriptor sd; - private String indexHandlerClass; - private boolean deferredRebuild; - - public MIndex() {} - -/** - * @param indexName - * @param baseTable - * @param createTime - * @param lastAccessTime - * @param parameters - * @param indexTable - * @param sd - * @param indexHandlerClass - * @param deferredRebuild - */ - public MIndex(String indexName, MTable baseTable, int createTime, - int lastAccessTime, Map parameters, MTable indexTable, - MStorageDescriptor sd, String indexHandlerClass, boolean deferredRebuild) { - super(); - this.indexName = indexName; - this.origTable = baseTable; - this.createTime = createTime; - this.lastAccessTime = lastAccessTime; - this.parameters = parameters; - this.indexTable = indexTable; - this.sd = sd; - this.indexHandlerClass = indexHandlerClass; - this.deferredRebuild = deferredRebuild; - } - - - - /** - * @return index name - */ - public String getIndexName() { - return indexName; - } - - /** - * @param indexName index name - */ - public void setIndexName(String indexName) { - this.indexName = indexName; - } - - /** - * @return create time - */ - public int getCreateTime() { - return createTime; - } - - /** - * @param createTime create time - */ - public void setCreateTime(int createTime) { - this.createTime = createTime; - } - - /** - * @return last access time - */ - public int getLastAccessTime() { - return lastAccessTime; - } - - /** - * @param lastAccessTime last access time - */ - public void setLastAccessTime(int lastAccessTime) { - this.lastAccessTime = lastAccessTime; - } - - /** - * @return parameters - */ - public Map getParameters() { - return parameters; - } - - /** - * @param parameters parameters - */ - public void setParameters(Map parameters) { - this.parameters = parameters; - } - - /** - * @return original table - */ - public MTable getOrigTable() { - return origTable; - } - - /** - * @param origTable - */ - public void setOrigTable(MTable origTable) { - this.origTable = origTable; - } - - /** - * @return index table - */ - public MTable getIndexTable() { - return indexTable; - } - - /** - * @param indexTable - */ - public void setIndexTable(MTable indexTable) { - this.indexTable = indexTable; - } - - /** - * @return storage descriptor - */ - public MStorageDescriptor getSd() { - return sd; - } - - /** - * @param sd - */ - public void setSd(MStorageDescriptor sd) { - this.sd = sd; - } - - /** - * @return indexHandlerClass - */ - public String getIndexHandlerClass() { - return indexHandlerClass; - } - - /** - * @param indexHandlerClass - */ - public void setIndexHandlerClass(String indexHandlerClass) { - this.indexHandlerClass = indexHandlerClass; - } - - /** - * @return auto rebuild - */ - public boolean isDeferredRebuild() { - return deferredRebuild; - } - - /** - * @return auto rebuild - */ - public boolean getDeferredRebuild() { - return deferredRebuild; - } - - /** - * @param deferredRebuild - */ - public void setDeferredRebuild(boolean deferredRebuild) { - this.deferredRebuild = deferredRebuild; - } -} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/SchemaToolTaskValidate.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/SchemaToolTaskValidate.java index e3428e8cc28e..91c9949f3175 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/SchemaToolTaskValidate.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/SchemaToolTaskValidate.java @@ -140,7 +140,6 @@ boolean validateSequences(Connection conn) throws HiveMetaException { .put("MPartition", Pair.of("PARTITIONS", "PART_ID")) .put("MPartitionColumnStatistics", Pair.of("PART_COL_STATS", "CS_ID")) .put("MFunction", Pair.of("FUNCS", "FUNC_ID")) - .put("MIndex", Pair.of("IDXS", "INDEX_ID")) .put("MStringList", Pair.of("SKEWED_STRING_LIST", "STRING_LIST_ID")) .build(); diff --git a/standalone-metastore/metastore-server/src/main/resources/package.jdo b/standalone-metastore/metastore-server/src/main/resources/package.jdo index a6b10ac6a6cb..c7493d3a7902 100644 --- a/standalone-metastore/metastore-server/src/main/resources/package.jdo +++ b/standalone-metastore/metastore-server/src/main/resources/package.jdo @@ -546,53 +546,6 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql index cfb5a7c3d5da..9feaa6a0c3ff 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql @@ -50,10 +50,6 @@ CREATE TABLE "APP"."DB_PRIVS" ("DB_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTE CREATE TABLE "APP"."DC_PRIVS" ("DC_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "NAME" VARCHAR(128), "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "DC_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); -CREATE TABLE "APP"."IDXS" ("INDEX_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DEFERRED_REBUILD" CHAR(1) NOT NULL, "INDEX_HANDLER_CLASS" VARCHAR(4000), "INDEX_NAME" VARCHAR(128), "INDEX_TBL_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "ORIG_TBL_ID" BIGINT, "SD_ID" BIGINT); - -CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); - CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT, "WRITE_ID" BIGINT DEFAULT 0); CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER); @@ -264,8 +260,6 @@ INSERT INTO "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") SELECT * FROM ( -- DDL Statements for indexes -- ---------------------------------------------- -CREATE UNIQUE INDEX "APP"."UNIQUEINDEX" ON "APP"."IDXS" ("INDEX_NAME", "ORIG_TBL_ID"); - CREATE INDEX "APP"."TABLECOLUMNPRIVILEGEINDEX" ON "APP"."TBL_COL_PRIVS" ("AUTHORIZER", "TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); CREATE UNIQUE INDEX "APP"."DBPRIVILEGEINDEX" ON "APP"."DB_PRIVS" ("AUTHORIZER", "DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE"); @@ -324,7 +318,6 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); -- ---------------------------------------------- -- primary/unique -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_PK" PRIMARY KEY ("INDEX_ID"); ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_PK" PRIMARY KEY ("TBL_COLUMN_GRANT_ID"); @@ -334,8 +327,6 @@ ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_PK" PRIMARY KEY ("DB_GRANT ALTER TABLE "APP"."DC_PRIVS" ADD CONSTRAINT "DC_PRIVS_PK" PRIMARY KEY ("DC_GRANT_ID"); -ALTER TABLE "APP"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_PK" PRIMARY KEY ("INDEX_ID", "PARAM_KEY"); - ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEY_PK" PRIMARY KEY ("TBL_ID", "PKEY_NAME"); ALTER TABLE "APP"."SEQUENCE_TABLE" ADD CONSTRAINT "SEQUENCE_TABLE_PK" PRIMARY KEY ("SEQUENCE_NAME"); @@ -424,18 +415,10 @@ ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID"); -- foreign -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK3" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; -ALTER TABLE "APP"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_FK1" FOREIGN KEY ("INDEX_ID") REFERENCES "APP"."IDXS" ("INDEX_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; - ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEYS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; @@ -538,8 +521,6 @@ ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") R -- DDL Statements for checks -- ---------------------------------------------- -ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "SQL110318025504980" CHECK (DEFERRED_REBUILD IN ('Y','N')); - ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); -- ---------------------------- diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql index 094383b2b612..2695e10742c8 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql @@ -1,2 +1,6 @@ +-- HIVE-24815 Remove "IDXS" Table from Metastore Schema +DROP TABLE "APP"."INDEX_PARAMS"; +DROP TABLE "APP"."IDXS"; + -- This needs to be the last thing done. Insert any changes above this line. UPDATE "APP".VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql index c0cefec9a3e7..190a245c2b66 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql @@ -54,22 +54,6 @@ CREATE TABLE MASTER_KEYS ALTER TABLE MASTER_KEYS ADD CONSTRAINT MASTER_KEYS_PK PRIMARY KEY (KEY_ID); --- Table IDXS for classes [org.apache.hadoop.hive.metastore.model.MIndex] -CREATE TABLE IDXS -( - INDEX_ID bigint NOT NULL, - CREATE_TIME int NOT NULL, - DEFERRED_REBUILD bit NOT NULL, - INDEX_HANDLER_CLASS nvarchar(4000) NULL, - INDEX_NAME nvarchar(128) NULL, - INDEX_TBL_ID bigint NULL, - LAST_ACCESS_TIME int NOT NULL, - ORIG_TBL_ID bigint NULL, - SD_ID bigint NULL -); - -ALTER TABLE IDXS ADD CONSTRAINT IDXS_PK PRIMARY KEY (INDEX_ID); - -- Table PART_COL_STATS for classes [org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics] CREATE TABLE PART_COL_STATS ( @@ -592,16 +576,6 @@ CREATE TABLE DATABASE_PARAMS ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_PK PRIMARY KEY (DB_ID,PARAM_KEY); --- Table INDEX_PARAMS for join relationship -CREATE TABLE INDEX_PARAMS -( - INDEX_ID bigint NOT NULL, - PARAM_KEY nvarchar(256) NOT NULL, - PARAM_VALUE nvarchar(4000) NULL -); - -ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_PK PRIMARY KEY (INDEX_ID,PARAM_KEY); - -- Table COLUMNS_V2 for join relationship CREATE TABLE COLUMNS_V2 ( @@ -746,22 +720,6 @@ CREATE UNIQUE INDEX UNIQUE_CTLG ON CTLGS ("NAME"); -- Constraints for table MASTER_KEYS for class(es) [org.apache.hadoop.hive.metastore.model.MMasterKey] --- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex] -ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK1 FOREIGN KEY (INDEX_TBL_ID) REFERENCES TBLS (TBL_ID) ; - -ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) ; - -ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK3 FOREIGN KEY (ORIG_TBL_ID) REFERENCES TBLS (TBL_ID) ; - -CREATE UNIQUE INDEX UNIQUEINDEX ON IDXS (INDEX_NAME,ORIG_TBL_ID); - -CREATE INDEX IDXS_N51 ON IDXS (SD_ID); - -CREATE INDEX IDXS_N50 ON IDXS (ORIG_TBL_ID); - -CREATE INDEX IDXS_N49 ON IDXS (INDEX_TBL_ID); - - -- Constraints for table PART_COL_STATS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics] ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) ; @@ -973,13 +931,6 @@ ALTER TABLE DATABASE_PARAMS ADD CONSTRAINT DATABASE_PARAMS_FK1 FOREIGN KEY (DB_I CREATE INDEX DATABASE_PARAMS_N49 ON DATABASE_PARAMS (DB_ID); - --- Constraints for table INDEX_PARAMS -ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_FK1 FOREIGN KEY (INDEX_ID) REFERENCES IDXS (INDEX_ID) ; - -CREATE INDEX INDEX_PARAMS_N49 ON INDEX_PARAMS (INDEX_ID); - - -- Constraints for table COLUMNS_V2 ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_FK1 FOREIGN KEY (CD_ID) REFERENCES CDS (CD_ID) ; diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql index 914dc7832522..a7afd05ceb3a 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql @@ -1,5 +1,9 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE; +-- HIVE-24815: Remove "IDXS" Table from Metastore Schema +DROP TABLE INDEX_PARAMS; +DROP TABLE IDXS; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql index 58b84e2a210b..d97577f7d211 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql @@ -159,49 +159,6 @@ CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` ( ) ENGINE=InnoDB DEFAULT CHARSET=latin1; /*!40101 SET character_set_client = @saved_cs_client */; --- --- Table structure for table `IDXS` --- - -/*!40101 SET @saved_cs_client = @@character_set_client */; -/*!40101 SET character_set_client = utf8 */; -CREATE TABLE IF NOT EXISTS `IDXS` ( - `INDEX_ID` bigint(20) NOT NULL, - `CREATE_TIME` int(11) NOT NULL, - `DEFERRED_REBUILD` bit(1) NOT NULL, - `INDEX_HANDLER_CLASS` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL, - `INDEX_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL, - `INDEX_TBL_ID` bigint(20) DEFAULT NULL, - `LAST_ACCESS_TIME` int(11) NOT NULL, - `ORIG_TBL_ID` bigint(20) DEFAULT NULL, - `SD_ID` bigint(20) DEFAULT NULL, - PRIMARY KEY (`INDEX_ID`), - UNIQUE KEY `UNIQUEINDEX` (`INDEX_NAME`,`ORIG_TBL_ID`), - KEY `IDXS_N51` (`SD_ID`), - KEY `IDXS_N50` (`INDEX_TBL_ID`), - KEY `IDXS_N49` (`ORIG_TBL_ID`), - CONSTRAINT `IDXS_FK1` FOREIGN KEY (`ORIG_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`), - CONSTRAINT `IDXS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`), - CONSTRAINT `IDXS_FK3` FOREIGN KEY (`INDEX_TBL_ID`) REFERENCES `TBLS` (`TBL_ID`) -) ENGINE=InnoDB DEFAULT CHARSET=latin1; -/*!40101 SET character_set_client = @saved_cs_client */; - --- --- Table structure for table `INDEX_PARAMS` --- - -/*!40101 SET @saved_cs_client = @@character_set_client */; -/*!40101 SET character_set_client = utf8 */; -CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` ( - `INDEX_ID` bigint(20) NOT NULL, - `PARAM_KEY` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL, - `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL, - PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`), - KEY `INDEX_PARAMS_N49` (`INDEX_ID`), - CONSTRAINT `INDEX_PARAMS_FK1` FOREIGN KEY (`INDEX_ID`) REFERENCES `IDXS` (`INDEX_ID`) -) ENGINE=InnoDB DEFAULT CHARSET=latin1; -/*!40101 SET character_set_client = @saved_cs_client */; - -- -- Table structure for table `NUCLEUS_TABLES` -- diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql index a22021c2d801..9d3bcb028abe 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql @@ -1,5 +1,9 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE; +-- HIVE-24815: Remove "IDXS" Table from Metastore Schema +DROP TABLE `INDEX_PARAMS`; +DROP TABLE `IDXS`; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql index 3f0f0a0d4b79..a2fb02c89111 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql @@ -184,16 +184,6 @@ CREATE TABLE PARTITIONS ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_PK PRIMARY KEY (PART_ID); --- Table INDEX_PARAMS for join relationship -CREATE TABLE INDEX_PARAMS -( - INDEX_ID NUMBER NOT NULL, - PARAM_KEY VARCHAR2(256) NOT NULL, - PARAM_VALUE VARCHAR2(4000) NULL -); - -ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_PK PRIMARY KEY (INDEX_ID,PARAM_KEY); - -- Table TBL_COL_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege] CREATE TABLE TBL_COL_PRIVS ( @@ -212,22 +202,6 @@ CREATE TABLE TBL_COL_PRIVS ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_PK PRIMARY KEY (TBL_COLUMN_GRANT_ID); --- Table IDXS for classes [org.apache.hadoop.hive.metastore.model.MIndex] -CREATE TABLE IDXS -( - INDEX_ID NUMBER NOT NULL, - CREATE_TIME NUMBER (10) NOT NULL, - DEFERRED_REBUILD NUMBER(1) NOT NULL CHECK (DEFERRED_REBUILD IN (1,0)), - INDEX_HANDLER_CLASS VARCHAR2(4000) NULL, - INDEX_NAME VARCHAR2(128) NULL, - INDEX_TBL_ID NUMBER NULL, - LAST_ACCESS_TIME NUMBER (10) NOT NULL, - ORIG_TBL_ID NUMBER NULL, - SD_ID NUMBER NULL -); - -ALTER TABLE IDXS ADD CONSTRAINT IDXS_PK PRIMARY KEY (INDEX_ID); - -- Table BUCKETING_COLS for join relationship CREATE TABLE BUCKETING_COLS ( @@ -793,13 +767,6 @@ CREATE INDEX PARTITIONS_N50 ON PARTITIONS (TBL_ID); CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (PART_NAME,TBL_ID); - --- Constraints for table INDEX_PARAMS -ALTER TABLE INDEX_PARAMS ADD CONSTRAINT INDEX_PARAMS_FK1 FOREIGN KEY (INDEX_ID) REFERENCES IDXS (INDEX_ID) INITIALLY DEFERRED ; - -CREATE INDEX INDEX_PARAMS_N49 ON INDEX_PARAMS (INDEX_ID); - - -- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege] ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ; @@ -807,23 +774,6 @@ CREATE INDEX TABLECOLUMNPRIVILEGEINDEX ON TBL_COL_PRIVS (AUTHORIZER,TBL_ID,"COLU CREATE INDEX TBL_COL_PRIVS_N49 ON TBL_COL_PRIVS (TBL_ID); - --- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex] -ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK2 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ; - -ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK1 FOREIGN KEY (ORIG_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ; - -ALTER TABLE IDXS ADD CONSTRAINT IDXS_FK3 FOREIGN KEY (INDEX_TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ; - -CREATE UNIQUE INDEX UNIQUEINDEX ON IDXS (INDEX_NAME,ORIG_TBL_ID); - -CREATE INDEX IDXS_N50 ON IDXS (INDEX_TBL_ID); - -CREATE INDEX IDXS_N51 ON IDXS (SD_ID); - -CREATE INDEX IDXS_N49 ON IDXS (ORIG_TBL_ID); - - -- Constraints for table BUCKETING_COLS ALTER TABLE BUCKETING_COLS ADD CONSTRAINT BUCKETING_COLS_FK1 FOREIGN KEY (SD_ID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ; diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql index 3a5ce634daf4..53c7cb140d77 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql @@ -1,5 +1,10 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS Status from dual; + +-- HIVE-24815: Remove "IDXS" Table from Metastore Schema +DROP TABLE INDEX_PARAMS; +DROP TABLE IDXS; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS Status from dual; diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql index 93c3561649b2..b247bc3fda09 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql @@ -142,35 +142,6 @@ CREATE TABLE "GLOBAL_PRIVS" ( "AUTHORIZER" character varying(128) DEFAULT NULL::character varying ); - --- --- Name: IDXS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE TABLE "IDXS" ( - "INDEX_ID" bigint NOT NULL, - "CREATE_TIME" bigint NOT NULL, - "DEFERRED_REBUILD" boolean NOT NULL, - "INDEX_HANDLER_CLASS" character varying(4000) DEFAULT NULL::character varying, - "INDEX_NAME" character varying(128) DEFAULT NULL::character varying, - "INDEX_TBL_ID" bigint, - "LAST_ACCESS_TIME" bigint NOT NULL, - "ORIG_TBL_ID" bigint, - "SD_ID" bigint -); - - --- --- Name: INDEX_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE TABLE "INDEX_PARAMS" ( - "INDEX_ID" bigint NOT NULL, - "PARAM_KEY" character varying(256) NOT NULL, - "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying -); - - -- -- Name: NUCLEUS_TABLES; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace: -- @@ -833,22 +804,6 @@ ALTER TABLE ONLY "GLOBAL_PRIVS" ADD CONSTRAINT "GLOBAL_PRIVS_pkey" PRIMARY KEY ("USER_GRANT_ID"); --- --- Name: IDXS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace: --- - -ALTER TABLE ONLY "IDXS" - ADD CONSTRAINT "IDXS_pkey" PRIMARY KEY ("INDEX_ID"); - - --- --- Name: INDEX_PARAMS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace: --- - -ALTER TABLE ONLY "INDEX_PARAMS" - ADD CONSTRAINT "INDEX_PARAMS_pkey" PRIMARY KEY ("INDEX_ID", "PARAM_KEY"); - - -- -- Name: ONE_ROW_CONSTRAINT; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace: -- @@ -1066,13 +1021,6 @@ ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_pkey" PRIMARY KEY -- ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_pkey" PRIMARY KEY("CS_ID"); --- --- Name: UNIQUEINDEX; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace: --- - -ALTER TABLE ONLY "IDXS" - ADD CONSTRAINT "UNIQUEINDEX" UNIQUE ("INDEX_NAME", "ORIG_TBL_ID"); - -- -- Name: UNIQUEPARTITION; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace: @@ -1173,34 +1121,6 @@ CREATE INDEX "DB_PRIVS_N49" ON "DB_PRIVS" USING btree ("DB_ID"); CREATE INDEX "DC_PRIVS_N49" ON "DC_PRIVS" USING btree ("NAME"); --- --- Name: IDXS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE INDEX "IDXS_N49" ON "IDXS" USING btree ("ORIG_TBL_ID"); - - --- --- Name: IDXS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE INDEX "IDXS_N50" ON "IDXS" USING btree ("INDEX_TBL_ID"); - - --- --- Name: IDXS_N51; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE INDEX "IDXS_N51" ON "IDXS" USING btree ("SD_ID"); - - --- --- Name: INDEX_PARAMS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE INDEX "INDEX_PARAMS_N49" ON "INDEX_PARAMS" USING btree ("INDEX_ID"); - - -- -- Name: PARTITIONCOLUMNPRIVILEGEINDEX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: -- @@ -1465,38 +1385,6 @@ ALTER TABLE ONLY "DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_DB_ID_fkey" FOREIGN KEY ("DB_ID") REFERENCES "DBS"("DB_ID") DEFERRABLE; --- --- Name: IDXS_INDEX_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser --- - -ALTER TABLE ONLY "IDXS" - ADD CONSTRAINT "IDXS_INDEX_TBL_ID_fkey" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE; - - --- --- Name: IDXS_ORIG_TBL_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser --- - -ALTER TABLE ONLY "IDXS" - ADD CONSTRAINT "IDXS_ORIG_TBL_ID_fkey" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE; - - --- --- Name: IDXS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser --- - -ALTER TABLE ONLY "IDXS" - ADD CONSTRAINT "IDXS_SD_ID_fkey" FOREIGN KEY ("SD_ID") REFERENCES "SDS"("SD_ID") DEFERRABLE; - - --- --- Name: INDEX_PARAMS_INDEX_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser --- - -ALTER TABLE ONLY "INDEX_PARAMS" - ADD CONSTRAINT "INDEX_PARAMS_INDEX_ID_fkey" FOREIGN KEY ("INDEX_ID") REFERENCES "IDXS"("INDEX_ID") DEFERRABLE; - - -- -- Name: PARTITIONS_SD_ID_fkey; Type: FK CONSTRAINT; Schema: public; Owner: hiveuser -- diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql index dd3cab5595a7..1d285bbe842f 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql @@ -1,4 +1,9 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2'; + +-- HIVE-24815: Remove "IDXS" Table from Metastore Schema +DROP TABLE IF EXISTS "INDEX_PARAMS"; +DROP TABLE IF EXISTS "IDXS"; + -- These lines need to be last. Insert any changes above. UPDATE "VERSION" SET "SCHEMA_VERSION"='4.0.0-beta-2', "VERSION_COMMENT"='Hive release version 4.0.0-beta-2' where "VER_ID"=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2'; From 5e6ce2a66404ef0267c27f407f14e601e566dfc0 Mon Sep 17 00:00:00 2001 From: Diksha628 <43694846+Diksha628@users.noreply.github.com> Date: Fri, 3 Nov 2023 21:08:26 +0530 Subject: [PATCH 036/179] HIVE-27324: Hive query with NOT IN condition is giving incorrect results when the sub query table contains the null value (Diksha, reviewed by Mahesh Kumar Behera, Sankar Hariappan) Signed-off-by: Sankar Hariappan Closes (#4636) --- .../resources/testconfiguration.properties | 1 + .../hive/ql/parse/SemanticAnalyzer.java | 42 +- .../test/queries/clientpositive/notInTest.q | 93 + .../llap/create_view_disable_cbo.q.out | 4 +- .../clientpositive/llap/notInTest.q.out | 1825 +++++++++++++++++ .../special_character_in_tabnames_1.q.out | 185 +- ...ecial_character_in_tabnames_quotes_1.q.out | 185 +- .../llap/subquery_unqual_corr_expr.q.out | 36 +- 8 files changed, 2225 insertions(+), 146 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/notInTest.q create mode 100644 ql/src/test/results/clientpositive/llap/notInTest.q.out diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index e56f6ba8bdb8..367b922d1300 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -126,6 +126,7 @@ minillap.query.files=\ multi_count_distinct_null.q,\ newline.q,\ nonreserved_keywords_insert_into1.q,\ + notInTest.q,\ nullscript.q,\ orc_createas1.q,\ orc_llap_counters.q,\ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 250f7c2fcbc9..ca0cc1798760 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -3704,7 +3704,14 @@ private Operator genFilterPlan(ASTNode searchCond, QB qb, Operator input, * push filters only for this QBJoinTree. Child QBJoinTrees have already been handled. */ pushJoinFilters(qb, joinTree, aliasToOpInfo, false); - input = genJoinOperator(qbSQ, joinTree, aliasToOpInfo, input); + + /* + * Note that: in case of multi dest queries, with even one containing a notIn operator, the code is not changed yet. + * That needs to be worked on as a separate bug : https://issues.apache.org/jira/browse/HIVE-27844 + */ + boolean notInCheckPresent = (subQuery.getNotInCheck() != null && !qb.isMultiDestQuery()); + input = genJoinOperator(qbSQ, joinTree, aliasToOpInfo , input, notInCheckPresent); + searchCond = subQuery.updateOuterQueryFilter(clonedSearchCond); } } @@ -3771,14 +3778,26 @@ private Operator genFilterPlan(QB qb, ASTNode condn, Operator input, boolean use * for inner joins push a 'is not null predicate' to the join sources for * every non nullSafe predicate. */ + private Operator genNotNullFilterForJoinSourcePlan(QB qb, Operator input, - QBJoinTree joinTree, ExprNodeDesc[] joinKeys) throws SemanticException { + QBJoinTree joinTree, ExprNodeDesc[] joinKeys) throws SemanticException { + return genNotNullFilterForJoinSourcePlan(qb, input, joinTree, joinKeys, false); + } + + private Operator genNotNullFilterForJoinSourcePlan(QB qb, Operator input, + QBJoinTree joinTree, ExprNodeDesc[] joinKeys, boolean OuternotInCheck) throws SemanticException { + + /* + * The notInCheck param is used for the purpose of adding an + * (outerQueryTable.outerQueryCol is not null ) predicate to the join, + * since it is not added naturally because of outer join + */ if (qb == null || joinTree == null) { return input; } - if (!joinTree.getNoOuterJoin()) { + if (!joinTree.getNoOuterJoin() && !OuternotInCheck) { return input; } @@ -3843,6 +3862,8 @@ private Operator genNotNullFilterForJoinSourcePlan(QB qb, Operator input, return output; } + + Integer genExprNodeDescRegex(String colRegex, String tabAlias, ASTNode sel, List exprList, Set excludeCols, RowResolver input, RowResolver colSrcRR, Integer pos, RowResolver output, List aliases, @@ -9855,8 +9876,15 @@ private void addJoinKeyToRowSchema( private Operator genJoinOperator(QB qb, QBJoinTree joinTree, Map map, Operator joiningOp) throws SemanticException { + return genJoinOperator(qb, joinTree, map, joiningOp, false); + } + + private Operator genJoinOperator(QB qb, QBJoinTree joinTree, + Map map, + Operator joiningOp, boolean notInCheckPresent) throws SemanticException { QBJoinTree leftChild = joinTree.getJoinSrc(); Operator joinSrcOp = joiningOp instanceof JoinOperator ? joiningOp : null; + Operator OuterSrcOp = joiningOp; if (joinSrcOp == null && leftChild != null) { joinSrcOp = genJoinOperator(qb, leftChild, map, null); @@ -9910,7 +9938,13 @@ private Operator genJoinOperator(QB qb, QBJoinTree joinTree, // generate a ReduceSink operator for the join String[] srcs = baseSrc[i] != null ? new String[] {baseSrc[i]} : joinTree.getLeftAliases(); if (!isCBOExecuted()) { - srcOps[i] = genNotNullFilterForJoinSourcePlan(qb, srcOps[i], joinTree, joinKeys[i]); + /* + * The condition srcOps[i] == OuterSrcOp is used to make sure that the predicate for notnull check + * is added only for the outer query table.outerqueryCol + * even after the outer join condition + */ + boolean outerNotInCheck = (notInCheckPresent && (srcOps[i] == OuterSrcOp)); + srcOps[i] = genNotNullFilterForJoinSourcePlan(qb, srcOps[i], joinTree, joinKeys[i], outerNotInCheck); } srcOps[i] = genJoinReduceSinkChild(joinKeys[i], srcOps[i], srcs, joinTree.getNextTag()); } diff --git a/ql/src/test/queries/clientpositive/notInTest.q b/ql/src/test/queries/clientpositive/notInTest.q new file mode 100644 index 000000000000..19da7c213afb --- /dev/null +++ b/ql/src/test/queries/clientpositive/notInTest.q @@ -0,0 +1,93 @@ +create table t3 (id int,name string, age int); +insert into t3 values(1,'Sagar',23),(2,'Sultan',NULL),(3,'Surya',23),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23),(9,'ron',3),(10,'Sam',22),(11,'nick',19),(12,'fed',18),(13,'kong',13),(14,'hela',45); + +create table t4 (id int,name string, age int); +insert into t4 values(1,'Sagar',23),(3,'Surya',23),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23); + +create table t5 (id int,name string, ages int); +insert into t5 values(1,'Sagar',23),(3,'Surya',NULL),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23); + +set hive.cbo.enable = false; + +select * from t3 +where age in (select distinct(age) age from t4) +order by age ; + +select * from t3 +where age not in (select distinct(age) age from t4 ) +order by age ; + + +select * from t3 +where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +order by age ; + + +select * from t3 +where age not in (select distinct(ages) ages from t5 ) +order by age ; + +select count(*) from t3 +where age not in (23,22, null ); + +explain select * from t3 + where age not in (select distinct(age) age from t4); + +explain select * from t3 +where age not in (select distinct(ages) ages from t5 ); + +explain select * from t3 + where age not in (select distinct(ages) ages from t5 where t5.ages is not null); + +select count(*) from t3 +where age not in (select distinct(age)age from t3 t1 where t1.age > 10); + + + +explain select id, name, age +from t3 b where b.age not in +(select min(age) + from (select id, age from t3) a + where age < 10 and b.age = a.age) + order by name; + +set hive.cbo.enable = true; + +select * from t3 +where age in (select distinct(age) age from t4) +order by age ; + +select * from t3 +where age not in (select distinct(age) age from t4 ) +order by age ; + +select * from t3 +where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +order by age ; + + +select * from t3 +where age not in (select distinct(ages) ages from t5 ) +order by age ; + +select count(*) from t3 +where age not in (23,22, null ); + +explain select * from t3 + where age not in (select distinct(age) age from t4); + +explain select * from t3 +where age not in (select distinct(ages) ages from t5 ); + +explain select * from t3 + where age not in (select distinct(ages) ages from t5 where t5.ages is not null); + +select count(*) from t3 +where age not in (select distinct(age)age from t3 t1 where t1.age > 10); + + explain select id, name, age + from t3 b where b.age not in + (select min(age) + from (select id, age from t3) a + where age < 10 and b.age = a.age) + order by name; diff --git a/ql/src/test/results/clientpositive/llap/create_view_disable_cbo.q.out b/ql/src/test/results/clientpositive/llap/create_view_disable_cbo.q.out index 31387cb4a978..d7a10c1ab412 100644 --- a/ql/src/test/results/clientpositive/llap/create_view_disable_cbo.q.out +++ b/ql/src/test/results/clientpositive/llap/create_view_disable_cbo.q.out @@ -53,7 +53,7 @@ POSTHOOK: Input: cdh_82023_repro_db@data POSTHOOK: Output: cdh_82023_repro_db@background POSTHOOK: Output: database:cdh_82023_repro_db POSTHOOK: Lineage: background.text EXPRESSION [(data)xouter.FieldSchema(name:text, type:string, comment:null), ] -Warning: Shuffle Join MERGEJOIN[42][tables = [xouter, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[45][tables = [xouter, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product PREHOOK: query: SELECT * FROM `cdh_82023_repro_db`.`background` PREHOOK: type: QUERY PREHOOK: Input: cdh_82023_repro_db@background @@ -85,7 +85,7 @@ POSTHOOK: Input: cdh_82023_repro_db@data POSTHOOK: Output: cdh_82023_repro_db@foreground POSTHOOK: Output: database:cdh_82023_repro_db POSTHOOK: Lineage: foreground.text EXPRESSION [(data)xouter.FieldSchema(name:text, type:string, comment:null), ] -Warning: Shuffle Join MERGEJOIN[43][tables = [xouter, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[46][tables = [xouter, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product PREHOOK: query: SELECT * FROM `cdh_82023_repro_db`.`foreground` PREHOOK: type: QUERY PREHOOK: Input: cdh_82023_repro_db@background diff --git a/ql/src/test/results/clientpositive/llap/notInTest.q.out b/ql/src/test/results/clientpositive/llap/notInTest.q.out new file mode 100644 index 000000000000..75a26c4b71f3 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/notInTest.q.out @@ -0,0 +1,1825 @@ +PREHOOK: query: create table t3 (id int,name string, age int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t3 +POSTHOOK: query: create table t3 (id int,name string, age int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t3 +PREHOOK: query: insert into t3 values(1,'Sagar',23),(2,'Sultan',NULL),(3,'Surya',23),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23),(9,'ron',3),(10,'Sam',22),(11,'nick',19),(12,'fed',18),(13,'kong',13),(14,'hela',45) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@t3 +POSTHOOK: query: insert into t3 values(1,'Sagar',23),(2,'Sultan',NULL),(3,'Surya',23),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23),(9,'ron',3),(10,'Sam',22),(11,'nick',19),(12,'fed',18),(13,'kong',13),(14,'hela',45) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@t3 +POSTHOOK: Lineage: t3.age SCRIPT [] +POSTHOOK: Lineage: t3.id SCRIPT [] +POSTHOOK: Lineage: t3.name SCRIPT [] +PREHOOK: query: create table t4 (id int,name string, age int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t4 +POSTHOOK: query: create table t4 (id int,name string, age int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t4 +PREHOOK: query: insert into t4 values(1,'Sagar',23),(3,'Surya',23),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@t4 +POSTHOOK: query: insert into t4 values(1,'Sagar',23),(3,'Surya',23),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@t4 +POSTHOOK: Lineage: t4.age SCRIPT [] +POSTHOOK: Lineage: t4.id SCRIPT [] +POSTHOOK: Lineage: t4.name SCRIPT [] +PREHOOK: query: create table t5 (id int,name string, ages int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t5 +POSTHOOK: query: create table t5 (id int,name string, ages int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t5 +PREHOOK: query: insert into t5 values(1,'Sagar',23),(3,'Surya',NULL),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@t5 +POSTHOOK: query: insert into t5 values(1,'Sagar',23),(3,'Surya',NULL),(4,'Raman',45),(5,'Scott',23),(6,'Ramya',5),(7,'',23),(8,'',23) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@t5 +POSTHOOK: Lineage: t5.ages SCRIPT [] +POSTHOOK: Lineage: t5.id SCRIPT [] +POSTHOOK: Lineage: t5.name SCRIPT [] +PREHOOK: query: select * from t3 +where age in (select distinct(age) age from t4) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age in (select distinct(age) age from t4) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: hdfs://### HDFS PATH ### +6 Ramya 5 +1 Sagar 23 +3 Surya 23 +5 Scott 23 +7 23 +8 23 +4 Raman 45 +14 hela 45 +Warning: Shuffle Join MERGEJOIN[46][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: select * from t3 +where age not in (select distinct(age) age from t4 ) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age not in (select distinct(age) age from t4 ) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: hdfs://### HDFS PATH ### +9 ron 3 +13 kong 13 +12 fed 18 +11 nick 19 +10 Sam 22 +Warning: Shuffle Join MERGEJOIN[48][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +9 ron 3 +13 kong 13 +12 fed 18 +11 nick 19 +10 Sam 22 +Warning: Shuffle Join MERGEJOIN[46][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 ) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 ) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: select count(*) from t3 +where age not in (23,22, null ) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from t3 +where age not in (23,22, null ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 +Warning: Shuffle Join MERGEJOIN[44][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain select * from t3 + where age not in (select distinct(age) age from t4) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select * from t3 + where age not in (select distinct(age) age from t4) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t3 + filterExpr: age is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: id (type: int), name (type: string), age (type: int) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t4 + filterExpr: (age is not null or age is null) (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is not null (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: age (type: int) + minReductionHashAggr: 0.57142854 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is null (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: null (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: null (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: null (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col7 + Statistics: Num rows: 17 Data size: 1652 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col7 is null (type: boolean) + Statistics: Num rows: 13 Data size: 1264 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1264 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1264 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: null (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = 0L) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: 0L (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join MERGEJOIN[44][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain select * from t3 +where age not in (select distinct(ages) ages from t5 ) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select * from t3 +where age not in (select distinct(ages) ages from t5 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t3 + filterExpr: age is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: id (type: int), name (type: string), age (type: int) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t5 + filterExpr: (ages is not null or ages is null) (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ages is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ages (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ages is null (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: null (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: null (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: null (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col7 + Statistics: Num rows: 17 Data size: 1648 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col7 is null (type: boolean) + Statistics: Num rows: 14 Data size: 1356 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1356 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 14 Data size: 1356 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: null (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = 0L) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: 0L (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join MERGEJOIN[46][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain select * from t3 + where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select * from t3 + where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t3 + filterExpr: age is not null (type: boolean) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: id (type: int), name (type: string), age (type: int) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t5 + filterExpr: (ages is not null or (ages is not null and ages is null)) (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ages is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ages (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (ages is not null and ages is null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: null (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: null (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: null (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col7 + Statistics: Num rows: 17 Data size: 1648 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col7 is null (type: boolean) + Statistics: Num rows: 14 Data size: 1356 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1356 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 14 Data size: 1356 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: null (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = 0L) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: 0L (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join MERGEJOIN[50][tables = [t3, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: select count(*) from t3 +where age not in (select distinct(age)age from t3 t1 where t1.age > 10) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from t3 +where age not in (select distinct(age)age from t3 t1 where t1.age > 10) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 +Warning: Shuffle Join MERGEJOIN[52][tables = [b, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain select id, name, age +from t3 b where b.age not in +(select min(age) + from (select id, age from t3) a + where age < 10 and b.age = a.age) + order by name +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select id, name, age +from t3 b where b.age not in +(select min(age) + from (select id, age from t3) a + where age < 10 and b.age = a.age) + order by name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Map 1 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + filterExpr: (age is not null or (age < 10)) (type: boolean) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is not null (type: boolean) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: id (type: int), name (type: string), age (type: int) + Filter Operator + predicate: (age < 10) (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: age (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col1) + keys: _col1 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int), _col2 (type: int) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col2 (type: int), _col2 (type: int) + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int), _col2 (type: int) + 1 _col0 (type: int), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col7 + Statistics: Num rows: 16 Data size: 1552 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col7 is null (type: boolean) + Statistics: Num rows: 13 Data size: 1260 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: z + sort order: + + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: int) + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 13 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col1 is not null (type: boolean) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col1 is null or _col0 is null) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = 0L) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: 0L (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from t3 +where age in (select distinct(age) age from t4) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age in (select distinct(age) age from t4) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: hdfs://### HDFS PATH ### +6 Ramya 5 +1 Sagar 23 +3 Surya 23 +5 Scott 23 +7 23 +8 23 +4 Raman 45 +14 hela 45 +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: select * from t3 +where age not in (select distinct(age) age from t4 ) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age not in (select distinct(age) age from t4 ) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: hdfs://### HDFS PATH ### +9 ron 3 +13 kong 13 +12 fed 18 +11 nick 19 +10 Sam 22 +Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +9 ron 3 +13 kong 13 +12 fed 18 +11 nick 19 +10 Sam 22 +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 ) +order by age +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from t3 +where age not in (select distinct(ages) ages from t5 ) +order by age +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: select count(*) from t3 +where age not in (23,22, null ) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Input: default@t3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from t3 +where age not in (23,22, null ) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Input: default@t3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain select * from t3 + where age not in (select distinct(age) age from t4) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select * from t3 + where age not in (select distinct(age) age from t4) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t3 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: id (type: int), name (type: string), age (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t4 + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: age is not null (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: age (type: int) + minReductionHashAggr: 0.57142854 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: age (type: int) + outputColumnNames: age + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: age (type: int) + minReductionHashAggr: 0.57142854 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 19 Data size: 1848 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 19 Data size: 1848 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col4 (type: boolean) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6 + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col5 (type: bigint), _col6 (type: bigint), _col4 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6 + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col3 = 0L) or (_col6 is null and (_col4 >= _col3) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 19 Data size: 1824 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 1824 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.6666666 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain select * from t3 +where age not in (select distinct(ages) ages from t5 ) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select * from t3 +where age not in (select distinct(ages) ages from t5 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 7 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t3 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: id (type: int), name (type: string), age (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t5 + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ages is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ages (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ages (type: int) + outputColumnNames: ages + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ages (type: int) + minReductionHashAggr: 0.57142854 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 19 Data size: 1848 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 19 Data size: 1848 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col4 (type: boolean) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6 + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col5 (type: bigint), _col6 (type: bigint), _col4 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6 + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col3 = 0L) or (_col6 is null and (_col4 >= _col3) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 19 Data size: 1824 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 1824 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.6666666 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: explain select * from t3 + where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t5 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select * from t3 + where age not in (select distinct(ages) ages from t5 where t5.ages is not null) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t5 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 6 (XPROD_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t3 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: id (type: int), name (type: string), age (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t5 + filterExpr: ages is not null (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ages is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: ages (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 19 Data size: 1848 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 19 Data size: 1848 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col4 (type: boolean) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6 + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col5 (type: bigint), _col6 (type: bigint), _col4 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6 + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((_col3 = 0L) or (_col6 is null and (_col4 >= _col3) and _col2 is not null)) (type: boolean) + Statistics: Num rows: 19 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 19 Data size: 1824 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 1824 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: boolean) + Group By Operator + aggregations: count(), count(_col0) + minReductionHashAggr: 0.6666666 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +PREHOOK: query: select count(*) from t3 +where age not in (select distinct(age)age from t3 t1 where t1.age > 10) +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from t3 +where age not in (select distinct(age)age from t3 t1 where t1.age > 10) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 +PREHOOK: query: explain select id, name, age + from t3 b where b.age not in + (select min(age) + from (select id, age from t3) a + where age < 10 and b.age = a.age) + order by name +PREHOOK: type: QUERY +PREHOOK: Input: default@t3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select id, name, age + from t3 b where b.age not in + (select min(age) + from (select id, age from t3) a + where age < 10 and b.age = a.age) + order by name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Map 1 (SIMPLE_EDGE) + Reducer 7 <- Map 1 (SIMPLE_EDGE) + Reducer 8 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: id (type: int), name (type: string), age (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 14 Data size: 1344 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Filter Operator + predicate: (age < 10) (type: boolean) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: age (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(age) + keys: age (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Group By Operator + aggregations: min(age) + keys: age (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4 + Statistics: Num rows: 17 Data size: 1664 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: if(_col4 is null, sq_count_check(0L, true), sq_count_check(_col4, true)) (type: boolean) + Statistics: Num rows: 8 Data size: 784 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 8 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 + Statistics: Num rows: 8 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) + Statistics: Num rows: 8 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col4 (type: bigint), _col5 (type: bigint) + Reducer 4 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col2 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6 + Statistics: Num rows: 8 Data size: 916 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col4 is null or (_col4 = 0L) or (_col6 is not null or _col2 is null or (_col5 < _col4)) is not true) (type: boolean) + Statistics: Num rows: 8 Data size: 916 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: z + sort order: + + Statistics: Num rows: 8 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col2 (type: int) + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 768 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(), count(_col1) + keys: _col0 (type: int) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: bigint), _col2 (type: bigint) + Reducer 8 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col0 = _col1) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: true (type: boolean), _col0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: boolean) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + diff --git a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out index c386a6dd0a06..335cc4a4b995 100644 --- a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out +++ b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_1.q.out @@ -2386,12 +2386,16 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src/_/cbo + filterExpr: (key is not null or (key > '2') or ((key > '2') and key is null)) (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: key (type: string), value (type: string) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: key (type: string), value (type: string) Filter Operator predicate: (key > '2') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE @@ -2560,14 +2564,18 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b + filterExpr: ((p_name is not null and p_mfgr is not null) or ((p_size < 10) and p_name is not null and p_mfgr is not null) or ((p_size < 10) and (p_name is null or p_mfgr is null))) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: (p_name is not null and p_mfgr is not null) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) Filter Operator - predicate: (p_size < 10) (type: boolean) + predicate: ((p_size < 10) and p_name is not null and p_mfgr is not null) (type: boolean) Statistics: Num rows: 5 Data size: 1115 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: p_name (type: string), p_mfgr (type: string) @@ -2734,12 +2742,16 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p/a/r/t + filterExpr: (UDFToDouble(p_size) is not null or (p_size < 10)) (type: boolean) Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: UDFToDouble(p_size) is not null (type: boolean) Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: p_name (type: string), p_size (type: int) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_size (type: int) Filter Operator predicate: (p_size < 10) (type: boolean) Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE @@ -2824,12 +2836,15 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: double) + Filter Operator + predicate: _col0 is not null (type: boolean) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: _col0 is null (type: boolean) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE @@ -2899,23 +2914,49 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 5 <- Map 1 (SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) + Reducer 6 <- Map 1 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan alias: b + filterExpr: ((p_size is not null and p_mfgr is not null) or ((p_size < 10) and p_mfgr is not null) or (p_size < 10)) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: (p_size is not null and p_mfgr is not null) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + Filter Operator + predicate: ((p_size < 10) and p_mfgr is not null) (type: boolean) + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_mfgr (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Filter Operator predicate: (p_size < 10) (type: boolean) Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE @@ -3004,16 +3045,28 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0, _col1 Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: int), _col0 (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: _col1 is not null (type: boolean) Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: string) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (_col1 is null or _col0 is null) (type: boolean) Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE @@ -3030,7 +3083,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 6 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3104,10 +3157,10 @@ STAGE PLANS: Map Operator Tree: TableScan alias: line/item - filterExpr: ((l_shipmode = 'AIR') or ((l_shipmode = 'AIR') and l_orderkey is null) or (l_linenumber = 1)) (type: boolean) + filterExpr: (((l_shipmode = 'AIR') and l_orderkey is not null) or ((l_shipmode = 'AIR') and l_orderkey is null) or (l_orderkey is not null and (l_linenumber = 1))) (type: boolean) Statistics: Num rows: 100 Data size: 9200 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (l_shipmode = 'AIR') (type: boolean) + predicate: ((l_shipmode = 'AIR') and l_orderkey is not null) (type: boolean) Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: l_orderkey (type: int) @@ -3136,7 +3189,7 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Filter Operator - predicate: (l_linenumber = 1) (type: boolean) + predicate: (l_orderkey is not null and (l_linenumber = 1)) (type: boolean) Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: @@ -3349,20 +3402,23 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: struct) - Group By Operator - aggregations: min(p_retailprice), max(p_retailprice), avg(p_retailprice) - keys: p_mfgr (type: string) - minReductionHashAggr: 0.8076923 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Filter Operator + predicate: p_mfgr is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(p_retailprice), max(p_retailprice), avg(p_retailprice) + keys: p_mfgr (type: string) + minReductionHashAggr: 0.8076923 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: struct) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: struct) Execution mode: vectorized, llap LLAP IO: all inputs Reducer 2 @@ -3500,7 +3556,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col2 - _col1) > 600) (type: boolean) + predicate: (((_col2 - _col1) > 600) and _col1 is not null) (type: boolean) Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: double) @@ -3604,20 +3660,23 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: double) - Group By Operator - aggregations: max(p_retailprice), min(p_retailprice) - keys: p_mfgr (type: string) - minReductionHashAggr: 0.8076923 - mode: hash - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Filter Operator + predicate: p_mfgr is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(p_retailprice), min(p_retailprice) + keys: p_mfgr (type: string) + minReductionHashAggr: 0.8076923 + mode: hash + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: double), _col2 (type: double) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double), _col2 (type: double) Filter Operator predicate: p_mfgr is null (type: boolean) Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE diff --git a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out index 56e922b69cde..ca3e3cc8ee89 100644 --- a/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out +++ b/ql/src/test/results/clientpositive/llap/special_character_in_tabnames_quotes_1.q.out @@ -2558,12 +2558,16 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src/_/cbo + filterExpr: (key is not null or (key > '2') or ((key > '2') and key is null)) (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: key (type: string), value (type: string) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: key (type: string), value (type: string) Filter Operator predicate: (key > '2') (type: boolean) Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE @@ -2732,14 +2736,18 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b + filterExpr: ((p_name is not null and p_mfgr is not null) or ((p_size < 10) and p_name is not null and p_mfgr is not null) or ((p_size < 10) and (p_name is null or p_mfgr is null))) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: (p_name is not null and p_mfgr is not null) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) Filter Operator - predicate: (p_size < 10) (type: boolean) + predicate: ((p_size < 10) and p_name is not null and p_mfgr is not null) (type: boolean) Statistics: Num rows: 5 Data size: 1115 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: p_name (type: string), p_mfgr (type: string) @@ -2906,12 +2914,16 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p/a/r/t + filterExpr: (UDFToDouble(p_size) is not null or (p_size < 10)) (type: boolean) Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: UDFToDouble(p_size) is not null (type: boolean) Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: p_name (type: string), p_size (type: int) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_size (type: int) Filter Operator predicate: (p_size < 10) (type: boolean) Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE @@ -2996,12 +3008,15 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: double) + Filter Operator + predicate: _col0 is not null (type: boolean) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: _col0 is null (type: boolean) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE @@ -3071,23 +3086,49 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) Reducer 4 <- Reducer 3 (SIMPLE_EDGE) Reducer 5 <- Map 1 (SIMPLE_EDGE) - Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE) + Reducer 6 <- Map 1 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan alias: b + filterExpr: ((p_size is not null and p_mfgr is not null) or ((p_size < 10) and p_mfgr is not null) or (p_size < 10)) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: (p_size is not null and p_mfgr is not null) (type: boolean) Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int) + Filter Operator + predicate: ((p_size < 10) and p_mfgr is not null) (type: boolean) + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: p_mfgr (type: string), p_size (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col1) + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Filter Operator predicate: (p_size < 10) (type: boolean) Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE @@ -3176,16 +3217,28 @@ STAGE PLANS: mode: mergepartial outputColumnNames: _col0, _col1 Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col1 (type: int), _col0 (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: _col1 is not null (type: boolean) Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: string) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Select Operator + expressions: _col1 (type: int), _col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 510 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (_col1 is null or _col0 is null) (type: boolean) Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE @@ -3202,7 +3255,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 6 + Reducer 7 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -3276,10 +3329,10 @@ STAGE PLANS: Map Operator Tree: TableScan alias: line/item - filterExpr: ((l_shipmode = 'AIR') or ((l_shipmode = 'AIR') and l_orderkey is null) or (l_linenumber = 1)) (type: boolean) + filterExpr: (((l_shipmode = 'AIR') and l_orderkey is not null) or ((l_shipmode = 'AIR') and l_orderkey is null) or (l_orderkey is not null and (l_linenumber = 1))) (type: boolean) Statistics: Num rows: 100 Data size: 9200 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (l_shipmode = 'AIR') (type: boolean) + predicate: ((l_shipmode = 'AIR') and l_orderkey is not null) (type: boolean) Statistics: Num rows: 14 Data size: 1288 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: l_orderkey (type: int) @@ -3308,7 +3361,7 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Filter Operator - predicate: (l_linenumber = 1) (type: boolean) + predicate: (l_orderkey is not null and (l_linenumber = 1)) (type: boolean) Statistics: Num rows: 14 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: @@ -3521,20 +3574,23 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: struct) - Group By Operator - aggregations: min(p_retailprice), max(p_retailprice), avg(p_retailprice) - keys: p_mfgr (type: string) - minReductionHashAggr: 0.8076923 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Filter Operator + predicate: p_mfgr is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(p_retailprice), max(p_retailprice), avg(p_retailprice) + keys: p_mfgr (type: string) + minReductionHashAggr: 0.8076923 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: struct) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: struct) Execution mode: vectorized, llap LLAP IO: all inputs Reducer 2 @@ -3672,7 +3728,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col2 - _col1) > 600) (type: boolean) + predicate: (((_col2 - _col1) > 600) and _col1 is not null) (type: boolean) Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: double) @@ -3776,20 +3832,23 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 5 Data size: 530 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: double) - Group By Operator - aggregations: max(p_retailprice), min(p_retailprice) - keys: p_mfgr (type: string) - minReductionHashAggr: 0.8076923 - mode: hash - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Filter Operator + predicate: p_mfgr is not null (type: boolean) + Statistics: Num rows: 26 Data size: 2756 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(p_retailprice), min(p_retailprice) + keys: p_mfgr (type: string) + minReductionHashAggr: 0.8076923 + mode: hash + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: double), _col2 (type: double) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: double), _col2 (type: double) Filter Operator predicate: p_mfgr is null (type: boolean) Statistics: Num rows: 1 Data size: 106 Basic stats: COMPLETE Column stats: COMPLETE diff --git a/ql/src/test/results/clientpositive/llap/subquery_unqual_corr_expr.q.out b/ql/src/test/results/clientpositive/llap/subquery_unqual_corr_expr.q.out index 5e193274860a..91b88fbc48ff 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_unqual_corr_expr.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_unqual_corr_expr.q.out @@ -109,7 +109,7 @@ POSTHOOK: Input: default@src 484 val_484 86 val_86 98 val_98 -Warning: Shuffle Join MERGEJOIN[28][tables = [src, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[31][tables = [src, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product PREHOOK: query: explain select * from src tablesample (10 rows) where concat(key,value) not in (select key from src) PREHOOK: type: QUERY @@ -138,30 +138,38 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src + filterExpr: concat(key, value) is not null (type: boolean) Row Limit Per Split: 10 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: + Filter Operator + predicate: concat(key, value) is not null (type: boolean) Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: key (type: string), value (type: string) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: key (type: string), value (type: string) Execution mode: vectorized, llap LLAP IO: all inputs Map 4 Map Operator Tree: TableScan alias: src + filterExpr: (key is not null or key is null) (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: key (type: string) - outputColumnNames: _col0 + Filter Operator + predicate: key is not null (type: boolean) Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: key is null (type: boolean) Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE @@ -253,7 +261,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[28][tables = [src, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[31][tables = [src, sq_1_notin_nullcheck]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select * from src tablesample (10 rows) where concat(key,value) not in (select key from src) PREHOOK: type: QUERY PREHOOK: Input: default@src From a84fca7616489116d9d72b930fe6aa47a83c9156 Mon Sep 17 00:00:00 2001 From: dengzh Date: Sat, 4 Nov 2023 10:19:31 +0800 Subject: [PATCH 037/179] HIVE-27114: Provide a configurable filter for removing useless properties in Partition objects from listPartitions calls (Zhihua Deng, reviewed by Sai Hemanth Gantasala) Closes #4726 --- .../listener/DummyRawStoreFailEvent.java | 35 +- .../example/AlterPartitionParamsExample.java | 113 +++++++ .../apache/hadoop/hive/ql/metadata/Hive.java | 2 + .../TestListPartitionsWithXIncludeParams.java | 234 ++++++++++++++ .../partition_params_xinclude.q | 23 ++ .../llap/partition_params_xinclude.q.out | 281 ++++++++++++++++ .../thrift/gen-cpp/hive_metastore_types.cpp | 220 +++++++++++++ .../gen/thrift/gen-cpp/hive_metastore_types.h | 110 ++++++- .../api/GetPartitionsByFilterRequest.java | 220 ++++++++++++- .../api/GetPartitionsByNamesRequest.java | 220 ++++++++++++- .../api/GetPartitionsPsWithAuthRequest.java | 220 ++++++++++++- .../api/PartitionsByExprRequest.java | 220 ++++++++++++- .../hive/metastore/api/PartitionsRequest.java | 220 ++++++++++++- .../GetPartitionsByFilterRequest.php | 48 +++ .../metastore/GetPartitionsByNamesRequest.php | 48 +++ .../GetPartitionsPsWithAuthRequest.php | 48 +++ .../metastore/PartitionsByExprRequest.php | 48 +++ .../gen-php/metastore/PartitionsRequest.php | 48 +++ .../thrift/gen-py/hive_metastore/ttypes.py | 130 +++++++- .../gen/thrift/gen-rb/hive_metastore_types.rb | 30 +- .../hive/metastore/HiveMetaStoreClient.java | 67 ++-- .../hive/metastore/conf/MetastoreConf.java | 14 +- .../hive/metastore/utils/JavaUtils.java | 13 + .../hive/metastore/utils/MetaStoreUtils.java | 17 + .../src/main/thrift/hive_metastore.thrift | 20 +- .../hadoop/hive/metastore/HMSHandler.java | 199 ++++++------ .../hive/metastore/MetaStoreDirectSql.java | 48 ++- .../metastore/MetastoreDirectSqlUtils.java | 18 -- .../hadoop/hive/metastore/ObjectStore.java | 302 ++++++++---------- .../hadoop/hive/metastore/RawStore.java | 93 +++--- .../hive/metastore/cache/CachedStore.java | 83 ++--- .../client/builder/GetPartitionsArgs.java | 185 +++++++++++ .../DummyRawStoreControlledCommit.java | 37 +-- .../DummyRawStoreForJdoConnection.java | 22 +- ...rtitionsUsingProjectionAndFilterSpecs.java | 8 +- .../hive/metastore/TestObjectStore.java | 13 +- .../hive/metastore/VerifyingObjectStore.java | 26 +- .../metastore/client/TestListPartitions.java | 55 +++- 38 files changed, 3141 insertions(+), 597 deletions(-) create mode 100644 itests/qtest/src/test/java/org/apache/hadoop/hive/udf/example/AlterPartitionParamsExample.java create mode 100644 ql/src/test/org/apache/hadoop/hive/metastore/TestListPartitionsWithXIncludeParams.java create mode 100644 ql/src/test/queries/clientpositive/partition_params_xinclude.q create mode 100644 ql/src/test/results/clientpositive/llap/partition_params_xinclude.q.out create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/client/builder/GetPartitionsArgs.java diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java index 5c7db6c948c9..31569d6793f1 100644 --- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java @@ -117,6 +117,7 @@ import org.apache.hadoop.hive.metastore.api.WriteEventInfo; import org.apache.hadoop.hive.metastore.api.ReplicationMetricList; import org.apache.hadoop.hive.metastore.api.GetReplicationMetricsRequest; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.model.MTable; import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo; @@ -1615,46 +1616,34 @@ public Map> updatePartitionColumnStatisticsInBatch( } @Override - public List getPartitions(String catName, String dbName, String tableName, int max, boolean skipColSchemaForPartitions) + public List getPartitions(String catName, String dbName, String tableName, GetPartitionsArgs args) throws MetaException, NoSuchObjectException { - return objectStore.getPartitions(catName, dbName, tableName, max, skipColSchemaForPartitions); + return objectStore.getPartitions(catName, dbName, tableName, args); } @Override public List getPartitionsByNames(String catName, String dbName, String tblName, - List partNames, boolean skipColSchemaForPartitions) + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { - return objectStore.getPartitionsByNames( - catName, dbName, tblName, partNames, skipColSchemaForPartitions); + return objectStore.getPartitionsByNames(catName, dbName, tblName, args); } @Override - public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, - String defaultPartitionName, short maxParts, List result, boolean skipColSchemaForPartitions) throws TException { - return objectStore.getPartitionsByExpr(catName, - dbName, tblName, expr, defaultPartitionName, maxParts, result, skipColSchemaForPartitions); - } - - @Override - public List getPartitionsWithAuth(String catName, String dbName, String tblName, - short maxParts, String userName, List groupNames, boolean skipColSchemaForPartitions) - throws MetaException, NoSuchObjectException, InvalidObjectException { - return objectStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, - groupNames, skipColSchemaForPartitions); + public boolean getPartitionsByExpr(String catName, String dbName, String tblName, + List result, GetPartitionsArgs args) throws TException { + return objectStore.getPartitionsByExpr(catName, dbName, tblName, result, args); } @Override public List listPartitionsPsWithAuth(String catName, String dbName, String tblName, - List partVals, short maxParts, String userName, List groupNames, - boolean skipColSchemaForPartitions) throws MetaException, InvalidObjectException, NoSuchObjectException { - return objectStore.listPartitionsPsWithAuth(catName, dbName, tblName, partVals, maxParts, - userName, groupNames, skipColSchemaForPartitions); + GetPartitionsArgs args) throws MetaException, InvalidObjectException, NoSuchObjectException { + return objectStore.listPartitionsPsWithAuth(catName, dbName, tblName, args); } @Override public List getPartitionsByFilter(String catName, String dbName, String tblName, - String filter, short maxParts, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { - return objectStore.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts, skipColSchemaForPartitions); + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { + return objectStore.getPartitionsByFilter(catName, dbName, tblName, args); } } diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/udf/example/AlterPartitionParamsExample.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/udf/example/AlterPartitionParamsExample.java new file mode 100644 index 000000000000..f69dca908588 --- /dev/null +++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/udf/example/AlterPartitionParamsExample.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.udf.example; + +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.BooleanWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is for testing only, will try to alter the partition's parameters with the input key value pairs. + */ +public class AlterPartitionParamsExample extends GenericUDF { + + private static final Logger LOG = LoggerFactory.getLogger(AlterPartitionParamsExample.class); + + private transient ObjectInspectorConverters.Converter[] converters; + private transient BooleanWritable ret = new BooleanWritable(false); + private transient Table table; + + // table, partition name, param_key, param_value + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 4) { + throw new UDFArgumentLengthException( + "Requires 4 argument, got " + arguments.length); + } + if (!(arguments[0] instanceof ConstantObjectInspector)) { + throw new UDFArgumentException( + "The first argument should be a string constant, got " + arguments[0].getTypeName()); + } + converters = new ObjectInspectorConverters.Converter[arguments.length]; + for (int i = 1; i < arguments.length; i++) { + converters[i] = ObjectInspectorConverters.getConverter(arguments[i], + PrimitiveObjectInspectorFactory.writableStringObjectInspector); + } + String tableName = ((ConstantObjectInspector) arguments[0]).getWritableConstantValue().toString(); + try { + table = Hive.get().getTable(tableName); + if (!table.isPartitioned()) { + throw new UDFArgumentException("The input table: " + table + " isn't a partitioned table!"); + } + } catch (Exception e) { + if (e instanceof UDFArgumentException) { + throw (UDFArgumentException) e; + } + throw new UDFArgumentException(e); + } + return PrimitiveObjectInspectorFactory.writableBooleanObjectInspector; + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + ret.set(false); + if (arguments[1].get() == null || arguments[2].get() == null || + arguments[3].get() == null) { + return ret; + } + String partName = converters[1].convert(arguments[1].get()).toString(); + String paramKey = converters[2].convert(arguments[2].get()).toString(); + String paramValue = converters[3].convert(arguments[3].get()).toString(); + try { + List partitionList = Hive.get() + .getPartitionsByNames(table, Arrays.asList(partName)); + if (partitionList == null || partitionList.isEmpty()) { + return ret; + } + Partition partition = partitionList.get(0); + partition.getParameters().put(paramKey, paramValue); + Hive.get() + .alterPartition(table.getCatName(), table.getDbName(), table.getTableName(), partition, null, true); + ret.set(true); + } catch (Exception e) { + LOG.debug("Error while altering the partition's parameters", e); + } + return ret; + } + + @Override + public String getDisplayString(String[] children) { + return getStandardDisplayString("alter_partition_params", children); + } + +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index d6189b1fa65f..51133a3636c5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -4132,6 +4132,8 @@ public List getPartitions(Table tbl) throws HiveException { GetPartitionsPsWithAuthResponse res = getMSC().listPartitionsWithAuthInfoRequest(req); tParts = res.getPartitions(); + } catch (NoSuchObjectException nsoe) { + return Lists.newArrayList(); } catch (Exception e) { LOG.error("Failed getPartitions", e); throw new HiveException(e); diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/TestListPartitionsWithXIncludeParams.java b/ql/src/test/org/apache/hadoop/hive/metastore/TestListPartitionsWithXIncludeParams.java new file mode 100644 index 000000000000..f0acbc67ee8d --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/metastore/TestListPartitionsWithXIncludeParams.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.metastore; + +import com.google.common.collect.Lists; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest; +import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest; +import org.apache.hadoop.hive.metastore.api.GetPartitionsFilterSpec; +import org.apache.hadoop.hive.metastore.api.GetPartitionsRequest; +import org.apache.hadoop.hive.metastore.api.GetProjectionsSpec; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PartitionFilterMode; +import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD; +import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.client.TestListPartitions; +import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService; +import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.thrift.TException; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static junit.framework.TestCase.assertTrue; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.convertToGetPartitionsByNamesRequest; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * Test class for list partitions with configurable include/exclude pattern on parameters. + * Embedded Metastore uses JDO, Remote Metastore uses direct SQL to query the partitions: + * MetaStoreFactoryForTests.getMetaStores() + */ +@RunWith(Parameterized.class) +@Category(MetastoreCheckinTest.class) +public class TestListPartitionsWithXIncludeParams + extends TestListPartitions { + + private Configuration hiveConf; + private Set includeKeys = new HashSet<>(); + private Set excludeKeys = new HashSet<>(); + private Map partParams = new HashMap<>(); + + public static class PartitionExpressionForMetastoreTest extends PartitionExpressionForMetastore { + // MetaStoreTestUtils.setConfForStandloneMode will change the default PartitionExpressionForMetastore + // to DefaultPartitionExpressionProxy, which doesn't support deserializing the Hive filter from a byte array. + // As this test sits inside the hive-exec module, it's safe to set the hive.metastore.expression.proxy to + // PartitionExpressionForMetastoreTest. + } + + public TestListPartitionsWithXIncludeParams(String name, + AbstractMetaStoreService metaStore) { + super(name, metaStore); + partParams.put("key1", "value1"); + partParams.put("akey1", "avalue1"); + partParams.put("akey10", "avalue10"); + partParams.put("excludekey1", "value1"); + partParams.put("excludekey2", "value1"); + includeKeys.add("key1"); + includeKeys.add("akey1"); + excludeKeys.add("excludekey1"); + excludeKeys.add("excludekey2"); + hiveConf = metaStore.getConf(); + MetastoreConf.setVar(hiveConf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS, + PartitionExpressionForMetastoreTest.class.getName()); + MetastoreConf.setVar(hiveConf, + MetastoreConf.ConfVars.METASTORE_PARTITIONS_PARAMETERS_INCLUDE_PATTERN, "%k_y_"); + MetastoreConf.setVar(hiveConf, + MetastoreConf.ConfVars.METASTORE_PARTITIONS_PARAMETERS_EXCLUDE_PATTERN, "%exclu%"); + } + + @Override + protected void addPartition(IMetaStoreClient client, Table table, + List values) throws TException { + PartitionBuilder partitionBuilder = new PartitionBuilder().inTable(table); + values.forEach(val -> partitionBuilder.addValue(val)); + partParams.forEach((k, v) -> partitionBuilder.addPartParam(k, v)); + client.add_partition(partitionBuilder.build(getMetaStore().getConf())); + } + + @Override + protected void addPartitions(IMetaStoreClient client, List partitions) + throws TException { + partitions.stream().forEach(partition -> partition.setParameters(partParams)); + super.addPartitions(client, partitions); + } + + @Override + protected void assertPartitionsHaveCorrectParams(List partitions) { + for (int i = 0; i < partitions.size(); i++) { + Map parameters = partitions.get(i).getParameters(); + assertTrue("included parameter key is not found in the partition", + parameters.keySet().containsAll(includeKeys)); + assertFalse("excluded parameter key is found in the partition", + parameters.keySet().stream().anyMatch(key -> excludeKeys.contains(key))); + assertEquals(includeKeys.size(), parameters.size()); + } + } + + @Test + public void testGetPartitionsByNames() throws Exception { + Table t = createTable4PartColsParts(getClient()).table; + List part_names = Arrays.asList("yyyy=1999/mm=01/dd=02", + "yyyy=2009/mm=02/dd=10", "yyyy=1999/mm=03/dd=02"); + GetPartitionsByNamesRequest request = convertToGetPartitionsByNamesRequest( + MetaStoreUtils.prependCatalogToDbName(t.getCatName(), t.getDbName(), hiveConf), t.getTableName(), + part_names); + List partitions = getClient().getPartitionsByNames(request).getPartitions(); + List> values = partitions.stream().map(partition -> partition.getValues()).collect(Collectors.toList()); + assertCorrectPartitionNames(part_names.subList(0, 2), values, Lists.newArrayList("yyyy", "mm", "dd")); + assertPartitionsHaveCorrectParams(partitions); + + // empty + part_names = Arrays.asList("yyyy=1999/mm=03/dd=02", "yyyy=2017/mm=02/dd=13"); + request = convertToGetPartitionsByNamesRequest( + MetaStoreUtils.prependCatalogToDbName(t.getCatName(), t.getDbName(), hiveConf), t.getTableName(), + part_names); + partitions = getClient().getPartitionsByNames(request).getPartitions(); + assertTrue(partitions.isEmpty()); + } + + @Test + public void testGetPartitionsRequest() throws Exception { + ReturnTable returnTable = createTable4PartColsParts(getClient()); + Table t = returnTable.table; + GetPartitionsRequest request = new GetPartitionsRequest(t.getDbName(), t.getTableName(), + new GetProjectionsSpec(), new GetPartitionsFilterSpec()); + request.setCatName(t.getCatName()); + + List partitions = MetaStoreServerUtils.getPartitionsByProjectSpec(getClient(), request); + assertPartitionsHaveCorrectParams(partitions); + List> values = partitions.stream().map(partition -> partition.getValues()).collect(Collectors.toList()); + assertEquals(returnTable.testValues, values); + + request.getProjectionSpec() + .setFieldList(Arrays.asList("dbName", "tableName", "catName", "parameters", "values")); + partitions = MetaStoreServerUtils.getPartitionsByProjectSpec(getClient(), request); + assertPartitionsHaveCorrectParams(partitions); + values = partitions.stream().map(partition -> partition.getValues()).collect(Collectors.toList()); + assertEquals(returnTable.testValues, values); + + request.getFilterSpec().setFilterMode(PartitionFilterMode.BY_VALUES); + request.getFilterSpec().setFilters(Arrays.asList("2017")); + partitions = MetaStoreServerUtils.getPartitionsByProjectSpec(getClient(), request); + assertPartitionsHaveCorrectParams(partitions); + values = partitions.stream().map(partition -> partition.getValues()).collect(Collectors.toList()); + assertEquals("Two partitions expected", 2, values.size()); + assertEquals(Arrays.asList(Arrays.asList("2017", "10", "26"), + Arrays.asList("2017", "11", "27")), returnTable.testValues.subList(2, 4)); + } + + @Test + public void testListPartitionsByExr() throws Exception { + createTable4PartColsParts(getClient()); + TestMetastoreExpr.ExprBuilder e = new TestMetastoreExpr.ExprBuilder(TABLE_NAME); + checkExpr(2, e.strCol("yyyy").val("2017").pred("=", 2).build()); + checkExpr(3, e.strCol("mm").val("11").pred(">", 2).build()); + checkExpr(4, e.strCol("dd").val("29").pred(">=", 2).build()); + checkExpr(2, e.strCol("yyyy").val("2017").pred("!=", 2).build()); + checkExpr(1, e.strCol("yyyy").val("2017").pred("=", 2).strCol("mm").val("10") + .pred(">=", 2).pred("and", 2).build()); + checkExpr(3, e.strCol("dd").val("10").pred("<", 2).strCol("yyyy").val("2009") + .pred("!=", 2).pred("or", 2).build()); + checkExpr(0, e.strCol("yyyy").val("2019").pred("=", 2).build()); + } + + private void checkExpr(int numParts, ExprNodeGenericFuncDesc expr) throws Exception { + List partitions = new ArrayList<>(); + byte[] exprBytes = SerializationUtilities.serializeObjectWithTypeInformation(expr); + getClient().listPartitionsByExpr(DB_NAME, TABLE_NAME, exprBytes, + null, (short) -1, partitions); + assertEquals("Partition check failed: " + expr.getExprString(), numParts, partitions.size()); + assertPartitionsHaveCorrectParams(partitions); + + PartitionsByExprRequest req = new PartitionsByExprRequest(DB_NAME, TABLE_NAME, + ByteBuffer.wrap(exprBytes)); + List msParts = + new ArrayList<>(); + getClient().listPartitionsSpecByExpr(req, msParts); + + int numPartitions = 0; + for (org.apache.hadoop.hive.metastore.api.PartitionSpec partitionSpec : msParts) { + assertTrue(partitionSpec.getPartitionList() == null || + partitionSpec.getPartitionList().getPartitions() == null || + partitionSpec.getPartitionList().getPartitions().isEmpty()); + for (PartitionWithoutSD partitionWithoutSD: partitionSpec.getSharedSDPartitionSpec().getPartitions()) { + numPartitions ++; + Map parameters = partitionWithoutSD.getParameters(); + assertTrue("included parameter key is not found in the partition", + parameters.keySet().containsAll(includeKeys)); + assertFalse("excluded parameter key is found in the partition", + parameters.keySet().stream().anyMatch(key -> excludeKeys.contains(key))); + assertEquals(includeKeys.size(), parameters.size()); + } + } + assertEquals("Partition check failed: " + expr.getExprString(), numParts, numPartitions); + } + +} diff --git a/ql/src/test/queries/clientpositive/partition_params_xinclude.q b/ql/src/test/queries/clientpositive/partition_params_xinclude.q new file mode 100644 index 000000000000..10d1230172c4 --- /dev/null +++ b/ql/src/test/queries/clientpositive/partition_params_xinclude.q @@ -0,0 +1,23 @@ +create table part_params_xin (customer int) partitioned by (dt string); +insert into part_params_xin partition(dt='2001-01-01') values(1); +insert into part_params_xin partition(dt='2001-01-03') values(3); + +set hive.optimize.metadata.query.cache.enabled=false; + +create table params(key string, value string); +insert into table params values('key1', 'value1'), ('akey1', 'avalue1'), ('akey10', 'avalue10'), ('excludekey1', 'value1'),('excludekey2', 'value1'); + +add jar ${system:maven.local.repository}/org/apache/hive/hive-it-qfile/${system:hive.version}/hive-it-qfile-${system:hive.version}.jar; + +create temporary function alter_partition_params as 'org.apache.hadoop.hive.udf.example.AlterPartitionParamsExample'; + +select alter_partition_params('part_params_xin', 'dt=2001-01-01', key, value) from params; + +explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false); + +set metastore.partitions.parameters.include.pattern=%k_y_; +set metastore.partitions.parameters.exclude.pattern=%exclu%; +explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false); + +set metastore.partitions.parameters.exclude.pattern=; +explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false); diff --git a/ql/src/test/results/clientpositive/llap/partition_params_xinclude.q.out b/ql/src/test/results/clientpositive/llap/partition_params_xinclude.q.out new file mode 100644 index 000000000000..32588d044322 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/partition_params_xinclude.q.out @@ -0,0 +1,281 @@ +PREHOOK: query: create table part_params_xin (customer int) partitioned by (dt string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part_params_xin +POSTHOOK: query: create table part_params_xin (customer int) partitioned by (dt string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part_params_xin +PREHOOK: query: insert into part_params_xin partition(dt='2001-01-01') values(1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@part_params_xin@dt=2001-01-01 +POSTHOOK: query: insert into part_params_xin partition(dt='2001-01-01') values(1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@part_params_xin@dt=2001-01-01 +POSTHOOK: Lineage: part_params_xin PARTITION(dt=2001-01-01).customer SCRIPT [] +PREHOOK: query: insert into part_params_xin partition(dt='2001-01-03') values(3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@part_params_xin@dt=2001-01-03 +POSTHOOK: query: insert into part_params_xin partition(dt='2001-01-03') values(3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@part_params_xin@dt=2001-01-03 +POSTHOOK: Lineage: part_params_xin PARTITION(dt=2001-01-03).customer SCRIPT [] +PREHOOK: query: create table params(key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@params +POSTHOOK: query: create table params(key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@params +PREHOOK: query: insert into table params values('key1', 'value1'), ('akey1', 'avalue1'), ('akey10', 'avalue10'), ('excludekey1', 'value1'),('excludekey2', 'value1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@params +POSTHOOK: query: insert into table params values('key1', 'value1'), ('akey1', 'avalue1'), ('akey10', 'avalue10'), ('excludekey1', 'value1'),('excludekey2', 'value1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@params +POSTHOOK: Lineage: params.key SCRIPT [] +POSTHOOK: Lineage: params.value SCRIPT [] +PREHOOK: query: create temporary function alter_partition_params as 'org.apache.hadoop.hive.udf.example.AlterPartitionParamsExample' +PREHOOK: type: CREATEFUNCTION +PREHOOK: Output: alter_partition_params +POSTHOOK: query: create temporary function alter_partition_params as 'org.apache.hadoop.hive.udf.example.AlterPartitionParamsExample' +POSTHOOK: type: CREATEFUNCTION +POSTHOOK: Output: alter_partition_params +PREHOOK: query: select alter_partition_params('part_params_xin', 'dt=2001-01-01', key, value) from params +PREHOOK: type: QUERY +PREHOOK: Input: default@params +#### A masked pattern was here #### +POSTHOOK: query: select alter_partition_params('part_params_xin', 'dt=2001-01-01', key, value) from params +POSTHOOK: type: QUERY +POSTHOOK: Input: default@params +#### A masked pattern was here #### +true +true +true +true +true +PREHOOK: query: explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_params_xin +PREHOOK: Input: default@part_params_xin@dt=2001-01-01 +#### A masked pattern was here #### +POSTHOOK: query: explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_params_xin +POSTHOOK: Input: default@part_params_xin@dt=2001-01-01 +#### A masked pattern was here #### +OPTIMIZED SQL: SELECT `customer`, `dt` +FROM `default`.`part_params_xin` +WHERE NVL(`dt` = '2001-01-01' AND `customer` = 1, FALSE) +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + dt 2001-01-01 + properties: + akey1 avalue1 + akey10 avalue10 + column.name.delimiter , + columns customer + columns.types int + excludekey1 value1 + excludekey2 value1 +#### A masked pattern was here #### + key1 value1 +#### A masked pattern was here #### + name default.part_params_xin + partition_columns dt + partition_columns.types string + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucketing_version 2 + column.name.delimiter , + columns customer + columns.comments + columns.types int +#### A masked pattern was here #### + name default.part_params_xin + partition_columns dt + partition_columns.types string + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_params_xin + name: default.part_params_xin + Processor Tree: + TableScan + alias: part_params_xin + filterExpr: COALESCE((customer = 1),false) (type: boolean) + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: COALESCE((customer = 1),false) (type: boolean) + Select Operator + expressions: customer (type: int), dt (type: string) + outputColumnNames: _col0, _col1 + ListSink + +PREHOOK: query: explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_params_xin +PREHOOK: Input: default@part_params_xin@dt=2001-01-01 +#### A masked pattern was here #### +POSTHOOK: query: explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_params_xin +POSTHOOK: Input: default@part_params_xin@dt=2001-01-01 +#### A masked pattern was here #### +OPTIMIZED SQL: SELECT `customer`, `dt` +FROM `default`.`part_params_xin` +WHERE NVL(`dt` = '2001-01-01' AND `customer` = 1, FALSE) +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + dt 2001-01-01 + properties: + akey1 avalue1 + column.name.delimiter , + columns customer + columns.types int +#### A masked pattern was here #### + key1 value1 +#### A masked pattern was here #### + name default.part_params_xin + partition_columns dt + partition_columns.types string + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucketing_version 2 + column.name.delimiter , + columns customer + columns.comments + columns.types int +#### A masked pattern was here #### + name default.part_params_xin + partition_columns dt + partition_columns.types string + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_params_xin + name: default.part_params_xin + Processor Tree: + TableScan + alias: part_params_xin + filterExpr: COALESCE((customer = 1),false) (type: boolean) + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: COALESCE((customer = 1),false) (type: boolean) + Select Operator + expressions: customer (type: int), dt (type: string) + outputColumnNames: _col0, _col1 + ListSink + +PREHOOK: query: explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false) +PREHOOK: type: QUERY +PREHOOK: Input: default@part_params_xin +PREHOOK: Input: default@part_params_xin@dt=2001-01-01 +#### A masked pattern was here #### +POSTHOOK: query: explain extended select * from part_params_xin where nvl(dt='2001-01-01' and customer=1, false) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_params_xin +POSTHOOK: Input: default@part_params_xin@dt=2001-01-01 +#### A masked pattern was here #### +OPTIMIZED SQL: SELECT `customer`, `dt` +FROM `default`.`part_params_xin` +WHERE NVL(`dt` = '2001-01-01' AND `customer` = 1, FALSE) +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Partition Description: + Partition + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + dt 2001-01-01 + properties: + akey1 avalue1 + column.name.delimiter , + columns customer + columns.types int + excludekey1 value1 + excludekey2 value1 +#### A masked pattern was here #### + key1 value1 +#### A masked pattern was here #### + name default.part_params_xin + partition_columns dt + partition_columns.types string + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucketing_version 2 + column.name.delimiter , + columns customer + columns.comments + columns.types int +#### A masked pattern was here #### + name default.part_params_xin + partition_columns dt + partition_columns.types string + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.part_params_xin + name: default.part_params_xin + Processor Tree: + TableScan + alias: part_params_xin + filterExpr: COALESCE((customer = 1),false) (type: boolean) + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: COALESCE((customer = 1),false) (type: boolean) + Select Operator + expressions: customer (type: int), dt (type: string) + outputColumnNames: _col0, _col1 + ListSink + diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index a0b3bc5ad88c..401d63df93fe 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -17606,6 +17606,16 @@ void PartitionsByExprRequest::__set_skipColumnSchemaForPartition(const bool val) this->skipColumnSchemaForPartition = val; __isset.skipColumnSchemaForPartition = true; } + +void PartitionsByExprRequest::__set_includeParamKeyPattern(const std::string& val) { + this->includeParamKeyPattern = val; +__isset.includeParamKeyPattern = true; +} + +void PartitionsByExprRequest::__set_excludeParamKeyPattern(const std::string& val) { + this->excludeParamKeyPattern = val; +__isset.excludeParamKeyPattern = true; +} std::ostream& operator<<(std::ostream& out, const PartitionsByExprRequest& obj) { obj.printTo(out); @@ -17717,6 +17727,22 @@ uint32_t PartitionsByExprRequest::read(::apache::thrift::protocol::TProtocol* ip xfer += iprot->skip(ftype); } break; + case 11: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->includeParamKeyPattern); + this->__isset.includeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->excludeParamKeyPattern); + this->__isset.excludeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -17787,6 +17813,16 @@ uint32_t PartitionsByExprRequest::write(::apache::thrift::protocol::TProtocol* o xfer += oprot->writeBool(this->skipColumnSchemaForPartition); xfer += oprot->writeFieldEnd(); } + if (this->__isset.includeParamKeyPattern) { + xfer += oprot->writeFieldBegin("includeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 11); + xfer += oprot->writeString(this->includeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.excludeParamKeyPattern) { + xfer += oprot->writeFieldBegin("excludeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 12); + xfer += oprot->writeString(this->excludeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -17804,6 +17840,8 @@ void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) { swap(a.validWriteIdList, b.validWriteIdList); swap(a.id, b.id); swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition); + swap(a.includeParamKeyPattern, b.includeParamKeyPattern); + swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern); swap(a.__isset, b.__isset); } @@ -17818,6 +17856,8 @@ PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& validWriteIdList = other651.validWriteIdList; id = other651.id; skipColumnSchemaForPartition = other651.skipColumnSchemaForPartition; + includeParamKeyPattern = other651.includeParamKeyPattern; + excludeParamKeyPattern = other651.excludeParamKeyPattern; __isset = other651.__isset; } PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other652) { @@ -17831,6 +17871,8 @@ PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByEx validWriteIdList = other652.validWriteIdList; id = other652.id; skipColumnSchemaForPartition = other652.skipColumnSchemaForPartition; + includeParamKeyPattern = other652.includeParamKeyPattern; + excludeParamKeyPattern = other652.excludeParamKeyPattern; __isset = other652.__isset; return *this; } @@ -17847,6 +17889,8 @@ void PartitionsByExprRequest::printTo(std::ostream& out) const { out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "")); out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "")); + out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "")); + out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "")); out << ")"; } @@ -20459,6 +20503,16 @@ void GetPartitionsByNamesRequest::__set_skipColumnSchemaForPartition(const bool this->skipColumnSchemaForPartition = val; __isset.skipColumnSchemaForPartition = true; } + +void GetPartitionsByNamesRequest::__set_includeParamKeyPattern(const std::string& val) { + this->includeParamKeyPattern = val; +__isset.includeParamKeyPattern = true; +} + +void GetPartitionsByNamesRequest::__set_excludeParamKeyPattern(const std::string& val) { + this->excludeParamKeyPattern = val; +__isset.excludeParamKeyPattern = true; +} std::ostream& operator<<(std::ostream& out, const GetPartitionsByNamesRequest& obj) { obj.printTo(out); @@ -20601,6 +20655,22 @@ uint32_t GetPartitionsByNamesRequest::read(::apache::thrift::protocol::TProtocol xfer += iprot->skip(ftype); } break; + case 12: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->includeParamKeyPattern); + this->__isset.includeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 13: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->excludeParamKeyPattern); + this->__isset.excludeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -20691,6 +20761,16 @@ uint32_t GetPartitionsByNamesRequest::write(::apache::thrift::protocol::TProtoco xfer += oprot->writeBool(this->skipColumnSchemaForPartition); xfer += oprot->writeFieldEnd(); } + if (this->__isset.includeParamKeyPattern) { + xfer += oprot->writeFieldBegin("includeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 12); + xfer += oprot->writeString(this->includeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.excludeParamKeyPattern) { + xfer += oprot->writeFieldBegin("excludeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 13); + xfer += oprot->writeString(this->excludeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -20709,6 +20789,8 @@ void swap(GetPartitionsByNamesRequest &a, GetPartitionsByNamesRequest &b) { swap(a.getFileMetadata, b.getFileMetadata); swap(a.id, b.id); swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition); + swap(a.includeParamKeyPattern, b.includeParamKeyPattern); + swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern); swap(a.__isset, b.__isset); } @@ -20724,6 +20806,8 @@ GetPartitionsByNamesRequest::GetPartitionsByNamesRequest(const GetPartitionsByNa getFileMetadata = other795.getFileMetadata; id = other795.id; skipColumnSchemaForPartition = other795.skipColumnSchemaForPartition; + includeParamKeyPattern = other795.includeParamKeyPattern; + excludeParamKeyPattern = other795.excludeParamKeyPattern; __isset = other795.__isset; } GetPartitionsByNamesRequest& GetPartitionsByNamesRequest::operator=(const GetPartitionsByNamesRequest& other796) { @@ -20738,6 +20822,8 @@ GetPartitionsByNamesRequest& GetPartitionsByNamesRequest::operator=(const GetPar getFileMetadata = other796.getFileMetadata; id = other796.id; skipColumnSchemaForPartition = other796.skipColumnSchemaForPartition; + includeParamKeyPattern = other796.includeParamKeyPattern; + excludeParamKeyPattern = other796.excludeParamKeyPattern; __isset = other796.__isset; return *this; } @@ -20755,6 +20841,8 @@ void GetPartitionsByNamesRequest::printTo(std::ostream& out) const { out << ", " << "getFileMetadata="; (__isset.getFileMetadata ? (out << to_string(getFileMetadata)) : (out << "")); out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "")); + out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "")); + out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "")); out << ")"; } @@ -49068,6 +49156,16 @@ void PartitionsRequest::__set_skipColumnSchemaForPartition(const bool val) { this->skipColumnSchemaForPartition = val; __isset.skipColumnSchemaForPartition = true; } + +void PartitionsRequest::__set_includeParamKeyPattern(const std::string& val) { + this->includeParamKeyPattern = val; +__isset.includeParamKeyPattern = true; +} + +void PartitionsRequest::__set_excludeParamKeyPattern(const std::string& val) { + this->excludeParamKeyPattern = val; +__isset.excludeParamKeyPattern = true; +} std::ostream& operator<<(std::ostream& out, const PartitionsRequest& obj) { obj.printTo(out); @@ -49154,6 +49252,22 @@ uint32_t PartitionsRequest::read(::apache::thrift::protocol::TProtocol* iprot) { xfer += iprot->skip(ftype); } break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->includeParamKeyPattern); + this->__isset.includeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->excludeParamKeyPattern); + this->__isset.excludeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -49208,6 +49322,16 @@ uint32_t PartitionsRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeBool(this->skipColumnSchemaForPartition); xfer += oprot->writeFieldEnd(); } + if (this->__isset.includeParamKeyPattern) { + xfer += oprot->writeFieldBegin("includeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 8); + xfer += oprot->writeString(this->includeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.excludeParamKeyPattern) { + xfer += oprot->writeFieldBegin("excludeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 9); + xfer += oprot->writeString(this->excludeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -49222,6 +49346,8 @@ void swap(PartitionsRequest &a, PartitionsRequest &b) { swap(a.validWriteIdList, b.validWriteIdList); swap(a.id, b.id); swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition); + swap(a.includeParamKeyPattern, b.includeParamKeyPattern); + swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern); swap(a.__isset, b.__isset); } @@ -49233,6 +49359,8 @@ PartitionsRequest::PartitionsRequest(const PartitionsRequest& other1733) { validWriteIdList = other1733.validWriteIdList; id = other1733.id; skipColumnSchemaForPartition = other1733.skipColumnSchemaForPartition; + includeParamKeyPattern = other1733.includeParamKeyPattern; + excludeParamKeyPattern = other1733.excludeParamKeyPattern; __isset = other1733.__isset; } PartitionsRequest& PartitionsRequest::operator=(const PartitionsRequest& other1734) { @@ -49243,6 +49371,8 @@ PartitionsRequest& PartitionsRequest::operator=(const PartitionsRequest& other17 validWriteIdList = other1734.validWriteIdList; id = other1734.id; skipColumnSchemaForPartition = other1734.skipColumnSchemaForPartition; + includeParamKeyPattern = other1734.includeParamKeyPattern; + excludeParamKeyPattern = other1734.excludeParamKeyPattern; __isset = other1734.__isset; return *this; } @@ -49256,6 +49386,8 @@ void PartitionsRequest::printTo(std::ostream& out) const { out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "")); out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "")); + out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "")); + out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "")); out << ")"; } @@ -49402,6 +49534,16 @@ void GetPartitionsByFilterRequest::__set_skipColumnSchemaForPartition(const bool this->skipColumnSchemaForPartition = val; __isset.skipColumnSchemaForPartition = true; } + +void GetPartitionsByFilterRequest::__set_includeParamKeyPattern(const std::string& val) { + this->includeParamKeyPattern = val; +__isset.includeParamKeyPattern = true; +} + +void GetPartitionsByFilterRequest::__set_excludeParamKeyPattern(const std::string& val) { + this->excludeParamKeyPattern = val; +__isset.excludeParamKeyPattern = true; +} std::ostream& operator<<(std::ostream& out, const GetPartitionsByFilterRequest& obj) { obj.printTo(out); @@ -49478,6 +49620,22 @@ uint32_t GetPartitionsByFilterRequest::read(::apache::thrift::protocol::TProtoco xfer += iprot->skip(ftype); } break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->includeParamKeyPattern); + this->__isset.includeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->excludeParamKeyPattern); + this->__isset.excludeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -49522,6 +49680,16 @@ uint32_t GetPartitionsByFilterRequest::write(::apache::thrift::protocol::TProtoc xfer += oprot->writeBool(this->skipColumnSchemaForPartition); xfer += oprot->writeFieldEnd(); } + if (this->__isset.includeParamKeyPattern) { + xfer += oprot->writeFieldBegin("includeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 7); + xfer += oprot->writeString(this->includeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.excludeParamKeyPattern) { + xfer += oprot->writeFieldBegin("excludeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 8); + xfer += oprot->writeString(this->excludeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -49535,6 +49703,8 @@ void swap(GetPartitionsByFilterRequest &a, GetPartitionsByFilterRequest &b) { swap(a.filter, b.filter); swap(a.maxParts, b.maxParts); swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition); + swap(a.includeParamKeyPattern, b.includeParamKeyPattern); + swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern); swap(a.__isset, b.__isset); } @@ -49545,6 +49715,8 @@ GetPartitionsByFilterRequest::GetPartitionsByFilterRequest(const GetPartitionsBy filter = other1743.filter; maxParts = other1743.maxParts; skipColumnSchemaForPartition = other1743.skipColumnSchemaForPartition; + includeParamKeyPattern = other1743.includeParamKeyPattern; + excludeParamKeyPattern = other1743.excludeParamKeyPattern; __isset = other1743.__isset; } GetPartitionsByFilterRequest& GetPartitionsByFilterRequest::operator=(const GetPartitionsByFilterRequest& other1744) { @@ -49554,6 +49726,8 @@ GetPartitionsByFilterRequest& GetPartitionsByFilterRequest::operator=(const GetP filter = other1744.filter; maxParts = other1744.maxParts; skipColumnSchemaForPartition = other1744.skipColumnSchemaForPartition; + includeParamKeyPattern = other1744.includeParamKeyPattern; + excludeParamKeyPattern = other1744.excludeParamKeyPattern; __isset = other1744.__isset; return *this; } @@ -49566,6 +49740,8 @@ void GetPartitionsByFilterRequest::printTo(std::ostream& out) const { out << ", " << "filter=" << to_string(filter); out << ", " << "maxParts="; (__isset.maxParts ? (out << to_string(maxParts)) : (out << "")); out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "")); + out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "")); + out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "")); out << ")"; } @@ -49981,6 +50157,16 @@ void GetPartitionsPsWithAuthRequest::__set_skipColumnSchemaForPartition(const bo this->skipColumnSchemaForPartition = val; __isset.skipColumnSchemaForPartition = true; } + +void GetPartitionsPsWithAuthRequest::__set_includeParamKeyPattern(const std::string& val) { + this->includeParamKeyPattern = val; +__isset.includeParamKeyPattern = true; +} + +void GetPartitionsPsWithAuthRequest::__set_excludeParamKeyPattern(const std::string& val) { + this->excludeParamKeyPattern = val; +__isset.excludeParamKeyPattern = true; +} std::ostream& operator<<(std::ostream& out, const GetPartitionsPsWithAuthRequest& obj) { obj.printTo(out); @@ -50115,6 +50301,22 @@ uint32_t GetPartitionsPsWithAuthRequest::read(::apache::thrift::protocol::TProto xfer += iprot->skip(ftype); } break; + case 11: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->includeParamKeyPattern); + this->__isset.includeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->excludeParamKeyPattern); + this->__isset.excludeParamKeyPattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -50200,6 +50402,16 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeBool(this->skipColumnSchemaForPartition); xfer += oprot->writeFieldEnd(); } + if (this->__isset.includeParamKeyPattern) { + xfer += oprot->writeFieldBegin("includeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 11); + xfer += oprot->writeString(this->includeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.excludeParamKeyPattern) { + xfer += oprot->writeFieldBegin("excludeParamKeyPattern", ::apache::thrift::protocol::T_STRING, 12); + xfer += oprot->writeString(this->excludeParamKeyPattern); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -50217,6 +50429,8 @@ void swap(GetPartitionsPsWithAuthRequest &a, GetPartitionsPsWithAuthRequest &b) swap(a.validWriteIdList, b.validWriteIdList); swap(a.id, b.id); swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition); + swap(a.includeParamKeyPattern, b.includeParamKeyPattern); + swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern); swap(a.__isset, b.__isset); } @@ -50231,6 +50445,8 @@ GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitio validWriteIdList = other1773.validWriteIdList; id = other1773.id; skipColumnSchemaForPartition = other1773.skipColumnSchemaForPartition; + includeParamKeyPattern = other1773.includeParamKeyPattern; + excludeParamKeyPattern = other1773.excludeParamKeyPattern; __isset = other1773.__isset; } GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1774) { @@ -50244,6 +50460,8 @@ GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const validWriteIdList = other1774.validWriteIdList; id = other1774.id; skipColumnSchemaForPartition = other1774.skipColumnSchemaForPartition; + includeParamKeyPattern = other1774.includeParamKeyPattern; + excludeParamKeyPattern = other1774.excludeParamKeyPattern; __isset = other1774.__isset; return *this; } @@ -50260,6 +50478,8 @@ void GetPartitionsPsWithAuthRequest::printTo(std::ostream& out) const { out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "")); out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "")); + out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "")); + out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "")); out << ")"; } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h index 09238fa5e617..47d79d6ae08d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -7238,7 +7238,7 @@ void swap(PartitionsSpecByExprResult &a, PartitionsSpecByExprResult &b); std::ostream& operator<<(std::ostream& out, const PartitionsSpecByExprResult& obj); typedef struct _PartitionsByExprRequest__isset { - _PartitionsByExprRequest__isset() : defaultPartitionName(false), maxParts(true), catName(false), order(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false) {} + _PartitionsByExprRequest__isset() : defaultPartitionName(false), maxParts(true), catName(false), order(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} bool defaultPartitionName :1; bool maxParts :1; bool catName :1; @@ -7246,6 +7246,8 @@ typedef struct _PartitionsByExprRequest__isset { bool validWriteIdList :1; bool id :1; bool skipColumnSchemaForPartition :1; + bool includeParamKeyPattern :1; + bool excludeParamKeyPattern :1; } _PartitionsByExprRequest__isset; class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { @@ -7263,7 +7265,9 @@ class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { order(), validWriteIdList(), id(-1LL), - skipColumnSchemaForPartition(0) { + skipColumnSchemaForPartition(0), + includeParamKeyPattern(), + excludeParamKeyPattern() { } virtual ~PartitionsByExprRequest() noexcept; @@ -7277,6 +7281,8 @@ class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { std::string validWriteIdList; int64_t id; bool skipColumnSchemaForPartition; + std::string includeParamKeyPattern; + std::string excludeParamKeyPattern; _PartitionsByExprRequest__isset __isset; @@ -7300,6 +7306,10 @@ class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { void __set_skipColumnSchemaForPartition(const bool val); + void __set_includeParamKeyPattern(const std::string& val); + + void __set_excludeParamKeyPattern(const std::string& val); + bool operator == (const PartitionsByExprRequest & rhs) const { if (!(dbName == rhs.dbName)) @@ -7336,6 +7346,14 @@ class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.skipColumnSchemaForPartition && !(skipColumnSchemaForPartition == rhs.skipColumnSchemaForPartition)) return false; + if (__isset.includeParamKeyPattern != rhs.__isset.includeParamKeyPattern) + return false; + else if (__isset.includeParamKeyPattern && !(includeParamKeyPattern == rhs.includeParamKeyPattern)) + return false; + if (__isset.excludeParamKeyPattern != rhs.__isset.excludeParamKeyPattern) + return false; + else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern)) + return false; return true; } bool operator != (const PartitionsByExprRequest &rhs) const { @@ -8236,7 +8254,7 @@ void swap(PartitionValuesResponse &a, PartitionValuesResponse &b); std::ostream& operator<<(std::ostream& out, const PartitionValuesResponse& obj); typedef struct _GetPartitionsByNamesRequest__isset { - _GetPartitionsByNamesRequest__isset() : names(false), get_col_stats(false), processorCapabilities(false), processorIdentifier(false), engine(false), validWriteIdList(false), getFileMetadata(false), id(true), skipColumnSchemaForPartition(false) {} + _GetPartitionsByNamesRequest__isset() : names(false), get_col_stats(false), processorCapabilities(false), processorIdentifier(false), engine(false), validWriteIdList(false), getFileMetadata(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} bool names :1; bool get_col_stats :1; bool processorCapabilities :1; @@ -8246,6 +8264,8 @@ typedef struct _GetPartitionsByNamesRequest__isset { bool getFileMetadata :1; bool id :1; bool skipColumnSchemaForPartition :1; + bool includeParamKeyPattern :1; + bool excludeParamKeyPattern :1; } _GetPartitionsByNamesRequest__isset; class GetPartitionsByNamesRequest : public virtual ::apache::thrift::TBase { @@ -8262,7 +8282,9 @@ class GetPartitionsByNamesRequest : public virtual ::apache::thrift::TBase { validWriteIdList(), getFileMetadata(0), id(-1LL), - skipColumnSchemaForPartition(0) { + skipColumnSchemaForPartition(0), + includeParamKeyPattern(), + excludeParamKeyPattern() { } virtual ~GetPartitionsByNamesRequest() noexcept; @@ -8277,6 +8299,8 @@ class GetPartitionsByNamesRequest : public virtual ::apache::thrift::TBase { bool getFileMetadata; int64_t id; bool skipColumnSchemaForPartition; + std::string includeParamKeyPattern; + std::string excludeParamKeyPattern; _GetPartitionsByNamesRequest__isset __isset; @@ -8302,6 +8326,10 @@ class GetPartitionsByNamesRequest : public virtual ::apache::thrift::TBase { void __set_skipColumnSchemaForPartition(const bool val); + void __set_includeParamKeyPattern(const std::string& val); + + void __set_excludeParamKeyPattern(const std::string& val); + bool operator == (const GetPartitionsByNamesRequest & rhs) const { if (!(db_name == rhs.db_name)) @@ -8344,6 +8372,14 @@ class GetPartitionsByNamesRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.skipColumnSchemaForPartition && !(skipColumnSchemaForPartition == rhs.skipColumnSchemaForPartition)) return false; + if (__isset.includeParamKeyPattern != rhs.__isset.includeParamKeyPattern) + return false; + else if (__isset.includeParamKeyPattern && !(includeParamKeyPattern == rhs.includeParamKeyPattern)) + return false; + if (__isset.excludeParamKeyPattern != rhs.__isset.excludeParamKeyPattern) + return false; + else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern)) + return false; return true; } bool operator != (const GetPartitionsByNamesRequest &rhs) const { @@ -19307,12 +19343,14 @@ void swap(GetPartitionResponse &a, GetPartitionResponse &b); std::ostream& operator<<(std::ostream& out, const GetPartitionResponse& obj); typedef struct _PartitionsRequest__isset { - _PartitionsRequest__isset() : catName(false), maxParts(true), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false) {} + _PartitionsRequest__isset() : catName(false), maxParts(true), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} bool catName :1; bool maxParts :1; bool validWriteIdList :1; bool id :1; bool skipColumnSchemaForPartition :1; + bool includeParamKeyPattern :1; + bool excludeParamKeyPattern :1; } _PartitionsRequest__isset; class PartitionsRequest : public virtual ::apache::thrift::TBase { @@ -19327,7 +19365,9 @@ class PartitionsRequest : public virtual ::apache::thrift::TBase { maxParts(-1), validWriteIdList(), id(-1LL), - skipColumnSchemaForPartition(0) { + skipColumnSchemaForPartition(0), + includeParamKeyPattern(), + excludeParamKeyPattern() { } virtual ~PartitionsRequest() noexcept; @@ -19338,6 +19378,8 @@ class PartitionsRequest : public virtual ::apache::thrift::TBase { std::string validWriteIdList; int64_t id; bool skipColumnSchemaForPartition; + std::string includeParamKeyPattern; + std::string excludeParamKeyPattern; _PartitionsRequest__isset __isset; @@ -19355,6 +19397,10 @@ class PartitionsRequest : public virtual ::apache::thrift::TBase { void __set_skipColumnSchemaForPartition(const bool val); + void __set_includeParamKeyPattern(const std::string& val); + + void __set_excludeParamKeyPattern(const std::string& val); + bool operator == (const PartitionsRequest & rhs) const { if (__isset.catName != rhs.__isset.catName) @@ -19381,6 +19427,14 @@ class PartitionsRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.skipColumnSchemaForPartition && !(skipColumnSchemaForPartition == rhs.skipColumnSchemaForPartition)) return false; + if (__isset.includeParamKeyPattern != rhs.__isset.includeParamKeyPattern) + return false; + else if (__isset.includeParamKeyPattern && !(includeParamKeyPattern == rhs.includeParamKeyPattern)) + return false; + if (__isset.excludeParamKeyPattern != rhs.__isset.excludeParamKeyPattern) + return false; + else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern)) + return false; return true; } bool operator != (const PartitionsRequest &rhs) const { @@ -19436,13 +19490,15 @@ void swap(PartitionsResponse &a, PartitionsResponse &b); std::ostream& operator<<(std::ostream& out, const PartitionsResponse& obj); typedef struct _GetPartitionsByFilterRequest__isset { - _GetPartitionsByFilterRequest__isset() : catName(false), dbName(false), tblName(false), filter(false), maxParts(true), skipColumnSchemaForPartition(false) {} + _GetPartitionsByFilterRequest__isset() : catName(false), dbName(false), tblName(false), filter(false), maxParts(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} bool catName :1; bool dbName :1; bool tblName :1; bool filter :1; bool maxParts :1; bool skipColumnSchemaForPartition :1; + bool includeParamKeyPattern :1; + bool excludeParamKeyPattern :1; } _GetPartitionsByFilterRequest__isset; class GetPartitionsByFilterRequest : public virtual ::apache::thrift::TBase { @@ -19456,7 +19512,9 @@ class GetPartitionsByFilterRequest : public virtual ::apache::thrift::TBase { tblName(), filter(), maxParts(-1), - skipColumnSchemaForPartition(0) { + skipColumnSchemaForPartition(0), + includeParamKeyPattern(), + excludeParamKeyPattern() { } virtual ~GetPartitionsByFilterRequest() noexcept; @@ -19466,6 +19524,8 @@ class GetPartitionsByFilterRequest : public virtual ::apache::thrift::TBase { std::string filter; int16_t maxParts; bool skipColumnSchemaForPartition; + std::string includeParamKeyPattern; + std::string excludeParamKeyPattern; _GetPartitionsByFilterRequest__isset __isset; @@ -19481,6 +19541,10 @@ class GetPartitionsByFilterRequest : public virtual ::apache::thrift::TBase { void __set_skipColumnSchemaForPartition(const bool val); + void __set_includeParamKeyPattern(const std::string& val); + + void __set_excludeParamKeyPattern(const std::string& val); + bool operator == (const GetPartitionsByFilterRequest & rhs) const { if (__isset.catName != rhs.__isset.catName) @@ -19501,6 +19565,14 @@ class GetPartitionsByFilterRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.skipColumnSchemaForPartition && !(skipColumnSchemaForPartition == rhs.skipColumnSchemaForPartition)) return false; + if (__isset.includeParamKeyPattern != rhs.__isset.includeParamKeyPattern) + return false; + else if (__isset.includeParamKeyPattern && !(includeParamKeyPattern == rhs.includeParamKeyPattern)) + return false; + if (__isset.excludeParamKeyPattern != rhs.__isset.excludeParamKeyPattern) + return false; + else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern)) + return false; return true; } bool operator != (const GetPartitionsByFilterRequest &rhs) const { @@ -19648,7 +19720,7 @@ void swap(GetPartitionNamesPsResponse &a, GetPartitionNamesPsResponse &b); std::ostream& operator<<(std::ostream& out, const GetPartitionNamesPsResponse& obj); typedef struct _GetPartitionsPsWithAuthRequest__isset { - _GetPartitionsPsWithAuthRequest__isset() : catName(false), partVals(false), maxParts(true), userName(false), groupNames(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false) {} + _GetPartitionsPsWithAuthRequest__isset() : catName(false), partVals(false), maxParts(true), userName(false), groupNames(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} bool catName :1; bool partVals :1; bool maxParts :1; @@ -19657,6 +19729,8 @@ typedef struct _GetPartitionsPsWithAuthRequest__isset { bool validWriteIdList :1; bool id :1; bool skipColumnSchemaForPartition :1; + bool includeParamKeyPattern :1; + bool excludeParamKeyPattern :1; } _GetPartitionsPsWithAuthRequest__isset; class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { @@ -19672,7 +19746,9 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { userName(), validWriteIdList(), id(-1LL), - skipColumnSchemaForPartition(0) { + skipColumnSchemaForPartition(0), + includeParamKeyPattern(), + excludeParamKeyPattern() { } virtual ~GetPartitionsPsWithAuthRequest() noexcept; @@ -19686,6 +19762,8 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { std::string validWriteIdList; int64_t id; bool skipColumnSchemaForPartition; + std::string includeParamKeyPattern; + std::string excludeParamKeyPattern; _GetPartitionsPsWithAuthRequest__isset __isset; @@ -19709,6 +19787,10 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { void __set_skipColumnSchemaForPartition(const bool val); + void __set_includeParamKeyPattern(const std::string& val); + + void __set_excludeParamKeyPattern(const std::string& val); + bool operator == (const GetPartitionsPsWithAuthRequest & rhs) const { if (__isset.catName != rhs.__isset.catName) @@ -19747,6 +19829,14 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.skipColumnSchemaForPartition && !(skipColumnSchemaForPartition == rhs.skipColumnSchemaForPartition)) return false; + if (__isset.includeParamKeyPattern != rhs.__isset.includeParamKeyPattern) + return false; + else if (__isset.includeParamKeyPattern && !(includeParamKeyPattern == rhs.includeParamKeyPattern)) + return false; + if (__isset.excludeParamKeyPattern != rhs.__isset.excludeParamKeyPattern) + return false; + else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern)) + return false; return true; } bool operator != (const GetPartitionsPsWithAuthRequest &rhs) const { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByFilterRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByFilterRequest.java index 88491fb7d0e7..2042e78886c4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByFilterRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByFilterRequest.java @@ -17,6 +17,8 @@ private static final org.apache.thrift.protocol.TField FILTER_FIELD_DESC = new org.apache.thrift.protocol.TField("filter", org.apache.thrift.protocol.TType.STRING, (short)4); private static final org.apache.thrift.protocol.TField MAX_PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxParts", org.apache.thrift.protocol.TType.I16, (short)5); private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)6); + private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)7); + private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)8); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new GetPartitionsByFilterRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new GetPartitionsByFilterRequestTupleSchemeFactory(); @@ -27,6 +29,8 @@ private @org.apache.thrift.annotation.Nullable java.lang.String filter; // required private short maxParts; // optional private boolean skipColumnSchemaForPartition; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -35,7 +39,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { TBL_NAME((short)3, "tblName"), FILTER((short)4, "filter"), MAX_PARTS((short)5, "maxParts"), - SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)6, "skipColumnSchemaForPartition"); + SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)6, "skipColumnSchemaForPartition"), + INCLUDE_PARAM_KEY_PATTERN((short)7, "includeParamKeyPattern"), + EXCLUDE_PARAM_KEY_PATTERN((short)8, "excludeParamKeyPattern"); private static final java.util.Map byName = new java.util.HashMap(); @@ -63,6 +69,10 @@ public static _Fields findByThriftId(int fieldId) { return MAX_PARTS; case 6: // SKIP_COLUMN_SCHEMA_FOR_PARTITION return SKIP_COLUMN_SCHEMA_FOR_PARTITION; + case 7: // INCLUDE_PARAM_KEY_PATTERN + return INCLUDE_PARAM_KEY_PATTERN; + case 8: // EXCLUDE_PARAM_KEY_PATTERN + return EXCLUDE_PARAM_KEY_PATTERN; default: return null; } @@ -107,7 +117,7 @@ public java.lang.String getFieldName() { private static final int __MAXPARTS_ISSET_ID = 0; private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 1; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.MAX_PARTS,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.MAX_PARTS,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -123,6 +133,10 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); tmpMap.put(_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION, new org.apache.thrift.meta_data.FieldMetaData("skipColumnSchemaForPartition", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.INCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("includeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsByFilterRequest.class, metaDataMap); } @@ -162,6 +176,12 @@ public GetPartitionsByFilterRequest(GetPartitionsByFilterRequest other) { } this.maxParts = other.maxParts; this.skipColumnSchemaForPartition = other.skipColumnSchemaForPartition; + if (other.isSetIncludeParamKeyPattern()) { + this.includeParamKeyPattern = other.includeParamKeyPattern; + } + if (other.isSetExcludeParamKeyPattern()) { + this.excludeParamKeyPattern = other.excludeParamKeyPattern; + } } public GetPartitionsByFilterRequest deepCopy() { @@ -178,6 +198,8 @@ public void clear() { setSkipColumnSchemaForPartitionIsSet(false); this.skipColumnSchemaForPartition = false; + this.includeParamKeyPattern = null; + this.excludeParamKeyPattern = null; } @org.apache.thrift.annotation.Nullable @@ -320,6 +342,54 @@ public void setSkipColumnSchemaForPartitionIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID, value); } + @org.apache.thrift.annotation.Nullable + public java.lang.String getIncludeParamKeyPattern() { + return this.includeParamKeyPattern; + } + + public void setIncludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern) { + this.includeParamKeyPattern = includeParamKeyPattern; + } + + public void unsetIncludeParamKeyPattern() { + this.includeParamKeyPattern = null; + } + + /** Returns true if field includeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetIncludeParamKeyPattern() { + return this.includeParamKeyPattern != null; + } + + public void setIncludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.includeParamKeyPattern = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getExcludeParamKeyPattern() { + return this.excludeParamKeyPattern; + } + + public void setExcludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern) { + this.excludeParamKeyPattern = excludeParamKeyPattern; + } + + public void unsetExcludeParamKeyPattern() { + this.excludeParamKeyPattern = null; + } + + /** Returns true if field excludeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetExcludeParamKeyPattern() { + return this.excludeParamKeyPattern != null; + } + + public void setExcludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.excludeParamKeyPattern = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case CAT_NAME: @@ -370,6 +440,22 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case INCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetIncludeParamKeyPattern(); + } else { + setIncludeParamKeyPattern((java.lang.String)value); + } + break; + + case EXCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetExcludeParamKeyPattern(); + } else { + setExcludeParamKeyPattern((java.lang.String)value); + } + break; + } } @@ -394,6 +480,12 @@ public java.lang.Object getFieldValue(_Fields field) { case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return getIncludeParamKeyPattern(); + + case EXCLUDE_PARAM_KEY_PATTERN: + return getExcludeParamKeyPattern(); + } throw new java.lang.IllegalStateException(); } @@ -417,6 +509,10 @@ public boolean isSet(_Fields field) { return isSetMaxParts(); case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSetSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return isSetIncludeParamKeyPattern(); + case EXCLUDE_PARAM_KEY_PATTERN: + return isSetExcludeParamKeyPattern(); } throw new java.lang.IllegalStateException(); } @@ -488,6 +584,24 @@ public boolean equals(GetPartitionsByFilterRequest that) { return false; } + boolean this_present_includeParamKeyPattern = true && this.isSetIncludeParamKeyPattern(); + boolean that_present_includeParamKeyPattern = true && that.isSetIncludeParamKeyPattern(); + if (this_present_includeParamKeyPattern || that_present_includeParamKeyPattern) { + if (!(this_present_includeParamKeyPattern && that_present_includeParamKeyPattern)) + return false; + if (!this.includeParamKeyPattern.equals(that.includeParamKeyPattern)) + return false; + } + + boolean this_present_excludeParamKeyPattern = true && this.isSetExcludeParamKeyPattern(); + boolean that_present_excludeParamKeyPattern = true && that.isSetExcludeParamKeyPattern(); + if (this_present_excludeParamKeyPattern || that_present_excludeParamKeyPattern) { + if (!(this_present_excludeParamKeyPattern && that_present_excludeParamKeyPattern)) + return false; + if (!this.excludeParamKeyPattern.equals(that.excludeParamKeyPattern)) + return false; + } + return true; } @@ -519,6 +633,14 @@ public int hashCode() { if (isSetSkipColumnSchemaForPartition()) hashCode = hashCode * 8191 + ((skipColumnSchemaForPartition) ? 131071 : 524287); + hashCode = hashCode * 8191 + ((isSetIncludeParamKeyPattern()) ? 131071 : 524287); + if (isSetIncludeParamKeyPattern()) + hashCode = hashCode * 8191 + includeParamKeyPattern.hashCode(); + + hashCode = hashCode * 8191 + ((isSetExcludeParamKeyPattern()) ? 131071 : 524287); + if (isSetExcludeParamKeyPattern()) + hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode(); + return hashCode; } @@ -590,6 +712,26 @@ public int compareTo(GetPartitionsByFilterRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetIncludeParamKeyPattern(), other.isSetIncludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIncludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeParamKeyPattern, other.includeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetExcludeParamKeyPattern(), other.isSetExcludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExcludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeParamKeyPattern, other.excludeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -656,6 +798,26 @@ public java.lang.String toString() { sb.append(this.skipColumnSchemaForPartition); first = false; } + if (isSetIncludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("includeParamKeyPattern:"); + if (this.includeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.includeParamKeyPattern); + } + first = false; + } + if (isSetExcludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("excludeParamKeyPattern:"); + if (this.excludeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.excludeParamKeyPattern); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -749,6 +911,22 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsByFilt org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 7: // INCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // EXCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -794,6 +972,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsByFil oprot.writeBool(struct.skipColumnSchemaForPartition); oprot.writeFieldEnd(); } + if (struct.includeParamKeyPattern != null) { + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeFieldBegin(INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.includeParamKeyPattern); + oprot.writeFieldEnd(); + } + } + if (struct.excludeParamKeyPattern != null) { + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeFieldBegin(EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.excludeParamKeyPattern); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -830,7 +1022,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByFilt if (struct.isSetSkipColumnSchemaForPartition()) { optionals.set(5); } - oprot.writeBitSet(optionals, 6); + if (struct.isSetIncludeParamKeyPattern()) { + optionals.set(6); + } + if (struct.isSetExcludeParamKeyPattern()) { + optionals.set(7); + } + oprot.writeBitSet(optionals, 8); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } @@ -849,12 +1047,18 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByFilt if (struct.isSetSkipColumnSchemaForPartition()) { oprot.writeBool(struct.skipColumnSchemaForPartition); } + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeString(struct.includeParamKeyPattern); + } + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeString(struct.excludeParamKeyPattern); + } } @Override public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByFilterRequest struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; - java.util.BitSet incoming = iprot.readBitSet(6); + java.util.BitSet incoming = iprot.readBitSet(8); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); @@ -879,6 +1083,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByFilte struct.skipColumnSchemaForPartition = iprot.readBool(); struct.setSkipColumnSchemaForPartitionIsSet(true); } + if (incoming.get(6)) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } + if (incoming.get(7)) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java index 4f91e2e10c40..2ecd9aa0c1a5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java @@ -22,6 +22,8 @@ private static final org.apache.thrift.protocol.TField GET_FILE_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("getFileMetadata", org.apache.thrift.protocol.TType.BOOL, (short)9); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)10); private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)11); + private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)12); + private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)13); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new GetPartitionsByNamesRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new GetPartitionsByNamesRequestTupleSchemeFactory(); @@ -37,6 +39,8 @@ private boolean getFileMetadata; // optional private long id; // optional private boolean skipColumnSchemaForPartition; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -50,7 +54,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { VALID_WRITE_ID_LIST((short)8, "validWriteIdList"), GET_FILE_METADATA((short)9, "getFileMetadata"), ID((short)10, "id"), - SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)11, "skipColumnSchemaForPartition"); + SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)11, "skipColumnSchemaForPartition"), + INCLUDE_PARAM_KEY_PATTERN((short)12, "includeParamKeyPattern"), + EXCLUDE_PARAM_KEY_PATTERN((short)13, "excludeParamKeyPattern"); private static final java.util.Map byName = new java.util.HashMap(); @@ -88,6 +94,10 @@ public static _Fields findByThriftId(int fieldId) { return ID; case 11: // SKIP_COLUMN_SCHEMA_FOR_PARTITION return SKIP_COLUMN_SCHEMA_FOR_PARTITION; + case 12: // INCLUDE_PARAM_KEY_PATTERN + return INCLUDE_PARAM_KEY_PATTERN; + case 13: // EXCLUDE_PARAM_KEY_PATTERN + return EXCLUDE_PARAM_KEY_PATTERN; default: return null; } @@ -134,7 +144,7 @@ public java.lang.String getFieldName() { private static final int __ID_ISSET_ID = 2; private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 3; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.NAMES,_Fields.GET_COL_STATS,_Fields.PROCESSOR_CAPABILITIES,_Fields.PROCESSOR_IDENTIFIER,_Fields.ENGINE,_Fields.VALID_WRITE_ID_LIST,_Fields.GET_FILE_METADATA,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION}; + private static final _Fields optionals[] = {_Fields.NAMES,_Fields.GET_COL_STATS,_Fields.PROCESSOR_CAPABILITIES,_Fields.PROCESSOR_IDENTIFIER,_Fields.ENGINE,_Fields.VALID_WRITE_ID_LIST,_Fields.GET_FILE_METADATA,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -162,6 +172,10 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION, new org.apache.thrift.meta_data.FieldMetaData("skipColumnSchemaForPartition", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.INCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("includeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsByNamesRequest.class, metaDataMap); } @@ -212,6 +226,12 @@ public GetPartitionsByNamesRequest(GetPartitionsByNamesRequest other) { this.getFileMetadata = other.getFileMetadata; this.id = other.id; this.skipColumnSchemaForPartition = other.skipColumnSchemaForPartition; + if (other.isSetIncludeParamKeyPattern()) { + this.includeParamKeyPattern = other.includeParamKeyPattern; + } + if (other.isSetExcludeParamKeyPattern()) { + this.excludeParamKeyPattern = other.excludeParamKeyPattern; + } } public GetPartitionsByNamesRequest deepCopy() { @@ -235,6 +255,8 @@ public void clear() { setSkipColumnSchemaForPartitionIsSet(false); this.skipColumnSchemaForPartition = false; + this.includeParamKeyPattern = null; + this.excludeParamKeyPattern = null; } @org.apache.thrift.annotation.Nullable @@ -525,6 +547,54 @@ public void setSkipColumnSchemaForPartitionIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID, value); } + @org.apache.thrift.annotation.Nullable + public java.lang.String getIncludeParamKeyPattern() { + return this.includeParamKeyPattern; + } + + public void setIncludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern) { + this.includeParamKeyPattern = includeParamKeyPattern; + } + + public void unsetIncludeParamKeyPattern() { + this.includeParamKeyPattern = null; + } + + /** Returns true if field includeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetIncludeParamKeyPattern() { + return this.includeParamKeyPattern != null; + } + + public void setIncludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.includeParamKeyPattern = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getExcludeParamKeyPattern() { + return this.excludeParamKeyPattern; + } + + public void setExcludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern) { + this.excludeParamKeyPattern = excludeParamKeyPattern; + } + + public void unsetExcludeParamKeyPattern() { + this.excludeParamKeyPattern = null; + } + + /** Returns true if field excludeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetExcludeParamKeyPattern() { + return this.excludeParamKeyPattern != null; + } + + public void setExcludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.excludeParamKeyPattern = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case DB_NAME: @@ -615,6 +685,22 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case INCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetIncludeParamKeyPattern(); + } else { + setIncludeParamKeyPattern((java.lang.String)value); + } + break; + + case EXCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetExcludeParamKeyPattern(); + } else { + setExcludeParamKeyPattern((java.lang.String)value); + } + break; + } } @@ -654,6 +740,12 @@ public java.lang.Object getFieldValue(_Fields field) { case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return getIncludeParamKeyPattern(); + + case EXCLUDE_PARAM_KEY_PATTERN: + return getExcludeParamKeyPattern(); + } throw new java.lang.IllegalStateException(); } @@ -687,6 +779,10 @@ public boolean isSet(_Fields field) { return isSetId(); case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSetSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return isSetIncludeParamKeyPattern(); + case EXCLUDE_PARAM_KEY_PATTERN: + return isSetExcludeParamKeyPattern(); } throw new java.lang.IllegalStateException(); } @@ -803,6 +899,24 @@ public boolean equals(GetPartitionsByNamesRequest that) { return false; } + boolean this_present_includeParamKeyPattern = true && this.isSetIncludeParamKeyPattern(); + boolean that_present_includeParamKeyPattern = true && that.isSetIncludeParamKeyPattern(); + if (this_present_includeParamKeyPattern || that_present_includeParamKeyPattern) { + if (!(this_present_includeParamKeyPattern && that_present_includeParamKeyPattern)) + return false; + if (!this.includeParamKeyPattern.equals(that.includeParamKeyPattern)) + return false; + } + + boolean this_present_excludeParamKeyPattern = true && this.isSetExcludeParamKeyPattern(); + boolean that_present_excludeParamKeyPattern = true && that.isSetExcludeParamKeyPattern(); + if (this_present_excludeParamKeyPattern || that_present_excludeParamKeyPattern) { + if (!(this_present_excludeParamKeyPattern && that_present_excludeParamKeyPattern)) + return false; + if (!this.excludeParamKeyPattern.equals(that.excludeParamKeyPattern)) + return false; + } + return true; } @@ -854,6 +968,14 @@ public int hashCode() { if (isSetSkipColumnSchemaForPartition()) hashCode = hashCode * 8191 + ((skipColumnSchemaForPartition) ? 131071 : 524287); + hashCode = hashCode * 8191 + ((isSetIncludeParamKeyPattern()) ? 131071 : 524287); + if (isSetIncludeParamKeyPattern()) + hashCode = hashCode * 8191 + includeParamKeyPattern.hashCode(); + + hashCode = hashCode * 8191 + ((isSetExcludeParamKeyPattern()) ? 131071 : 524287); + if (isSetExcludeParamKeyPattern()) + hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode(); + return hashCode; } @@ -975,6 +1097,26 @@ public int compareTo(GetPartitionsByNamesRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetIncludeParamKeyPattern(), other.isSetIncludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIncludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeParamKeyPattern, other.includeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetExcludeParamKeyPattern(), other.isSetExcludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExcludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeParamKeyPattern, other.excludeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1085,6 +1227,26 @@ public java.lang.String toString() { sb.append(this.skipColumnSchemaForPartition); first = false; } + if (isSetIncludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("includeParamKeyPattern:"); + if (this.includeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.includeParamKeyPattern); + } + first = false; + } + if (isSetExcludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("excludeParamKeyPattern:"); + if (this.excludeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.excludeParamKeyPattern); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -1246,6 +1408,22 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsByName org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 12: // INCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 13: // EXCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1338,6 +1516,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsByNam oprot.writeBool(struct.skipColumnSchemaForPartition); oprot.writeFieldEnd(); } + if (struct.includeParamKeyPattern != null) { + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeFieldBegin(INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.includeParamKeyPattern); + oprot.writeFieldEnd(); + } + } + if (struct.excludeParamKeyPattern != null) { + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeFieldBegin(EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.excludeParamKeyPattern); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1385,7 +1577,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByName if (struct.isSetSkipColumnSchemaForPartition()) { optionals.set(8); } - oprot.writeBitSet(optionals, 9); + if (struct.isSetIncludeParamKeyPattern()) { + optionals.set(9); + } + if (struct.isSetExcludeParamKeyPattern()) { + optionals.set(10); + } + oprot.writeBitSet(optionals, 11); if (struct.isSetNames()) { { oprot.writeI32(struct.names.size()); @@ -1425,6 +1623,12 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByName if (struct.isSetSkipColumnSchemaForPartition()) { oprot.writeBool(struct.skipColumnSchemaForPartition); } + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeString(struct.includeParamKeyPattern); + } + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeString(struct.excludeParamKeyPattern); + } } @Override @@ -1434,7 +1638,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByNames struct.setDb_nameIsSet(true); struct.tbl_name = iprot.readString(); struct.setTbl_nameIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(9); + java.util.BitSet incoming = iprot.readBitSet(11); if (incoming.get(0)) { { org.apache.thrift.protocol.TList _list732 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); @@ -1489,6 +1693,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByNames struct.skipColumnSchemaForPartition = iprot.readBool(); struct.setSkipColumnSchemaForPartitionIsSet(true); } + if (incoming.get(9)) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } + if (incoming.get(10)) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java index 865f3fe36646..8cd28971b2ae 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java @@ -21,6 +21,8 @@ private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)8); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)9); private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)10); + private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)11); + private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)12); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new GetPartitionsPsWithAuthRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new GetPartitionsPsWithAuthRequestTupleSchemeFactory(); @@ -35,6 +37,8 @@ private @org.apache.thrift.annotation.Nullable java.lang.String validWriteIdList; // optional private long id; // optional private boolean skipColumnSchemaForPartition; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -47,7 +51,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { GROUP_NAMES((short)7, "groupNames"), VALID_WRITE_ID_LIST((short)8, "validWriteIdList"), ID((short)9, "id"), - SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)10, "skipColumnSchemaForPartition"); + SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)10, "skipColumnSchemaForPartition"), + INCLUDE_PARAM_KEY_PATTERN((short)11, "includeParamKeyPattern"), + EXCLUDE_PARAM_KEY_PATTERN((short)12, "excludeParamKeyPattern"); private static final java.util.Map byName = new java.util.HashMap(); @@ -83,6 +89,10 @@ public static _Fields findByThriftId(int fieldId) { return ID; case 10: // SKIP_COLUMN_SCHEMA_FOR_PARTITION return SKIP_COLUMN_SCHEMA_FOR_PARTITION; + case 11: // INCLUDE_PARAM_KEY_PATTERN + return INCLUDE_PARAM_KEY_PATTERN; + case 12: // EXCLUDE_PARAM_KEY_PATTERN + return EXCLUDE_PARAM_KEY_PATTERN; default: return null; } @@ -128,7 +138,7 @@ public java.lang.String getFieldName() { private static final int __ID_ISSET_ID = 1; private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 2; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.PART_VALS,_Fields.MAX_PARTS,_Fields.USER_NAME,_Fields.GROUP_NAMES,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.PART_VALS,_Fields.MAX_PARTS,_Fields.USER_NAME,_Fields.GROUP_NAMES,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -154,6 +164,10 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION, new org.apache.thrift.meta_data.FieldMetaData("skipColumnSchemaForPartition", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.INCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("includeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsPsWithAuthRequest.class, metaDataMap); } @@ -205,6 +219,12 @@ public GetPartitionsPsWithAuthRequest(GetPartitionsPsWithAuthRequest other) { } this.id = other.id; this.skipColumnSchemaForPartition = other.skipColumnSchemaForPartition; + if (other.isSetIncludeParamKeyPattern()) { + this.includeParamKeyPattern = other.includeParamKeyPattern; + } + if (other.isSetExcludeParamKeyPattern()) { + this.excludeParamKeyPattern = other.excludeParamKeyPattern; + } } public GetPartitionsPsWithAuthRequest deepCopy() { @@ -226,6 +246,8 @@ public void clear() { setSkipColumnSchemaForPartitionIsSet(false); this.skipColumnSchemaForPartition = false; + this.includeParamKeyPattern = null; + this.excludeParamKeyPattern = null; } @org.apache.thrift.annotation.Nullable @@ -494,6 +516,54 @@ public void setSkipColumnSchemaForPartitionIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID, value); } + @org.apache.thrift.annotation.Nullable + public java.lang.String getIncludeParamKeyPattern() { + return this.includeParamKeyPattern; + } + + public void setIncludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern) { + this.includeParamKeyPattern = includeParamKeyPattern; + } + + public void unsetIncludeParamKeyPattern() { + this.includeParamKeyPattern = null; + } + + /** Returns true if field includeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetIncludeParamKeyPattern() { + return this.includeParamKeyPattern != null; + } + + public void setIncludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.includeParamKeyPattern = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getExcludeParamKeyPattern() { + return this.excludeParamKeyPattern; + } + + public void setExcludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern) { + this.excludeParamKeyPattern = excludeParamKeyPattern; + } + + public void unsetExcludeParamKeyPattern() { + this.excludeParamKeyPattern = null; + } + + /** Returns true if field excludeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetExcludeParamKeyPattern() { + return this.excludeParamKeyPattern != null; + } + + public void setExcludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.excludeParamKeyPattern = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case CAT_NAME: @@ -576,6 +646,22 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case INCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetIncludeParamKeyPattern(); + } else { + setIncludeParamKeyPattern((java.lang.String)value); + } + break; + + case EXCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetExcludeParamKeyPattern(); + } else { + setExcludeParamKeyPattern((java.lang.String)value); + } + break; + } } @@ -612,6 +698,12 @@ public java.lang.Object getFieldValue(_Fields field) { case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return getIncludeParamKeyPattern(); + + case EXCLUDE_PARAM_KEY_PATTERN: + return getExcludeParamKeyPattern(); + } throw new java.lang.IllegalStateException(); } @@ -643,6 +735,10 @@ public boolean isSet(_Fields field) { return isSetId(); case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSetSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return isSetIncludeParamKeyPattern(); + case EXCLUDE_PARAM_KEY_PATTERN: + return isSetExcludeParamKeyPattern(); } throw new java.lang.IllegalStateException(); } @@ -750,6 +846,24 @@ public boolean equals(GetPartitionsPsWithAuthRequest that) { return false; } + boolean this_present_includeParamKeyPattern = true && this.isSetIncludeParamKeyPattern(); + boolean that_present_includeParamKeyPattern = true && that.isSetIncludeParamKeyPattern(); + if (this_present_includeParamKeyPattern || that_present_includeParamKeyPattern) { + if (!(this_present_includeParamKeyPattern && that_present_includeParamKeyPattern)) + return false; + if (!this.includeParamKeyPattern.equals(that.includeParamKeyPattern)) + return false; + } + + boolean this_present_excludeParamKeyPattern = true && this.isSetExcludeParamKeyPattern(); + boolean that_present_excludeParamKeyPattern = true && that.isSetExcludeParamKeyPattern(); + if (this_present_excludeParamKeyPattern || that_present_excludeParamKeyPattern) { + if (!(this_present_excludeParamKeyPattern && that_present_excludeParamKeyPattern)) + return false; + if (!this.excludeParamKeyPattern.equals(that.excludeParamKeyPattern)) + return false; + } + return true; } @@ -797,6 +911,14 @@ public int hashCode() { if (isSetSkipColumnSchemaForPartition()) hashCode = hashCode * 8191 + ((skipColumnSchemaForPartition) ? 131071 : 524287); + hashCode = hashCode * 8191 + ((isSetIncludeParamKeyPattern()) ? 131071 : 524287); + if (isSetIncludeParamKeyPattern()) + hashCode = hashCode * 8191 + includeParamKeyPattern.hashCode(); + + hashCode = hashCode * 8191 + ((isSetExcludeParamKeyPattern()) ? 131071 : 524287); + if (isSetExcludeParamKeyPattern()) + hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode(); + return hashCode; } @@ -908,6 +1030,26 @@ public int compareTo(GetPartitionsPsWithAuthRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetIncludeParamKeyPattern(), other.isSetIncludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIncludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeParamKeyPattern, other.includeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetExcludeParamKeyPattern(), other.isSetExcludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExcludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeParamKeyPattern, other.excludeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1012,6 +1154,26 @@ public java.lang.String toString() { sb.append(this.skipColumnSchemaForPartition); first = false; } + if (isSetIncludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("includeParamKeyPattern:"); + if (this.includeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.includeParamKeyPattern); + } + first = false; + } + if (isSetExcludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("excludeParamKeyPattern:"); + if (this.excludeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.excludeParamKeyPattern); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -1165,6 +1327,22 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsPsWith org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 11: // INCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 12: // EXCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1252,6 +1430,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeBool(struct.skipColumnSchemaForPartition); oprot.writeFieldEnd(); } + if (struct.includeParamKeyPattern != null) { + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeFieldBegin(INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.includeParamKeyPattern); + oprot.writeFieldEnd(); + } + } + if (struct.excludeParamKeyPattern != null) { + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeFieldBegin(EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.excludeParamKeyPattern); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1296,7 +1488,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetSkipColumnSchemaForPartition()) { optionals.set(7); } - oprot.writeBitSet(optionals, 8); + if (struct.isSetIncludeParamKeyPattern()) { + optionals.set(8); + } + if (struct.isSetExcludeParamKeyPattern()) { + optionals.set(9); + } + oprot.writeBitSet(optionals, 10); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } @@ -1333,6 +1531,12 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetSkipColumnSchemaForPartition()) { oprot.writeBool(struct.skipColumnSchemaForPartition); } + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeString(struct.includeParamKeyPattern); + } + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeString(struct.excludeParamKeyPattern); + } } @Override @@ -1342,7 +1546,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA struct.setDbNameIsSet(true); struct.tblName = iprot.readString(); struct.setTblNameIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(8); + java.util.BitSet incoming = iprot.readBitSet(10); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); @@ -1393,6 +1597,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA struct.skipColumnSchemaForPartition = iprot.readBool(); struct.setSkipColumnSchemaForPartitionIsSet(true); } + if (incoming.get(8)) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } + if (incoming.get(9)) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java index 70894d36394f..7ae789bf46cb 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java @@ -21,6 +21,8 @@ private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)8); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)9); private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)10); + private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)11); + private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)12); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new PartitionsByExprRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new PartitionsByExprRequestTupleSchemeFactory(); @@ -35,6 +37,8 @@ private @org.apache.thrift.annotation.Nullable java.lang.String validWriteIdList; // optional private long id; // optional private boolean skipColumnSchemaForPartition; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -47,7 +51,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { ORDER((short)7, "order"), VALID_WRITE_ID_LIST((short)8, "validWriteIdList"), ID((short)9, "id"), - SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)10, "skipColumnSchemaForPartition"); + SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)10, "skipColumnSchemaForPartition"), + INCLUDE_PARAM_KEY_PATTERN((short)11, "includeParamKeyPattern"), + EXCLUDE_PARAM_KEY_PATTERN((short)12, "excludeParamKeyPattern"); private static final java.util.Map byName = new java.util.HashMap(); @@ -83,6 +89,10 @@ public static _Fields findByThriftId(int fieldId) { return ID; case 10: // SKIP_COLUMN_SCHEMA_FOR_PARTITION return SKIP_COLUMN_SCHEMA_FOR_PARTITION; + case 11: // INCLUDE_PARAM_KEY_PATTERN + return INCLUDE_PARAM_KEY_PATTERN; + case 12: // EXCLUDE_PARAM_KEY_PATTERN + return EXCLUDE_PARAM_KEY_PATTERN; default: return null; } @@ -128,7 +138,7 @@ public java.lang.String getFieldName() { private static final int __ID_ISSET_ID = 1; private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 2; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.DEFAULT_PARTITION_NAME,_Fields.MAX_PARTS,_Fields.CAT_NAME,_Fields.ORDER,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION}; + private static final _Fields optionals[] = {_Fields.DEFAULT_PARTITION_NAME,_Fields.MAX_PARTS,_Fields.CAT_NAME,_Fields.ORDER,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -152,6 +162,10 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION, new org.apache.thrift.meta_data.FieldMetaData("skipColumnSchemaForPartition", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.INCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("includeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionsByExprRequest.class, metaDataMap); } @@ -203,6 +217,12 @@ public PartitionsByExprRequest(PartitionsByExprRequest other) { } this.id = other.id; this.skipColumnSchemaForPartition = other.skipColumnSchemaForPartition; + if (other.isSetIncludeParamKeyPattern()) { + this.includeParamKeyPattern = other.includeParamKeyPattern; + } + if (other.isSetExcludeParamKeyPattern()) { + this.excludeParamKeyPattern = other.excludeParamKeyPattern; + } } public PartitionsByExprRequest deepCopy() { @@ -224,6 +244,8 @@ public void clear() { setSkipColumnSchemaForPartitionIsSet(false); this.skipColumnSchemaForPartition = false; + this.includeParamKeyPattern = null; + this.excludeParamKeyPattern = null; } @org.apache.thrift.annotation.Nullable @@ -468,6 +490,54 @@ public void setSkipColumnSchemaForPartitionIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID, value); } + @org.apache.thrift.annotation.Nullable + public java.lang.String getIncludeParamKeyPattern() { + return this.includeParamKeyPattern; + } + + public void setIncludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern) { + this.includeParamKeyPattern = includeParamKeyPattern; + } + + public void unsetIncludeParamKeyPattern() { + this.includeParamKeyPattern = null; + } + + /** Returns true if field includeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetIncludeParamKeyPattern() { + return this.includeParamKeyPattern != null; + } + + public void setIncludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.includeParamKeyPattern = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getExcludeParamKeyPattern() { + return this.excludeParamKeyPattern; + } + + public void setExcludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern) { + this.excludeParamKeyPattern = excludeParamKeyPattern; + } + + public void unsetExcludeParamKeyPattern() { + this.excludeParamKeyPattern = null; + } + + /** Returns true if field excludeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetExcludeParamKeyPattern() { + return this.excludeParamKeyPattern != null; + } + + public void setExcludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.excludeParamKeyPattern = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case DB_NAME: @@ -554,6 +624,22 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case INCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetIncludeParamKeyPattern(); + } else { + setIncludeParamKeyPattern((java.lang.String)value); + } + break; + + case EXCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetExcludeParamKeyPattern(); + } else { + setExcludeParamKeyPattern((java.lang.String)value); + } + break; + } } @@ -590,6 +676,12 @@ public java.lang.Object getFieldValue(_Fields field) { case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return getIncludeParamKeyPattern(); + + case EXCLUDE_PARAM_KEY_PATTERN: + return getExcludeParamKeyPattern(); + } throw new java.lang.IllegalStateException(); } @@ -621,6 +713,10 @@ public boolean isSet(_Fields field) { return isSetId(); case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSetSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return isSetIncludeParamKeyPattern(); + case EXCLUDE_PARAM_KEY_PATTERN: + return isSetExcludeParamKeyPattern(); } throw new java.lang.IllegalStateException(); } @@ -728,6 +824,24 @@ public boolean equals(PartitionsByExprRequest that) { return false; } + boolean this_present_includeParamKeyPattern = true && this.isSetIncludeParamKeyPattern(); + boolean that_present_includeParamKeyPattern = true && that.isSetIncludeParamKeyPattern(); + if (this_present_includeParamKeyPattern || that_present_includeParamKeyPattern) { + if (!(this_present_includeParamKeyPattern && that_present_includeParamKeyPattern)) + return false; + if (!this.includeParamKeyPattern.equals(that.includeParamKeyPattern)) + return false; + } + + boolean this_present_excludeParamKeyPattern = true && this.isSetExcludeParamKeyPattern(); + boolean that_present_excludeParamKeyPattern = true && that.isSetExcludeParamKeyPattern(); + if (this_present_excludeParamKeyPattern || that_present_excludeParamKeyPattern) { + if (!(this_present_excludeParamKeyPattern && that_present_excludeParamKeyPattern)) + return false; + if (!this.excludeParamKeyPattern.equals(that.excludeParamKeyPattern)) + return false; + } + return true; } @@ -775,6 +889,14 @@ public int hashCode() { if (isSetSkipColumnSchemaForPartition()) hashCode = hashCode * 8191 + ((skipColumnSchemaForPartition) ? 131071 : 524287); + hashCode = hashCode * 8191 + ((isSetIncludeParamKeyPattern()) ? 131071 : 524287); + if (isSetIncludeParamKeyPattern()) + hashCode = hashCode * 8191 + includeParamKeyPattern.hashCode(); + + hashCode = hashCode * 8191 + ((isSetExcludeParamKeyPattern()) ? 131071 : 524287); + if (isSetExcludeParamKeyPattern()) + hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode(); + return hashCode; } @@ -886,6 +1008,26 @@ public int compareTo(PartitionsByExprRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetIncludeParamKeyPattern(), other.isSetIncludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIncludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeParamKeyPattern, other.includeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetExcludeParamKeyPattern(), other.isSetExcludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExcludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeParamKeyPattern, other.excludeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -988,6 +1130,26 @@ public java.lang.String toString() { sb.append(this.skipColumnSchemaForPartition); first = false; } + if (isSetIncludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("includeParamKeyPattern:"); + if (this.includeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.includeParamKeyPattern); + } + first = false; + } + if (isSetExcludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("excludeParamKeyPattern:"); + if (this.excludeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.excludeParamKeyPattern); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -1125,6 +1287,22 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsByExprReq org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 11: // INCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 12: // EXCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1196,6 +1374,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, PartitionsByExprRe oprot.writeBool(struct.skipColumnSchemaForPartition); oprot.writeFieldEnd(); } + if (struct.includeParamKeyPattern != null) { + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeFieldBegin(INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.includeParamKeyPattern); + oprot.writeFieldEnd(); + } + } + if (struct.excludeParamKeyPattern != null) { + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeFieldBegin(EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.excludeParamKeyPattern); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1238,7 +1430,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprReq if (struct.isSetSkipColumnSchemaForPartition()) { optionals.set(6); } - oprot.writeBitSet(optionals, 7); + if (struct.isSetIncludeParamKeyPattern()) { + optionals.set(7); + } + if (struct.isSetExcludeParamKeyPattern()) { + optionals.set(8); + } + oprot.writeBitSet(optionals, 9); if (struct.isSetDefaultPartitionName()) { oprot.writeString(struct.defaultPartitionName); } @@ -1260,6 +1458,12 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprReq if (struct.isSetSkipColumnSchemaForPartition()) { oprot.writeBool(struct.skipColumnSchemaForPartition); } + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeString(struct.includeParamKeyPattern); + } + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeString(struct.excludeParamKeyPattern); + } } @Override @@ -1271,7 +1475,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprRequ struct.setTblNameIsSet(true); struct.expr = iprot.readBinary(); struct.setExprIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(7); + java.util.BitSet incoming = iprot.readBitSet(9); if (incoming.get(0)) { struct.defaultPartitionName = iprot.readString(); struct.setDefaultPartitionNameIsSet(true); @@ -1300,6 +1504,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprRequ struct.skipColumnSchemaForPartition = iprot.readBool(); struct.setSkipColumnSchemaForPartitionIsSet(true); } + if (incoming.get(7)) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } + if (incoming.get(8)) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsRequest.java index 52e2ec0d01f6..69c241ccad78 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsRequest.java @@ -18,6 +18,8 @@ private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)5); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)6); private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)7); + private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)8); + private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)9); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new PartitionsRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new PartitionsRequestTupleSchemeFactory(); @@ -29,6 +31,8 @@ private @org.apache.thrift.annotation.Nullable java.lang.String validWriteIdList; // optional private long id; // optional private boolean skipColumnSchemaForPartition; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional + private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -38,7 +42,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { MAX_PARTS((short)4, "maxParts"), VALID_WRITE_ID_LIST((short)5, "validWriteIdList"), ID((short)6, "id"), - SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)7, "skipColumnSchemaForPartition"); + SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)7, "skipColumnSchemaForPartition"), + INCLUDE_PARAM_KEY_PATTERN((short)8, "includeParamKeyPattern"), + EXCLUDE_PARAM_KEY_PATTERN((short)9, "excludeParamKeyPattern"); private static final java.util.Map byName = new java.util.HashMap(); @@ -68,6 +74,10 @@ public static _Fields findByThriftId(int fieldId) { return ID; case 7: // SKIP_COLUMN_SCHEMA_FOR_PARTITION return SKIP_COLUMN_SCHEMA_FOR_PARTITION; + case 8: // INCLUDE_PARAM_KEY_PATTERN + return INCLUDE_PARAM_KEY_PATTERN; + case 9: // EXCLUDE_PARAM_KEY_PATTERN + return EXCLUDE_PARAM_KEY_PATTERN; default: return null; } @@ -113,7 +123,7 @@ public java.lang.String getFieldName() { private static final int __ID_ISSET_ID = 1; private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 2; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.MAX_PARTS,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.MAX_PARTS,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -131,6 +141,10 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION, new org.apache.thrift.meta_data.FieldMetaData("skipColumnSchemaForPartition", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.INCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("includeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionsRequest.class, metaDataMap); } @@ -171,6 +185,12 @@ public PartitionsRequest(PartitionsRequest other) { } this.id = other.id; this.skipColumnSchemaForPartition = other.skipColumnSchemaForPartition; + if (other.isSetIncludeParamKeyPattern()) { + this.includeParamKeyPattern = other.includeParamKeyPattern; + } + if (other.isSetExcludeParamKeyPattern()) { + this.excludeParamKeyPattern = other.excludeParamKeyPattern; + } } public PartitionsRequest deepCopy() { @@ -189,6 +209,8 @@ public void clear() { setSkipColumnSchemaForPartitionIsSet(false); this.skipColumnSchemaForPartition = false; + this.includeParamKeyPattern = null; + this.excludeParamKeyPattern = null; } @org.apache.thrift.annotation.Nullable @@ -353,6 +375,54 @@ public void setSkipColumnSchemaForPartitionIsSet(boolean value) { __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID, value); } + @org.apache.thrift.annotation.Nullable + public java.lang.String getIncludeParamKeyPattern() { + return this.includeParamKeyPattern; + } + + public void setIncludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern) { + this.includeParamKeyPattern = includeParamKeyPattern; + } + + public void unsetIncludeParamKeyPattern() { + this.includeParamKeyPattern = null; + } + + /** Returns true if field includeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetIncludeParamKeyPattern() { + return this.includeParamKeyPattern != null; + } + + public void setIncludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.includeParamKeyPattern = null; + } + } + + @org.apache.thrift.annotation.Nullable + public java.lang.String getExcludeParamKeyPattern() { + return this.excludeParamKeyPattern; + } + + public void setExcludeParamKeyPattern(@org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern) { + this.excludeParamKeyPattern = excludeParamKeyPattern; + } + + public void unsetExcludeParamKeyPattern() { + this.excludeParamKeyPattern = null; + } + + /** Returns true if field excludeParamKeyPattern is set (has been assigned a value) and false otherwise */ + public boolean isSetExcludeParamKeyPattern() { + return this.excludeParamKeyPattern != null; + } + + public void setExcludeParamKeyPatternIsSet(boolean value) { + if (!value) { + this.excludeParamKeyPattern = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case CAT_NAME: @@ -411,6 +481,22 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case INCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetIncludeParamKeyPattern(); + } else { + setIncludeParamKeyPattern((java.lang.String)value); + } + break; + + case EXCLUDE_PARAM_KEY_PATTERN: + if (value == null) { + unsetExcludeParamKeyPattern(); + } else { + setExcludeParamKeyPattern((java.lang.String)value); + } + break; + } } @@ -438,6 +524,12 @@ public java.lang.Object getFieldValue(_Fields field) { case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return getIncludeParamKeyPattern(); + + case EXCLUDE_PARAM_KEY_PATTERN: + return getExcludeParamKeyPattern(); + } throw new java.lang.IllegalStateException(); } @@ -463,6 +555,10 @@ public boolean isSet(_Fields field) { return isSetId(); case SKIP_COLUMN_SCHEMA_FOR_PARTITION: return isSetSkipColumnSchemaForPartition(); + case INCLUDE_PARAM_KEY_PATTERN: + return isSetIncludeParamKeyPattern(); + case EXCLUDE_PARAM_KEY_PATTERN: + return isSetExcludeParamKeyPattern(); } throw new java.lang.IllegalStateException(); } @@ -543,6 +639,24 @@ public boolean equals(PartitionsRequest that) { return false; } + boolean this_present_includeParamKeyPattern = true && this.isSetIncludeParamKeyPattern(); + boolean that_present_includeParamKeyPattern = true && that.isSetIncludeParamKeyPattern(); + if (this_present_includeParamKeyPattern || that_present_includeParamKeyPattern) { + if (!(this_present_includeParamKeyPattern && that_present_includeParamKeyPattern)) + return false; + if (!this.includeParamKeyPattern.equals(that.includeParamKeyPattern)) + return false; + } + + boolean this_present_excludeParamKeyPattern = true && this.isSetExcludeParamKeyPattern(); + boolean that_present_excludeParamKeyPattern = true && that.isSetExcludeParamKeyPattern(); + if (this_present_excludeParamKeyPattern || that_present_excludeParamKeyPattern) { + if (!(this_present_excludeParamKeyPattern && that_present_excludeParamKeyPattern)) + return false; + if (!this.excludeParamKeyPattern.equals(that.excludeParamKeyPattern)) + return false; + } + return true; } @@ -578,6 +692,14 @@ public int hashCode() { if (isSetSkipColumnSchemaForPartition()) hashCode = hashCode * 8191 + ((skipColumnSchemaForPartition) ? 131071 : 524287); + hashCode = hashCode * 8191 + ((isSetIncludeParamKeyPattern()) ? 131071 : 524287); + if (isSetIncludeParamKeyPattern()) + hashCode = hashCode * 8191 + includeParamKeyPattern.hashCode(); + + hashCode = hashCode * 8191 + ((isSetExcludeParamKeyPattern()) ? 131071 : 524287); + if (isSetExcludeParamKeyPattern()) + hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode(); + return hashCode; } @@ -659,6 +781,26 @@ public int compareTo(PartitionsRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetIncludeParamKeyPattern(), other.isSetIncludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIncludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeParamKeyPattern, other.includeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = java.lang.Boolean.compare(isSetExcludeParamKeyPattern(), other.isSetExcludeParamKeyPattern()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExcludeParamKeyPattern()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeParamKeyPattern, other.excludeParamKeyPattern); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -733,6 +875,26 @@ public java.lang.String toString() { sb.append(this.skipColumnSchemaForPartition); first = false; } + if (isSetIncludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("includeParamKeyPattern:"); + if (this.includeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.includeParamKeyPattern); + } + first = false; + } + if (isSetExcludeParamKeyPattern()) { + if (!first) sb.append(", "); + sb.append("excludeParamKeyPattern:"); + if (this.excludeParamKeyPattern == null) { + sb.append("null"); + } else { + sb.append(this.excludeParamKeyPattern); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -842,6 +1004,22 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsRequest s org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 8: // INCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // EXCLUDE_PARAM_KEY_PATTERN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -894,6 +1072,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, PartitionsRequest oprot.writeBool(struct.skipColumnSchemaForPartition); oprot.writeFieldEnd(); } + if (struct.includeParamKeyPattern != null) { + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeFieldBegin(INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.includeParamKeyPattern); + oprot.writeFieldEnd(); + } + } + if (struct.excludeParamKeyPattern != null) { + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeFieldBegin(EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC); + oprot.writeString(struct.excludeParamKeyPattern); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -929,7 +1121,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsRequest s if (struct.isSetSkipColumnSchemaForPartition()) { optionals.set(4); } - oprot.writeBitSet(optionals, 5); + if (struct.isSetIncludeParamKeyPattern()) { + optionals.set(5); + } + if (struct.isSetExcludeParamKeyPattern()) { + optionals.set(6); + } + oprot.writeBitSet(optionals, 7); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } @@ -945,6 +1143,12 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsRequest s if (struct.isSetSkipColumnSchemaForPartition()) { oprot.writeBool(struct.skipColumnSchemaForPartition); } + if (struct.isSetIncludeParamKeyPattern()) { + oprot.writeString(struct.includeParamKeyPattern); + } + if (struct.isSetExcludeParamKeyPattern()) { + oprot.writeString(struct.excludeParamKeyPattern); + } } @Override @@ -954,7 +1158,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsRequest st struct.setDbNameIsSet(true); struct.tblName = iprot.readString(); struct.setTblNameIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(5); + java.util.BitSet incoming = iprot.readBitSet(7); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); @@ -975,6 +1179,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsRequest st struct.skipColumnSchemaForPartition = iprot.readBool(); struct.setSkipColumnSchemaForPartitionIsSet(true); } + if (incoming.get(5)) { + struct.includeParamKeyPattern = iprot.readString(); + struct.setIncludeParamKeyPatternIsSet(true); + } + if (incoming.get(6)) { + struct.excludeParamKeyPattern = iprot.readString(); + struct.setExcludeParamKeyPatternIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByFilterRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByFilterRequest.php index 549e4c908d70..348ae509ffdf 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByFilterRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByFilterRequest.php @@ -51,6 +51,16 @@ class GetPartitionsByFilterRequest 'isRequired' => false, 'type' => TType::BOOL, ), + 7 => array( + 'var' => 'includeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 8 => array( + 'var' => 'excludeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), ); /** @@ -77,6 +87,14 @@ class GetPartitionsByFilterRequest * @var bool */ public $skipColumnSchemaForPartition = null; + /** + * @var string + */ + public $includeParamKeyPattern = null; + /** + * @var string + */ + public $excludeParamKeyPattern = null; public function __construct($vals = null) { @@ -99,6 +117,12 @@ public function __construct($vals = null) if (isset($vals['skipColumnSchemaForPartition'])) { $this->skipColumnSchemaForPartition = $vals['skipColumnSchemaForPartition']; } + if (isset($vals['includeParamKeyPattern'])) { + $this->includeParamKeyPattern = $vals['includeParamKeyPattern']; + } + if (isset($vals['excludeParamKeyPattern'])) { + $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern']; + } } } @@ -163,6 +187,20 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 7: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->includeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; + case 8: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->excludeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -207,6 +245,16 @@ public function write($output) $xfer += $output->writeBool($this->skipColumnSchemaForPartition); $xfer += $output->writeFieldEnd(); } + if ($this->includeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('includeParamKeyPattern', TType::STRING, 7); + $xfer += $output->writeString($this->includeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } + if ($this->excludeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('excludeParamKeyPattern', TType::STRING, 8); + $xfer += $output->writeString($this->excludeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php index d0b4aa715214..c06aa5ac27e9 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php @@ -84,6 +84,16 @@ class GetPartitionsByNamesRequest 'isRequired' => false, 'type' => TType::BOOL, ), + 12 => array( + 'var' => 'includeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 13 => array( + 'var' => 'excludeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), ); /** @@ -130,6 +140,14 @@ class GetPartitionsByNamesRequest * @var bool */ public $skipColumnSchemaForPartition = null; + /** + * @var string + */ + public $includeParamKeyPattern = null; + /** + * @var string + */ + public $excludeParamKeyPattern = null; public function __construct($vals = null) { @@ -167,6 +185,12 @@ public function __construct($vals = null) if (isset($vals['skipColumnSchemaForPartition'])) { $this->skipColumnSchemaForPartition = $vals['skipColumnSchemaForPartition']; } + if (isset($vals['includeParamKeyPattern'])) { + $this->includeParamKeyPattern = $vals['includeParamKeyPattern']; + } + if (isset($vals['excludeParamKeyPattern'])) { + $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern']; + } } } @@ -284,6 +308,20 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 12: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->includeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; + case 13: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->excludeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -367,6 +405,16 @@ public function write($output) $xfer += $output->writeBool($this->skipColumnSchemaForPartition); $xfer += $output->writeFieldEnd(); } + if ($this->includeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('includeParamKeyPattern', TType::STRING, 12); + $xfer += $output->writeString($this->includeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } + if ($this->excludeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('excludeParamKeyPattern', TType::STRING, 13); + $xfer += $output->writeString($this->excludeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php index 0eecd2f5ad9b..19c68368627e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php @@ -79,6 +79,16 @@ class GetPartitionsPsWithAuthRequest 'isRequired' => false, 'type' => TType::BOOL, ), + 11 => array( + 'var' => 'includeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 12 => array( + 'var' => 'excludeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), ); /** @@ -121,6 +131,14 @@ class GetPartitionsPsWithAuthRequest * @var bool */ public $skipColumnSchemaForPartition = null; + /** + * @var string + */ + public $includeParamKeyPattern = null; + /** + * @var string + */ + public $excludeParamKeyPattern = null; public function __construct($vals = null) { @@ -155,6 +173,12 @@ public function __construct($vals = null) if (isset($vals['skipColumnSchemaForPartition'])) { $this->skipColumnSchemaForPartition = $vals['skipColumnSchemaForPartition']; } + if (isset($vals['includeParamKeyPattern'])) { + $this->includeParamKeyPattern = $vals['includeParamKeyPattern']; + } + if (isset($vals['excludeParamKeyPattern'])) { + $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern']; + } } } @@ -265,6 +289,20 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 11: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->includeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; + case 12: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->excludeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -343,6 +381,16 @@ public function write($output) $xfer += $output->writeBool($this->skipColumnSchemaForPartition); $xfer += $output->writeFieldEnd(); } + if ($this->includeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('includeParamKeyPattern', TType::STRING, 11); + $xfer += $output->writeString($this->includeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } + if ($this->excludeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('excludeParamKeyPattern', TType::STRING, 12); + $xfer += $output->writeString($this->excludeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsByExprRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsByExprRequest.php index 175fa5c3d5e8..23c31bfd8a33 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsByExprRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsByExprRequest.php @@ -71,6 +71,16 @@ class PartitionsByExprRequest 'isRequired' => false, 'type' => TType::BOOL, ), + 11 => array( + 'var' => 'includeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 12 => array( + 'var' => 'excludeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), ); /** @@ -113,6 +123,14 @@ class PartitionsByExprRequest * @var bool */ public $skipColumnSchemaForPartition = null; + /** + * @var string + */ + public $includeParamKeyPattern = null; + /** + * @var string + */ + public $excludeParamKeyPattern = null; public function __construct($vals = null) { @@ -147,6 +165,12 @@ public function __construct($vals = null) if (isset($vals['skipColumnSchemaForPartition'])) { $this->skipColumnSchemaForPartition = $vals['skipColumnSchemaForPartition']; } + if (isset($vals['includeParamKeyPattern'])) { + $this->includeParamKeyPattern = $vals['includeParamKeyPattern']; + } + if (isset($vals['excludeParamKeyPattern'])) { + $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern']; + } } } @@ -239,6 +263,20 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 11: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->includeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; + case 12: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->excludeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -303,6 +341,16 @@ public function write($output) $xfer += $output->writeBool($this->skipColumnSchemaForPartition); $xfer += $output->writeFieldEnd(); } + if ($this->includeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('includeParamKeyPattern', TType::STRING, 11); + $xfer += $output->writeString($this->includeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } + if ($this->excludeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('excludeParamKeyPattern', TType::STRING, 12); + $xfer += $output->writeString($this->excludeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsRequest.php index 7bf93c4034d4..a6167887646f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsRequest.php @@ -56,6 +56,16 @@ class PartitionsRequest 'isRequired' => false, 'type' => TType::BOOL, ), + 8 => array( + 'var' => 'includeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), + 9 => array( + 'var' => 'excludeParamKeyPattern', + 'isRequired' => false, + 'type' => TType::STRING, + ), ); /** @@ -86,6 +96,14 @@ class PartitionsRequest * @var bool */ public $skipColumnSchemaForPartition = null; + /** + * @var string + */ + public $includeParamKeyPattern = null; + /** + * @var string + */ + public $excludeParamKeyPattern = null; public function __construct($vals = null) { @@ -111,6 +129,12 @@ public function __construct($vals = null) if (isset($vals['skipColumnSchemaForPartition'])) { $this->skipColumnSchemaForPartition = $vals['skipColumnSchemaForPartition']; } + if (isset($vals['includeParamKeyPattern'])) { + $this->includeParamKeyPattern = $vals['includeParamKeyPattern']; + } + if (isset($vals['excludeParamKeyPattern'])) { + $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern']; + } } } @@ -182,6 +206,20 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 8: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->includeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; + case 9: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->excludeParamKeyPattern); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -231,6 +269,16 @@ public function write($output) $xfer += $output->writeBool($this->skipColumnSchemaForPartition); $xfer += $output->writeFieldEnd(); } + if ($this->includeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('includeParamKeyPattern', TType::STRING, 8); + $xfer += $output->writeString($this->includeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } + if ($this->excludeParamKeyPattern !== null) { + $xfer += $output->writeFieldBegin('excludeParamKeyPattern', TType::STRING, 9); + $xfer += $output->writeString($this->excludeParamKeyPattern); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py index a06a33c405b2..7b8052771aad 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -10113,11 +10113,13 @@ class PartitionsByExprRequest(object): - validWriteIdList - id - skipColumnSchemaForPartition + - includeParamKeyPattern + - excludeParamKeyPattern """ - def __init__(self, dbName=None, tblName=None, expr=None, defaultPartitionName=None, maxParts=-1, catName=None, order=None, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None,): + def __init__(self, dbName=None, tblName=None, expr=None, defaultPartitionName=None, maxParts=-1, catName=None, order=None, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): self.dbName = dbName self.tblName = tblName self.expr = expr @@ -10128,6 +10130,8 @@ def __init__(self, dbName=None, tblName=None, expr=None, defaultPartitionName=No self.validWriteIdList = validWriteIdList self.id = id self.skipColumnSchemaForPartition = skipColumnSchemaForPartition + self.includeParamKeyPattern = includeParamKeyPattern + self.excludeParamKeyPattern = excludeParamKeyPattern def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -10188,6 +10192,16 @@ def read(self, iprot): self.skipColumnSchemaForPartition = iprot.readBool() else: iprot.skip(ftype) + elif fid == 11: + if ftype == TType.STRING: + self.includeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 12: + if ftype == TType.STRING: + self.excludeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10238,6 +10252,14 @@ def write(self, oprot): oprot.writeFieldBegin('skipColumnSchemaForPartition', TType.BOOL, 10) oprot.writeBool(self.skipColumnSchemaForPartition) oprot.writeFieldEnd() + if self.includeParamKeyPattern is not None: + oprot.writeFieldBegin('includeParamKeyPattern', TType.STRING, 11) + oprot.writeString(self.includeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.includeParamKeyPattern) + oprot.writeFieldEnd() + if self.excludeParamKeyPattern is not None: + oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 12) + oprot.writeString(self.excludeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.excludeParamKeyPattern) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11700,11 +11722,13 @@ class GetPartitionsByNamesRequest(object): - getFileMetadata - id - skipColumnSchemaForPartition + - includeParamKeyPattern + - excludeParamKeyPattern """ - def __init__(self, db_name=None, tbl_name=None, names=None, get_col_stats=None, processorCapabilities=None, processorIdentifier=None, engine=None, validWriteIdList=None, getFileMetadata=None, id=-1, skipColumnSchemaForPartition=None,): + def __init__(self, db_name=None, tbl_name=None, names=None, get_col_stats=None, processorCapabilities=None, processorIdentifier=None, engine=None, validWriteIdList=None, getFileMetadata=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): self.db_name = db_name self.tbl_name = tbl_name self.names = names @@ -11716,6 +11740,8 @@ def __init__(self, db_name=None, tbl_name=None, names=None, get_col_stats=None, self.getFileMetadata = getFileMetadata self.id = id self.skipColumnSchemaForPartition = skipColumnSchemaForPartition + self.includeParamKeyPattern = includeParamKeyPattern + self.excludeParamKeyPattern = excludeParamKeyPattern def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -11791,6 +11817,16 @@ def read(self, iprot): self.skipColumnSchemaForPartition = iprot.readBool() else: iprot.skip(ftype) + elif fid == 12: + if ftype == TType.STRING: + self.includeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 13: + if ftype == TType.STRING: + self.excludeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -11851,6 +11887,14 @@ def write(self, oprot): oprot.writeFieldBegin('skipColumnSchemaForPartition', TType.BOOL, 11) oprot.writeBool(self.skipColumnSchemaForPartition) oprot.writeFieldEnd() + if self.includeParamKeyPattern is not None: + oprot.writeFieldBegin('includeParamKeyPattern', TType.STRING, 12) + oprot.writeString(self.includeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.includeParamKeyPattern) + oprot.writeFieldEnd() + if self.excludeParamKeyPattern is not None: + oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 13) + oprot.writeString(self.excludeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.excludeParamKeyPattern) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -27993,11 +28037,13 @@ class PartitionsRequest(object): - validWriteIdList - id - skipColumnSchemaForPartition + - includeParamKeyPattern + - excludeParamKeyPattern """ - def __init__(self, catName=None, dbName=None, tblName=None, maxParts=-1, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None,): + def __init__(self, catName=None, dbName=None, tblName=None, maxParts=-1, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): self.catName = catName self.dbName = dbName self.tblName = tblName @@ -28005,6 +28051,8 @@ def __init__(self, catName=None, dbName=None, tblName=None, maxParts=-1, validWr self.validWriteIdList = validWriteIdList self.id = id self.skipColumnSchemaForPartition = skipColumnSchemaForPartition + self.includeParamKeyPattern = includeParamKeyPattern + self.excludeParamKeyPattern = excludeParamKeyPattern def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -28050,6 +28098,16 @@ def read(self, iprot): self.skipColumnSchemaForPartition = iprot.readBool() else: iprot.skip(ftype) + elif fid == 8: + if ftype == TType.STRING: + self.includeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 9: + if ftype == TType.STRING: + self.excludeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -28088,6 +28146,14 @@ def write(self, oprot): oprot.writeFieldBegin('skipColumnSchemaForPartition', TType.BOOL, 7) oprot.writeBool(self.skipColumnSchemaForPartition) oprot.writeFieldEnd() + if self.includeParamKeyPattern is not None: + oprot.writeFieldBegin('includeParamKeyPattern', TType.STRING, 8) + oprot.writeString(self.includeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.includeParamKeyPattern) + oprot.writeFieldEnd() + if self.excludeParamKeyPattern is not None: + oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 9) + oprot.writeString(self.excludeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.excludeParamKeyPattern) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -28187,17 +28253,21 @@ class GetPartitionsByFilterRequest(object): - filter - maxParts - skipColumnSchemaForPartition + - includeParamKeyPattern + - excludeParamKeyPattern """ - def __init__(self, catName=None, dbName=None, tblName=None, filter=None, maxParts=-1, skipColumnSchemaForPartition=None,): + def __init__(self, catName=None, dbName=None, tblName=None, filter=None, maxParts=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): self.catName = catName self.dbName = dbName self.tblName = tblName self.filter = filter self.maxParts = maxParts self.skipColumnSchemaForPartition = skipColumnSchemaForPartition + self.includeParamKeyPattern = includeParamKeyPattern + self.excludeParamKeyPattern = excludeParamKeyPattern def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -28238,6 +28308,16 @@ def read(self, iprot): self.skipColumnSchemaForPartition = iprot.readBool() else: iprot.skip(ftype) + elif fid == 7: + if ftype == TType.STRING: + self.includeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.STRING: + self.excludeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -28272,6 +28352,14 @@ def write(self, oprot): oprot.writeFieldBegin('skipColumnSchemaForPartition', TType.BOOL, 6) oprot.writeBool(self.skipColumnSchemaForPartition) oprot.writeFieldEnd() + if self.includeParamKeyPattern is not None: + oprot.writeFieldBegin('includeParamKeyPattern', TType.STRING, 7) + oprot.writeString(self.includeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.includeParamKeyPattern) + oprot.writeFieldEnd() + if self.excludeParamKeyPattern is not None: + oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 8) + oprot.writeString(self.excludeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.excludeParamKeyPattern) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -28505,11 +28593,13 @@ class GetPartitionsPsWithAuthRequest(object): - validWriteIdList - id - skipColumnSchemaForPartition + - includeParamKeyPattern + - excludeParamKeyPattern """ - def __init__(self, catName=None, dbName=None, tblName=None, partVals=None, maxParts=-1, userName=None, groupNames=None, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None,): + def __init__(self, catName=None, dbName=None, tblName=None, partVals=None, maxParts=-1, userName=None, groupNames=None, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): self.catName = catName self.dbName = dbName self.tblName = tblName @@ -28520,6 +28610,8 @@ def __init__(self, catName=None, dbName=None, tblName=None, partVals=None, maxPa self.validWriteIdList = validWriteIdList self.id = id self.skipColumnSchemaForPartition = skipColumnSchemaForPartition + self.includeParamKeyPattern = includeParamKeyPattern + self.excludeParamKeyPattern = excludeParamKeyPattern def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -28590,6 +28682,16 @@ def read(self, iprot): self.skipColumnSchemaForPartition = iprot.readBool() else: iprot.skip(ftype) + elif fid == 11: + if ftype == TType.STRING: + self.includeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) + elif fid == 12: + if ftype == TType.STRING: + self.excludeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -28646,6 +28748,14 @@ def write(self, oprot): oprot.writeFieldBegin('skipColumnSchemaForPartition', TType.BOOL, 10) oprot.writeBool(self.skipColumnSchemaForPartition) oprot.writeFieldEnd() + if self.includeParamKeyPattern is not None: + oprot.writeFieldBegin('includeParamKeyPattern', TType.STRING, 11) + oprot.writeString(self.includeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.includeParamKeyPattern) + oprot.writeFieldEnd() + if self.excludeParamKeyPattern is not None: + oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 12) + oprot.writeString(self.excludeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.excludeParamKeyPattern) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -31892,6 +32002,8 @@ def __ne__(self, other): (8, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 8 (9, TType.I64, 'id', None, -1, ), # 9 (10, TType.BOOL, 'skipColumnSchemaForPartition', None, None, ), # 10 + (11, TType.STRING, 'includeParamKeyPattern', 'UTF8', None, ), # 11 + (12, TType.STRING, 'excludeParamKeyPattern', 'UTF8', None, ), # 12 ) all_structs.append(TableStatsResult) TableStatsResult.thrift_spec = ( @@ -32016,6 +32128,8 @@ def __ne__(self, other): (9, TType.BOOL, 'getFileMetadata', None, None, ), # 9 (10, TType.I64, 'id', None, -1, ), # 10 (11, TType.BOOL, 'skipColumnSchemaForPartition', None, None, ), # 11 + (12, TType.STRING, 'includeParamKeyPattern', 'UTF8', None, ), # 12 + (13, TType.STRING, 'excludeParamKeyPattern', 'UTF8', None, ), # 13 ) all_structs.append(GetPartitionsByNamesResult) GetPartitionsByNamesResult.thrift_spec = ( @@ -33348,6 +33462,8 @@ def __ne__(self, other): (5, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 5 (6, TType.I64, 'id', None, -1, ), # 6 (7, TType.BOOL, 'skipColumnSchemaForPartition', None, None, ), # 7 + (8, TType.STRING, 'includeParamKeyPattern', 'UTF8', None, ), # 8 + (9, TType.STRING, 'excludeParamKeyPattern', 'UTF8', None, ), # 9 ) all_structs.append(PartitionsResponse) PartitionsResponse.thrift_spec = ( @@ -33363,6 +33479,8 @@ def __ne__(self, other): (4, TType.STRING, 'filter', 'UTF8', None, ), # 4 (5, TType.I16, 'maxParts', None, -1, ), # 5 (6, TType.BOOL, 'skipColumnSchemaForPartition', None, None, ), # 6 + (7, TType.STRING, 'includeParamKeyPattern', 'UTF8', None, ), # 7 + (8, TType.STRING, 'excludeParamKeyPattern', 'UTF8', None, ), # 8 ) all_structs.append(GetPartitionNamesPsRequest) GetPartitionNamesPsRequest.thrift_spec = ( @@ -33393,6 +33511,8 @@ def __ne__(self, other): (8, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 8 (9, TType.I64, 'id', None, -1, ), # 9 (10, TType.BOOL, 'skipColumnSchemaForPartition', None, None, ), # 10 + (11, TType.STRING, 'includeParamKeyPattern', 'UTF8', None, ), # 11 + (12, TType.STRING, 'excludeParamKeyPattern', 'UTF8', None, ), # 12 ) all_structs.append(GetPartitionsPsWithAuthResponse) GetPartitionsPsWithAuthResponse.thrift_spec = ( diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb index 72595ed42688..ab4c608d8e8e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -3191,6 +3191,8 @@ class PartitionsByExprRequest VALIDWRITEIDLIST = 8 ID = 9 SKIPCOLUMNSCHEMAFORPARTITION = 10 + INCLUDEPARAMKEYPATTERN = 11 + EXCLUDEPARAMKEYPATTERN = 12 FIELDS = { DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}, @@ -3202,7 +3204,9 @@ class PartitionsByExprRequest ORDER => {:type => ::Thrift::Types::STRING, :name => 'order', :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true}, - SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true} + SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true}, + INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern', :optional => true}, + EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true} } def struct_fields; FIELDS; end @@ -3560,6 +3564,8 @@ class GetPartitionsByNamesRequest GETFILEMETADATA = 9 ID = 10 SKIPCOLUMNSCHEMAFORPARTITION = 11 + INCLUDEPARAMKEYPATTERN = 12 + EXCLUDEPARAMKEYPATTERN = 13 FIELDS = { DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'}, @@ -3572,7 +3578,9 @@ class GetPartitionsByNamesRequest VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, GETFILEMETADATA => {:type => ::Thrift::Types::BOOL, :name => 'getFileMetadata', :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true}, - SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true} + SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true}, + INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern', :optional => true}, + EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true} } def struct_fields; FIELDS; end @@ -7660,6 +7668,8 @@ class PartitionsRequest VALIDWRITEIDLIST = 5 ID = 6 SKIPCOLUMNSCHEMAFORPARTITION = 7 + INCLUDEPARAMKEYPATTERN = 8 + EXCLUDEPARAMKEYPATTERN = 9 FIELDS = { CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, @@ -7668,7 +7678,9 @@ class PartitionsRequest MAXPARTS => {:type => ::Thrift::Types::I16, :name => 'maxParts', :default => -1, :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true}, - SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true} + SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true}, + INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern', :optional => true}, + EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true} } def struct_fields; FIELDS; end @@ -7706,6 +7718,8 @@ class GetPartitionsByFilterRequest FILTER = 4 MAXPARTS = 5 SKIPCOLUMNSCHEMAFORPARTITION = 6 + INCLUDEPARAMKEYPATTERN = 7 + EXCLUDEPARAMKEYPATTERN = 8 FIELDS = { CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, @@ -7713,7 +7727,9 @@ class GetPartitionsByFilterRequest TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'}, FILTER => {:type => ::Thrift::Types::STRING, :name => 'filter'}, MAXPARTS => {:type => ::Thrift::Types::I16, :name => 'maxParts', :default => -1, :optional => true}, - SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true} + SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true}, + INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern', :optional => true}, + EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true} } def struct_fields; FIELDS; end @@ -7783,6 +7799,8 @@ class GetPartitionsPsWithAuthRequest VALIDWRITEIDLIST = 8 ID = 9 SKIPCOLUMNSCHEMAFORPARTITION = 10 + INCLUDEPARAMKEYPATTERN = 11 + EXCLUDEPARAMKEYPATTERN = 12 FIELDS = { CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, @@ -7794,7 +7812,9 @@ class GetPartitionsPsWithAuthRequest GROUPNAMES => {:type => ::Thrift::Types::LIST, :name => 'groupNames', :element => {:type => ::Thrift::Types::STRING}, :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true}, - SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true} + SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true}, + INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern', :optional => true}, + EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true} } def struct_fields; FIELDS; end diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index 228baed81aef..71a8dfb62008 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_TABLE; import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME; import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.convertToGetPartitionsByNamesRequest; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.createThriftPartitionsReq; import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName; @@ -1731,10 +1732,7 @@ public void dropDatabase(DropDatabaseRequest req) * server side when the client invokes drop_database. * Note that this is 'less transactional' than dropDatabaseCascadePerDb since we're dropping * table level objects, so the overall outcome of this method might result in a halfly dropped DB. - * @param catName - * @param dbName * @param tableList - * @param deleteData * @param maxBatchSize * @throws TException */ @@ -2188,12 +2186,12 @@ public List listPartitions(String catName, String db_name, String tbl if (db_name == null || tbl_name == null) { throw new MetaException("Database name/Table name should not be null"); } - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); // TODO should we add capabilities here as well as it returns Partition objects - PartitionsRequest req = new PartitionsRequest(db_name, tbl_name); + PartitionsRequest req = createThriftPartitionsReq(PartitionsRequest.class, conf); + req.setDbName(db_name); + req.setTblName(tbl_name); req.setCatName(catName); req.setMaxParts(shrinkMaxtoShort(max_parts)); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); List parts = client.get_partitions_req(req).getPartitions(); return deepCopyPartitions( FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts)); @@ -2227,12 +2225,12 @@ public List listPartitions(String catName, String db_name, String tbl if (db_name == null || tbl_name == null || part_vals == null) { throw new MetaException("Database name/Table name/partition values should not be null"); } - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest(db_name, tbl_name); + GetPartitionsPsWithAuthRequest req = createThriftPartitionsReq(GetPartitionsPsWithAuthRequest.class, conf); + req.setDbName(db_name); + req.setTblName(tbl_name); req.setCatName(catName); req.setPartVals(part_vals); req.setMaxParts(shrinkMaxtoShort(max_parts)); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); List parts = client.get_partitions_ps_with_auth_req(req).getPartitions(); return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts)); } @@ -2265,9 +2263,8 @@ public GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequest(GetPart protected GetPartitionsPsWithAuthResponse listPartitionsWithAuthInfoRequestInternal(GetPartitionsPsWithAuthRequest req) throws TException { - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); - return client.get_partitions_ps_with_auth_req(req); + return client.get_partitions_ps_with_auth_req( + createThriftPartitionsReq(GetPartitionsPsWithAuthRequest.class, conf, req)); } @Override @@ -2295,13 +2292,13 @@ protected List listPartitionsWithAuthInfoInternal(String catName, Str if (dbName == null || tableName == null) { throw new MetaException("Database name/Table name should not be null"); } - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest(dbName, tableName); + GetPartitionsPsWithAuthRequest req = createThriftPartitionsReq(GetPartitionsPsWithAuthRequest.class, conf); + req.setTblName(tableName); + req.setDbName(dbName); req.setCatName(catName); req.setMaxParts(shrinkMaxtoShort(maxParts)); req.setUserName(userName); req.setGroupNames(groupNames); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); List partsList = client.get_partitions_ps_with_auth_req(req).getPartitions(); return partsList; } @@ -2344,14 +2341,14 @@ protected List listPartitionsWithAuthInfoInternal(String catName, Str if (dbName == null || tableName == null || partialPvals == null) { throw new MetaException("Database name/Table name/partition values should not be null"); } - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest(dbName, tableName); + GetPartitionsPsWithAuthRequest req = createThriftPartitionsReq(GetPartitionsPsWithAuthRequest.class, conf); + req.setTblName(tableName); + req.setDbName(dbName); req.setCatName(catName); req.setPartVals(partialPvals); req.setMaxParts(shrinkMaxtoShort(maxParts)); req.setUserName(userName); req.setGroupNames(groupNames); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); return client.get_partitions_ps_with_auth_req(req).getPartitions(); } @@ -2364,9 +2361,14 @@ public List listPartitionsByFilter(String db_name, String tbl_name, @Override public List listPartitionsByFilter(String catName, String db_name, String tbl_name, String filter, int max_parts) throws TException { + GetPartitionsByFilterRequest req = createThriftPartitionsReq(GetPartitionsByFilterRequest.class, conf); + req.setTblName(tbl_name); + req.setDbName(db_name); + req.setCatName(catName); + req.setFilter(filter); + req.setMaxParts(shrinkMaxtoShort(max_parts)); // TODO should we add capabilities here as well as it returns Partition objects - List parts = client.get_partitions_by_filter(prependCatalogToDbName( - catName, db_name, conf), tbl_name, filter, shrinkMaxtoShort(max_parts)); + List parts = client.get_partitions_by_filter_req(req); return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts)); } @@ -2417,9 +2419,7 @@ protected PartitionsByExprRequest buildPartitionsByExprRequest(String catName, S } protected PartitionsByExprResult getPartitionsByExprInternal(PartitionsByExprRequest req) throws TException { - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); - return client.get_partitions_by_expr(req); + return client.get_partitions_by_expr(createThriftPartitionsReq(PartitionsByExprRequest.class, conf, req)); } @Override @@ -2469,7 +2469,7 @@ private void rethrowException(TApplicationException te) throws TException{ } protected PartitionsSpecByExprResult getPartitionsSpecByExprInternal(PartitionsByExprRequest req) throws TException { - return client.get_partitions_spec_by_expr(req); + return client.get_partitions_spec_by_expr(createThriftPartitionsReq(PartitionsByExprRequest.class, conf, req)); } @Override @@ -2574,13 +2574,10 @@ public List getPartitionsByNames(String db_name, String tbl_name, @Override public PartitionsResponse getPartitionsRequest(PartitionsRequest req) throws NoSuchObjectException, MetaException, TException { - if (req.getValidWriteIdList() == null) { req.setValidWriteIdList(getValidWriteIdList(req.getDbName(), req.getTblName())); } - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); - PartitionsResponse res = client.get_partitions_req(req); + PartitionsResponse res = client.get_partitions_req(createThriftPartitionsReq(PartitionsRequest.class, conf, req)); List parts = deepCopyPartitions( FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, res.getPartitions())); res.setPartitions(parts); @@ -2607,8 +2604,6 @@ public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesReque req.setProcessorCapabilities(new ArrayList<>(Arrays.asList(processorCapabilities))); if (processorIdentifier != null) req.setProcessorIdentifier(processorIdentifier); - boolean skipColumnSchemaForPartition = MetastoreConf.getBoolVar(conf, ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS); - req.setSkipColumnSchemaForPartition(skipColumnSchemaForPartition); List parts = getPartitionsByNamesInternal(req).getPartitions(); GetPartitionsByNamesResult res = new GetPartitionsByNamesResult(); res.setPartitions(deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled( @@ -2618,7 +2613,7 @@ public GetPartitionsByNamesResult getPartitionsByNames(GetPartitionsByNamesReque protected GetPartitionsByNamesResult getPartitionsByNamesInternal(GetPartitionsByNamesRequest gpbnr) throws TException { - return client.get_partitions_by_names_req(gpbnr); + return client.get_partitions_by_names_req(createThriftPartitionsReq(GetPartitionsByNamesRequest.class, conf, gpbnr)); } @Override @@ -5119,6 +5114,16 @@ public GetPartitionsResponse getPartitionsWithSpecs(GetPartitionsRequest request request.setProcessorCapabilities(new ArrayList(Arrays.asList(processorCapabilities))); if (processorIdentifier != null) request.setProcessorIdentifier(processorIdentifier); + if (request.isSetProjectionSpec()) { + if (!request.getProjectionSpec().isSetExcludeParamKeyPattern()) { + request.getProjectionSpec().setExcludeParamKeyPattern(MetastoreConf.getAsString(conf, + MetastoreConf.ConfVars.METASTORE_PARTITIONS_PARAMETERS_EXCLUDE_PATTERN)); + } + if (!request.getProjectionSpec().isSetIncludeParamKeyPattern()) { + request.getProjectionSpec().setIncludeParamKeyPattern(MetastoreConf.getAsString(conf, + MetastoreConf.ConfVars.METASTORE_PARTITIONS_PARAMETERS_INCLUDE_PATTERN)); + } + } return client.get_partitions_with_specs(request); } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index f8814a6838fd..7208867eaa4f 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -240,7 +240,9 @@ public String toString() { ConfVars.DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES, ConfVars.FILE_METADATA_THREADS, ConfVars.METASTORE_CLIENT_FILTER_ENABLED, - ConfVars.METASTORE_SERVER_FILTER_ENABLED + ConfVars.METASTORE_SERVER_FILTER_ENABLED, + ConfVars.METASTORE_PARTITIONS_PARAMETERS_INCLUDE_PATTERN, + ConfVars.METASTORE_PARTITIONS_PARAMETERS_EXCLUDE_PATTERN }; /** @@ -1127,6 +1129,16 @@ public enum ConfVars { + "table carries the field schema that is same as that of table schema. For a table with \n" + "wider partitions fetching duplicated field schema in every partition increases memory footprint\n" + "and thrift communication timeout errors. Set this config to 'true' to ignore column schema in partitions."), + METASTORE_PARTITIONS_PARAMETERS_EXCLUDE_PATTERN("metastore.partitions.parameters.exclude.pattern", + "hive.metastore.partitions.parameters.exclude.pattern", "", + "SQL pattern used to exclude the matched parameters for get-partitions APIs.\n" + + "Any key-value pair from parameters whose key matches with the pattern will be excluded from the partitions.\n" + + "This property doesn't work for the temporary table."), + METASTORE_PARTITIONS_PARAMETERS_INCLUDE_PATTERN("metastore.partitions.parameters.include.pattern", + "hive.metastore.partitions.parameters.include.pattern", "", + "SQL pattern used to select the matched parameters for get-partitions APIs.\n" + + "Any key-value pair from parameters whose key matches with the pattern will be included in the partitions.\n" + + "This property doesn't work for the temporary table."), METASTORE_CLIENT_FILTER_ENABLED("metastore.client.filter.enabled", "hive.metastore.client.filter.enabled", true, "Enable filtering the metadata read results at HMS client. Default is true."), METASTORE_SERVER_FILTER_ENABLED("metastore.server.filter.enabled", "hive.metastore.server.filter.enabled", false, diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java index 503345d043d7..d00c8f050191 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/JavaUtils.java @@ -23,6 +23,7 @@ import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; +import java.lang.reflect.Method; import java.net.InetAddress; import java.net.UnknownHostException; @@ -115,6 +116,18 @@ public static String hostname() { } } + public static void setField(T req, String methodName, Class[] argsCls, Object... args) { + try { + Method method = req.getClass().getDeclaredMethod(methodName, argsCls); + method.setAccessible(true); + method.invoke(req, args); + } catch (Exception e) { + LOG.error("Unable to invoke the underlying method: {} of the instance: {}, message: {}", + methodName, req, e.getMessage()); + throw new RuntimeException(e); + } + } + /** * Utility method for ACID to normalize logging info. Matches * org.apache.hadoop.hive.metastore.api.LockRequest#toString diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java index a95c4c1d19be..1491b45a3537 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java @@ -1261,6 +1261,23 @@ public static GetPartitionsByNamesRequest convertToGetPartitionsByNamesRequest(S return result; } + public static T createThriftPartitionsReq(Class clazz, Configuration conf, T... deepCopy) { + final T req; + if (deepCopy != null && deepCopy.length == 1) { + assert clazz.isAssignableFrom(deepCopy[0].getClass()); + req = JavaUtils.newInstance(clazz, new Class[]{clazz}, deepCopy); + } else { + req = JavaUtils.newInstance(clazz); + } + JavaUtils.setField(req, "setSkipColumnSchemaForPartition", new Class[]{boolean.class}, + MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_CLIENT_FIELD_SCHEMA_FOR_PARTITIONS)); + JavaUtils.setField(req, "setIncludeParamKeyPattern", new Class[]{String.class}, + MetastoreConf.getAsString(conf, MetastoreConf.ConfVars.METASTORE_PARTITIONS_PARAMETERS_INCLUDE_PATTERN)); + JavaUtils.setField(req, "setExcludeParamKeyPattern", new Class[]{String.class}, + MetastoreConf.getAsString(conf, MetastoreConf.ConfVars.METASTORE_PARTITIONS_PARAMETERS_EXCLUDE_PATTERN)); + return req; + } + /** * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on. * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*" diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift index 935e286c85d4..e5869736403f 100644 --- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift +++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift @@ -880,7 +880,9 @@ struct PartitionsByExprRequest { 7: optional string order 8: optional string validWriteIdList, 9: optional i64 id=-1, // table id - 10: optional bool skipColumnSchemaForPartition + 10: optional bool skipColumnSchemaForPartition, + 11: optional string includeParamKeyPattern, + 12: optional string excludeParamKeyPattern } struct TableStatsResult { @@ -995,7 +997,9 @@ struct GetPartitionsByNamesRequest { 8: optional string validWriteIdList, 9: optional bool getFileMetadata, 10: optional i64 id=-1, // table id - 11: optional bool skipColumnSchemaForPartition + 11: optional bool skipColumnSchemaForPartition, + 12: optional string includeParamKeyPattern, + 13: optional string excludeParamKeyPattern } struct GetPartitionsByNamesResult { @@ -2310,7 +2314,9 @@ struct PartitionsRequest { // Not using Get prefix as that name is already used 4: optional i16 maxParts=-1, 5: optional string validWriteIdList, 6: optional i64 id=-1, // table id - 7: optional bool skipColumnSchemaForPartition + 7: optional bool skipColumnSchemaForPartition, + 8: optional string includeParamKeyPattern, + 9: optional string excludeParamKeyPattern } struct PartitionsResponse { // Not using Get prefix as that name is already used for a different method @@ -2323,7 +2329,9 @@ struct GetPartitionsByFilterRequest { 3: string tblName, 4: string filter, 5: optional i16 maxParts=-1, - 6: optional bool skipColumnSchemaForPartition + 6: optional bool skipColumnSchemaForPartition, + 7: optional string includeParamKeyPattern, + 8: optional string excludeParamKeyPattern } struct GetPartitionNamesPsRequest { @@ -2350,7 +2358,9 @@ struct GetPartitionsPsWithAuthRequest { 7: optional list groupNames, 8: optional string validWriteIdList, 9: optional i64 id=-1 // table id - 10: optional bool skipColumnSchemaForPartition + 10: optional bool skipColumnSchemaForPartition, + 11: optional string includeParamKeyPattern, + 12: optional string excludeParamKeyPattern } struct GetPartitionsPsWithAuthResponse { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index c908028bea88..feb9701fac49 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hive.common.repl.ReplConst; import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.api.Package; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; import org.apache.hadoop.hive.metastore.dataconnector.DataConnectorProviderFactory; @@ -2320,9 +2321,10 @@ private void create_table_core(final RawStore ms, final CreateTableRequest req) if (!TableType.VIRTUAL_VIEW.toString().equals(tbl.getTableType())) { if (tbl.getSd().getLocation() == null || tbl.getSd().getLocation().isEmpty()) { - tblPath = wh.getDefaultTablePath(db, tbl.getTableName() + getTableSuffix(tbl), isExternal(tbl)); + tblPath = wh.getDefaultTablePath(db, tbl.getTableName() + getTableSuffix(tbl), + MetaStoreUtils.isExternalTable(tbl)); } else { - if (!isExternal(tbl) && !MetaStoreUtils.isNonNativeTable(tbl)) { + if (!MetaStoreUtils.isExternalTable(tbl) && !MetaStoreUtils.isNonNativeTable(tbl)) { LOG.warn("Location: " + tbl.getSd().getLocation() + " specified for non-external table:" + tbl.getTableName()); } @@ -2970,9 +2972,9 @@ private boolean drop_table_core(final RawStore ms, final String catName, final S } private boolean checkTableDataShouldBeDeleted(Table tbl, boolean deleteData) { - if (deleteData && isExternal(tbl)) { + if (deleteData && MetaStoreUtils.isExternalTable(tbl)) { // External table data can be deleted if EXTERNAL_TABLE_PURGE is true - return isExternalTablePurge(tbl); + return MetaStoreUtils.isExternalTablePurge(tbl); } return deleteData; } @@ -3413,21 +3415,6 @@ private void truncateDataFiles(Path location, boolean isSkipTrash, boolean needC } } - /** - * Is this an external table? - * - * @param table - * Check if this table is external. - * @return True if the table is external, otherwise false. - */ - private boolean isExternal(Table table) { - return MetaStoreUtils.isExternalTable(table); - } - - private boolean isExternalTablePurge(Table table) { - return MetaStoreUtils.isExternalTablePurge(table); - } - @Override @Deprecated public Table get_table(final String dbname, final String name) throws MetaException, @@ -5151,9 +5138,10 @@ public DropPartitionsResult drop_partitions_req( for (DropPartitionsExpr expr : spec.getExprs()) { ++minCount; // At least one partition per expression, if not ifExists List result = new ArrayList<>(); - boolean hasUnknown = ms.getPartitionsByExpr( - catName, dbName, tblName, expr.getExpr(), null, - (short)-1, result, request.isSkipColumnSchemaForPartition()); + boolean hasUnknown = ms.getPartitionsByExpr(catName, dbName, tblName, result, + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .expr(expr.getExpr()).skipColumnSchemaForPartition(request.isSkipColumnSchemaForPartition()) + .build()); if (hasUnknown) { // Expr is built by DDLSA, it should only contain part cols and simple ops throw new MetaException("Unexpected unknown partitions to drop"); @@ -5174,8 +5162,10 @@ public DropPartitionsResult drop_partitions_req( } else if (spec.isSetNames()) { partNames = spec.getNames(); minCount = partNames.size(); - parts = ms.getPartitionsByNames(catName, dbName, tblName, partNames, - request.isSkipColumnSchemaForPartition()); + parts = ms.getPartitionsByNames(catName, dbName, tblName, + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .partNames(partNames).skipColumnSchemaForPartition(request.isSkipColumnSchemaForPartition()) + .build()); } else { throw new MetaException("Partition spec is not set"); } @@ -5491,12 +5481,12 @@ public Partition get_partition_with_auth(final String db_name, @Deprecated public List get_partitions(final String db_name, final String tbl_name, final short max_parts) throws NoSuchObjectException, MetaException { - return get_partitions(db_name, tbl_name, max_parts, false); - + return get_partitions(db_name, tbl_name, + new GetPartitionsArgs.GetPartitionsArgsBuilder().max(max_parts).build()); } private List get_partitions(final String db_name, final String tbl_name, - final short max_parts, boolean skipColumnSchemaForPartition) throws NoSuchObjectException, MetaException { + GetPartitionsArgs args) throws NoSuchObjectException, MetaException { String[] parsedDbName = parseDbName(db_name, conf); startTableFunction("get_partitions", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); @@ -5504,12 +5494,11 @@ private List get_partitions(final String db_name, final String tbl_na Exception ex = null; try { checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], - tbl_name, NO_FILTER_STRING, max_parts); + tbl_name, NO_FILTER_STRING, args.getMax()); authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); - ret = getMS().getPartitions(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, - max_parts, skipColumnSchemaForPartition); + ret = getMS().getPartitions(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, args); ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret); } catch (Exception e) { ex = e; @@ -5525,8 +5514,13 @@ private List get_partitions(final String db_name, final String tbl_na public PartitionsResponse get_partitions_req(PartitionsRequest req) throws NoSuchObjectException, MetaException, TException { String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf); - List partitions = get_partitions(dbName, req.getTblName(), req.getMaxParts(), - req.isSkipColumnSchemaForPartition()); + List partitions = get_partitions(dbName, req.getTblName(), + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .max(req.getMaxParts()) + .includeParamKeyPattern(req.getIncludeParamKeyPattern()) + .excludeParamKeyPattern(req.getExcludeParamKeyPattern()) + .skipColumnSchemaForPartition(req.isSkipColumnSchemaForPartition()) + .build()); PartitionsResponse res = new PartitionsResponse(); res.setPartitions(partitions); return res; @@ -5537,36 +5531,10 @@ public PartitionsResponse get_partitions_req(PartitionsRequest req) public List get_partitions_with_auth(final String dbName, final String tblName, final short maxParts, final String userName, final List groupNames) throws TException { - return get_partitions_with_auth_optional_schema(dbName, tblName, maxParts, userName, groupNames , false); - - } - - private List get_partitions_with_auth_optional_schema(final String dbName, - final String tblName, final short maxParts, final String userName, - final List groupNames, boolean skipColSchemaForPartitions) throws TException { - String[] parsedDbName = parseDbName(dbName, conf); - startTableFunction("get_partitions_with_auth", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName); - - List ret = null; - Exception ex = null; - try { - checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], - tblName, NO_FILTER_STRING, maxParts); - - authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName); - - ret = getMS().getPartitionsWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName, - maxParts, userName, groupNames, skipColSchemaForPartitions); - ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret); - } catch (Exception e) { - ex = e; - handleException(e).convertIfInstance(InvalidObjectException.class, NoSuchObjectException.class) - .rethrowException(e); - } finally { - endFunction("get_partitions_with_auth", ret != null, ex, tblName); - } - return ret; - + return get_partitions_ps_with_auth(dbName, tblName, + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .max(maxParts).userName(userName).groupNames(groupNames) + .build()); } private void checkLimitNumberOfPartitionsByFilter(String catName, String dbName, @@ -6601,8 +6569,9 @@ public List get_partitions_ps(final String db_name, try { authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); // Don't send the parsedDbName, as this method will parse itself. - ret = get_partitions_ps_with_auth(db_name, tbl_name, part_vals, - max_parts, null, null); + ret = get_partitions_ps_with_auth(db_name, tbl_name, new GetPartitionsArgs.GetPartitionsArgsBuilder() + .part_vals(part_vals).max(max_parts) + .build()); ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret); } catch (Exception e) { ex = e; @@ -6624,25 +6593,30 @@ public List get_partitions_ps_with_auth(final String db_name, final String tbl_name, final List part_vals, final short max_parts, final String userName, final List groupNames) throws TException { - return get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, userName, groupNames, false); + return get_partitions_ps_with_auth(db_name, tbl_name, new GetPartitionsArgs.GetPartitionsArgsBuilder() + .part_vals(part_vals).max(max_parts).userName(userName).groupNames(groupNames) + .build()); } private List get_partitions_ps_with_auth(final String db_name, - final String tbl_name, final List part_vals, - final short max_parts, final String userName, - final List groupNames, boolean skipColSchemaForPartitions) throws TException { + final String tbl_name, GetPartitionsArgs args) throws TException { String[] parsedDbName = parseDbName(db_name, conf); startPartitionFunction("get_partitions_ps_with_auth", parsedDbName[CAT_NAME], - parsedDbName[DB_NAME], tbl_name, part_vals); + parsedDbName[DB_NAME], tbl_name, args.getPart_vals()); fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); List ret = null; Exception ex = null; try { - checkLimitNumberOfPartitionsByPs(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], - tbl_name, part_vals, max_parts); + if (args.getPart_vals() != null) { + checkLimitNumberOfPartitionsByPs(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], + tbl_name, args.getPart_vals(), args.getMax()); + } else { + checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], + tbl_name, NO_FILTER_STRING, args.getMax()); + } authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); ret = getMS().listPartitionsPsWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], - tbl_name, part_vals, max_parts, userName, groupNames, skipColSchemaForPartitions); + tbl_name, args); ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret); } catch (Exception e) { ex = e; @@ -6657,15 +6631,14 @@ private List get_partitions_ps_with_auth(final String db_name, public GetPartitionsPsWithAuthResponse get_partitions_ps_with_auth_req(GetPartitionsPsWithAuthRequest req) throws MetaException, NoSuchObjectException, TException { String dbName = MetaStoreUtils.prependCatalogToDbName(req.getCatName(), req.getDbName(), conf); - List partitions = null; - if (req.getPartVals() == null) { - partitions = get_partitions_with_auth_optional_schema(dbName, req.getTblName(), req.getMaxParts(), req.getUserName(), - req.getGroupNames(), req.isSkipColumnSchemaForPartition()); - } else { - partitions = - get_partitions_ps_with_auth(dbName, req.getTblName(), req.getPartVals(), req.getMaxParts(), - req.getUserName(), req.getGroupNames(), req.isSkipColumnSchemaForPartition()); - } + List partitions = + get_partitions_ps_with_auth(dbName, req.getTblName(), new GetPartitionsArgs.GetPartitionsArgsBuilder() + .part_vals(req.getPartVals()).max(req.getMaxParts()) + .userName(req.getUserName()).groupNames(req.getGroupNames()) + .skipColumnSchemaForPartition(req.isSkipColumnSchemaForPartition()) + .includeParamKeyPattern(req.getIncludeParamKeyPattern()) + .excludeParamKeyPattern(req.getExcludeParamKeyPattern()) + .build()); GetPartitionsPsWithAuthResponse res = new GetPartitionsPsWithAuthResponse(); res.setPartitions(partitions); return res; @@ -7245,12 +7218,12 @@ public List get_partitions_by_filter(final String dbName, final Strin final String filter, final short maxParts) throws TException { String[] parsedDbName = parseDbName(dbName, conf); - return get_partitions_by_filter_internal(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName, filter, maxParts, false); + return get_partitions_by_filter_internal(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName, + new GetPartitionsArgs.GetPartitionsArgsBuilder().filter(filter).max(maxParts).build()); } private List get_partitions_by_filter_internal(final String catName, - final String dbName, final String tblName, final String filter, final short maxParts, - boolean skipColSchemaForPartitions) throws TException { + final String dbName, final String tblName, GetPartitionsArgs args) throws TException { startTableFunction("get_partitions_by_filter", catName, dbName, tblName); fireReadTablePreEvent(catName, dbName, tblName); @@ -7258,12 +7231,11 @@ private List get_partitions_by_filter_internal(final String catName, Exception ex = null; try { checkLimitNumberOfPartitionsByFilter(catName, dbName, - tblName, filter, maxParts); + tblName, args.getFilter(), args.getMax()); authorizeTableForPartitionMetadata(catName, dbName, tblName); - ret = getMS().getPartitionsByFilter(catName, dbName, tblName, - filter, maxParts, skipColSchemaForPartitions); + ret = getMS().getPartitionsByFilter(catName, dbName, tblName, args); ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret); } catch (Exception e) { ex = e; @@ -7275,8 +7247,13 @@ private List get_partitions_by_filter_internal(final String catName, } public List get_partitions_by_filter_req(GetPartitionsByFilterRequest req) throws TException { - return get_partitions_by_filter_internal(req.getCatName(), req.getDbName(), req.getTblName(), req.getFilter(), - req.getMaxParts(), req.isSkipColumnSchemaForPartition()); + return get_partitions_by_filter_internal(req.getCatName(), req.getDbName(), req.getTblName(), + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .filter(req.getFilter()).max(req.getMaxParts()) + .skipColumnSchemaForPartition(req.isSkipColumnSchemaForPartition()) + .excludeParamKeyPattern(req.getExcludeParamKeyPattern()) + .includeParamKeyPattern(req.getIncludeParamKeyPattern()) + .build()); } @Override @@ -7327,8 +7304,13 @@ public PartitionsSpecByExprResult get_partitions_spec_by_expr( try { checkLimitNumberOfPartitionsByExpr(catName, dbName, tblName, req.getExpr(), UNLIMITED_MAX_PARTITIONS); List partitions = new LinkedList<>(); - boolean hasUnknownPartitions = getMS().getPartitionsByExpr(catName, dbName, tblName, - req.getExpr(), req.getDefaultPartitionName(), req.getMaxParts(), partitions); + boolean hasUnknownPartitions = getMS().getPartitionsByExpr(catName, dbName, tblName, partitions, + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .expr(req.getExpr()).max(req.getMaxParts()).defaultPartName(req.getDefaultPartitionName()) + .skipColumnSchemaForPartition(req.isSkipColumnSchemaForPartition()) + .includeParamKeyPattern(req.getIncludeParamKeyPattern()) + .excludeParamKeyPattern(req.getExcludeParamKeyPattern()) + .build()); Table table = get_table_core(catName, dbName, tblName); List partitionSpecs = MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(table, partitions); @@ -7354,9 +7336,13 @@ public PartitionsByExprResult get_partitions_by_expr( try { checkLimitNumberOfPartitionsByExpr(catName, dbName, tblName, req.getExpr(), UNLIMITED_MAX_PARTITIONS); List partitions = new LinkedList<>(); - boolean hasUnknownPartitions = getMS().getPartitionsByExpr(catName, dbName, tblName, - req.getExpr(), req.getDefaultPartitionName(), req.getMaxParts(), partitions, - req.isSkipColumnSchemaForPartition()); + boolean hasUnknownPartitions = getMS().getPartitionsByExpr(catName, dbName, tblName, partitions, + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .expr(req.getExpr()).defaultPartName(req.getDefaultPartitionName()).max(req.getMaxParts()) + .skipColumnSchemaForPartition(req.isSkipColumnSchemaForPartition()) + .excludeParamKeyPattern(req.getExcludeParamKeyPattern()) + .includeParamKeyPattern(req.getIncludeParamKeyPattern()) + .build()); ret = new PartitionsByExprResult(partitions, hasUnknownPartitions); } catch (Exception e) { ex = e; @@ -7434,31 +7420,30 @@ private int getNumPartitionsByPs(final String catName, final String dbName, public List get_partitions_by_names(final String dbName, final String tblName, final List partNames) throws TException { - return get_partitions_by_names(dbName, tblName, partNames, false, null, null, false); + return get_partitions_by_names(dbName, tblName, false, null, null, null, + new GetPartitionsArgs.GetPartitionsArgsBuilder().partNames(partNames).build()); } @Override public GetPartitionsByNamesResult get_partitions_by_names_req(GetPartitionsByNamesRequest gpbnr) throws TException { List partitions = get_partitions_by_names(gpbnr.getDb_name(), - gpbnr.getTbl_name(), gpbnr.getNames(), + gpbnr.getTbl_name(), gpbnr.isSetGet_col_stats() && gpbnr.isGet_col_stats(), gpbnr.getEngine(), - gpbnr.getProcessorCapabilities(), gpbnr.getProcessorIdentifier(), gpbnr.isSkipColumnSchemaForPartition()); + gpbnr.getProcessorCapabilities(), gpbnr.getProcessorIdentifier(), + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .partNames(gpbnr.getNames()).skipColumnSchemaForPartition(gpbnr.isSkipColumnSchemaForPartition()) + .excludeParamKeyPattern(gpbnr.getExcludeParamKeyPattern()) + .includeParamKeyPattern(gpbnr.getIncludeParamKeyPattern()) + .build()); GetPartitionsByNamesResult result = new GetPartitionsByNamesResult(partitions); return result; } public List get_partitions_by_names(final String dbName, final String tblName, - final List partNames, boolean getColStats, String engine, String validWriteIdList, - boolean skipColSchemaForPartitions) throws TException { - return get_partitions_by_names( - dbName, tblName, partNames, getColStats, engine, null, null, skipColSchemaForPartitions); - } - - public List get_partitions_by_names(final String dbName, final String tblName, - final List partNames, boolean getColStats, String engine, + boolean getColStats, String engine, List processorCapabilities, String processorId, - boolean skipColSchemaForPartitions) throws TException { + GetPartitionsArgs args) throws TException { String[] dbNameParts = parseDbName(dbName, conf); String parsedCatName = dbNameParts[CAT_NAME]; @@ -7475,7 +7460,7 @@ public List get_partitions_by_names(final String dbName, final String fireReadTablePreEvent(parsedCatName, parsedDbName, tblName); - ret = getMS().getPartitionsByNames(parsedCatName, parsedDbName, tblName, partNames, skipColSchemaForPartitions); + ret = getMS().getPartitionsByNames(parsedCatName, parsedDbName, tblName, args); ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret); table = getTable(parsedCatName, parsedDbName, tblName); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java index 6f04fd03720d..97956660791a 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java @@ -89,6 +89,7 @@ import org.apache.hadoop.hive.metastore.api.SkewedInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; import org.apache.hadoop.hive.metastore.model.MConstraint; @@ -388,7 +389,7 @@ public Database getDatabase(String catName, String dbName) throws MetaException{ String queryTextDbSelector= "select " + "\"DB_ID\", \"NAME\", \"DB_LOCATION_URI\", \"DESC\", " + "\"OWNER_NAME\", \"OWNER_TYPE\", \"CTLG_NAME\" , \"CREATE_TIME\", \"DB_MANAGED_LOCATION_URI\", " - + "\"TYPE\", \"DATACONNECTOR_NAME\", \"REMOTE_DBNAME\"" + + "\"TYPE\", \"DATACONNECTOR_NAME\", \"REMOTE_DBNAME\" " + "FROM "+ DBS + " where \"NAME\" = ? and \"CTLG_NAME\" = ? "; String queryTextDbParams = "select \"PARAM_KEY\", \"PARAM_VALUE\" " @@ -661,13 +662,13 @@ public List getPartitionNamesViaSql(SqlFilterForPushdown filter, List getPartitionsViaSqlFilter(final String catName, final String dbName, - final String tblName, List partNames, boolean skipColSchemaForPartitions) + public List getPartitionsViaPartNames(final String catName, final String dbName, + final String tblName, GetPartitionsArgs args) throws MetaException { + List partNames = args.getPartNames(); if (partNames.isEmpty()) { return Collections.emptyList(); } @@ -681,7 +682,7 @@ public List run(List input) throws MetaException { return Collections.emptyList(); // no partitions, bail early. } return getPartitionsFromPartitionIds(catName, dbName, tblName, null, - partitionIds, Collections.emptyList(), skipColSchemaForPartitions); + partitionIds, Collections.emptyList(), false, args); } }); } @@ -689,17 +690,16 @@ public List run(List input) throws MetaException { /** * Gets partitions by using direct SQL queries. * @param filter The filter. - * @param max The maximum number of partitions to return. * @param isAcidTable True if the table is ACID - * @param skipColSchemaForPartitions skip column schema for partitions + * @param args additional arguments for getting partitions * @return List of partitions. */ public List getPartitionsViaSqlFilter(String catName, String dbName, String tableName, - SqlFilterForPushdown filter, Integer max, boolean isAcidTable, - boolean skipColSchemaForPartitions) throws MetaException { + SqlFilterForPushdown filter, boolean isAcidTable, + GetPartitionsArgs args) throws MetaException { List partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, tableName, filter.filter, filter.params, - filter.joins, max); + filter.joins, args.getMax()); if (partitionIds.isEmpty()) { return Collections.emptyList(); // no partitions, bail early. } @@ -707,7 +707,7 @@ public List getPartitionsViaSqlFilter(String catName, String dbName, @Override public List run(List input) throws MetaException { return getPartitionsFromPartitionIds(catName, dbName, - tableName, null, input, Collections.emptyList(), isAcidTable, skipColSchemaForPartitions); + tableName, null, input, Collections.emptyList(), isAcidTable, args); } }); } @@ -846,14 +846,13 @@ public boolean generateSqlFilterForPushdown(String catName, String dbName, Strin * @param catName Metastore catalog name. * @param dbName Metastore db name. * @param tblName Metastore table name. - * @param max The maximum number of partitions to return. - * @param skipColumnSchemaForPartition skip column schema for partitions + * @param args additional arguments for getting partitions * @return List of partitions. */ public List getPartitions(String catName, - String dbName, String tblName, Integer max, boolean skipColumnSchemaForPartition) throws MetaException { + String dbName, String tblName, GetPartitionsArgs args) throws MetaException { List partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, - tblName, null, Collections.emptyList(), Collections.emptyList(), max); + tblName, null, Collections.emptyList(), Collections.emptyList(), args.getMax()); if (partitionIds.isEmpty()) { return Collections.emptyList(); // no partitions, bail early. } @@ -862,8 +861,7 @@ public List getPartitions(String catName, List result = Batchable.runBatched(batchSize, partitionIds, new Batchable() { @Override public List run(List input) throws MetaException { - return getPartitionsFromPartitionIds(catName, dbName, tblName, null, input, Collections.emptyList(), - skipColumnSchemaForPartition); + return getPartitionsFromPartitionIds(catName, dbName, tblName, null, input, Collections.emptyList(), false, args); } }); return result; @@ -947,17 +945,10 @@ private List getPartitionIdsViaSqlFilter( } } - /** Should be called with the list short enough to not trip up Oracle/etc. */ - private List getPartitionsFromPartitionIds(String catName, String dbName, String tblName, - Boolean isView, List partIdList, List projectionFields, boolean skipColumnSchemaForPartition) throws MetaException { - return getPartitionsFromPartitionIds(catName, dbName, tblName, isView, partIdList, - projectionFields, false, skipColumnSchemaForPartition); - } - /** Should be called with the list short enough to not trip up Oracle/etc. */ private List getPartitionsFromPartitionIds(String catName, String dbName, String tblName, Boolean isView, List partIdList, List projectionFields, - boolean isAcidTable, boolean skipColumnSchemaForPartition) throws MetaException { + boolean isAcidTable, GetPartitionsArgs args) throws MetaException { boolean doTrace = LOG.isDebugEnabled(); @@ -1093,7 +1084,8 @@ private List getPartitionsFromPartitionIds(String catName, String dbN } // Now get all the one-to-many things. Start with partitions. MetastoreDirectSqlUtils - .setPartitionParameters(PARTITION_PARAMS, convertMapNullsToEmptyStrings, pm, partIds, partitions); + .setPartitionParametersWithFilter(PARTITION_PARAMS, convertMapNullsToEmptyStrings, pm, + partIds, partitions, args.getIncludeParamKeyPattern(), args.getExcludeParamKeyPattern()); MetastoreDirectSqlUtils.setPartitionValues(PARTITION_KEY_VALS, pm, partIds, partitions); @@ -1136,7 +1128,7 @@ private List getPartitionsFromPartitionIds(String catName, String dbN } // if (hasSkewedColumns) // Get FieldSchema stuff if any. - if (!colss.isEmpty() && !skipColumnSchemaForPartition) { + if (!colss.isEmpty() && !args.isSkipColumnSchemaForPartition()) { // We are skipping the CDS table here, as it seems to be totally useless. MetastoreDirectSqlUtils.setSDCols(COLUMNS_V2, pm, colss, colIds); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java index b30999773128..067e415d7253 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java @@ -167,24 +167,6 @@ static int loopJoinOrderedResult(PersistenceManager pm, TreeMap tre return rv; } - static void setPartitionParameters(String PARTITION_PARAMS, boolean convertMapNullsToEmptyStrings, - PersistenceManager pm, String partIds, TreeMap partitions) - throws MetaException { - String queryText; - queryText = "select \"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + PARTITION_PARAMS + "" - + " where \"PART_ID\" in (" + partIds + ") and \"PARAM_KEY\" is not null" - + " order by \"PART_ID\" asc"; - loopJoinOrderedResult(pm, partitions, queryText, 0, new ApplyFunc() { - @Override - public void apply(Partition t, Object[] fields) { - t.putToParameters(extractSqlClob(fields[1]), extractSqlClob(fields[2])); - }}); - // Perform conversion of null map values - for (Partition t : partitions.values()) { - t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings)); - } - } - static void setPartitionParametersWithFilter(String PARTITION_PARAMS, boolean convertMapNullsToEmptyStrings, PersistenceManager pm, String partIds, TreeMap partitions, String includeParamKeyPattern, String excludeParamKeyPattern) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 05a421e93fa8..fa132629934b 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -58,6 +58,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.regex.Pattern; +import java.util.stream.Collectors; import javax.jdo.JDODataStoreException; import javax.jdo.JDOException; @@ -192,6 +193,7 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger; import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse; import org.apache.hadoop.hive.metastore.api.WriteEventInfo; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; import org.apache.hadoop.hive.metastore.metrics.Metrics; @@ -2262,9 +2264,38 @@ private List convertList(List dnList) { } /** Makes shallow copy of a map to avoid DataNucleus mucking with our objects. */ - private Map convertMap(Map dnMap) { - return MetaStoreServerUtils.trimMapNulls(dnMap, + private Map convertMap(Map dnMap, GetPartitionsArgs... args) { + Map parameters = MetaStoreServerUtils.trimMapNulls(dnMap, MetastoreConf.getBoolVar(getConf(), ConfVars.ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS)); + if (parameters != null && args != null && args.length == 1) { + // Pattern matching in Java might be different from the one used by the metastore backends, + // An underscore (_) in pattern stands for (matches) any single character; + // a percent sign (%) matches any sequence of zero or more characters. + // See TestGetPartitionsUsingProjectionAndFilterSpecs#testPartitionProjectionEmptySpec. + Pattern includePatt = null; + if (StringUtils.isNotBlank(args[0].getIncludeParamKeyPattern())) { + includePatt = Optional.of(args[0].getIncludeParamKeyPattern()).map(regex -> + Pattern.compile(regex.replaceAll("%", ".*").replaceAll("_", "."))).get(); + } + Pattern excludePatt = null; + if (StringUtils.isNotBlank(args[0].getExcludeParamKeyPattern())) { + excludePatt = Optional.of(args[0].getExcludeParamKeyPattern()).map(regex -> + Pattern.compile(regex.replaceAll("%", ".*").replaceAll("_", "."))).get();; + } + final Pattern includePattern = includePatt; + final Pattern excludePattern = excludePatt; + return parameters.entrySet().stream().filter(entry -> { + boolean matches = true; + if (includePattern != null) { + matches &= includePattern.matcher(entry.getKey()).matches(); + } + if (excludePattern != null) { + matches &= !excludePattern.matcher(entry.getKey()).matches(); + } + return matches; + }).collect(Collectors.toMap(x -> x.getKey(), x -> x.getValue())); + } + return parameters; } private Table convertToTable(MTable mtbl) throws MetaException { @@ -2866,7 +2897,8 @@ public Partition getPartition(String catName, String dbName, String tableName, " does not exist"); } MPartition mpart = getMPartition(catName, dbName, tableName, part_vals, table); - part = convertToPart(mpart, false); + part = convertToPart(catName, dbName, tableName, mpart, + TxnUtils.isAcidTable(table.getParameters())); committed = commitTransaction(); if (part == null) { throw new NoSuchObjectException("partition values=" @@ -3031,45 +3063,20 @@ private MPartition convertToMPart(Partition part, MTable mt, boolean useTableCD) msd, part.getParameters()); } - private Partition convertToPart(MPartition mpart, boolean isAcidTable) throws MetaException { - return convertToPart(mpart, isAcidTable, false); - } - - private Partition convertToPart(MPartition mpart, boolean isAcidTable, - boolean skipColSchemaForPartitions) throws MetaException { - if (mpart == null) { - return null; - } - //its possible that MPartition is partially filled, do null checks to avoid NPE - MTable table = mpart.getTable(); - String dbName = - table == null ? null : table.getDatabase() == null ? null : table.getDatabase().getName(); - String tableName = table == null ? null : table.getTableName(); - String catName = table == null ? null : - table.getDatabase() == null ? null : table.getDatabase().getCatalogName(); - Map params = convertMap(mpart.getParameters()); - Partition p = new Partition(convertList(mpart.getValues()), dbName, tableName, mpart.getCreateTime(), - mpart.getLastAccessTime(), convertToStorageDescriptor(mpart.getSd(), skipColSchemaForPartitions, isAcidTable), - params); - p.setCatName(catName); - if(mpart.getWriteId()>0) { - p.setWriteId(mpart.getWriteId()); - }else { - p.setWriteId(-1L); - } - return p; - } - private Partition convertToPart(String catName, String dbName, String tblName, - MPartition mpart, boolean isAcidTable, boolean skipColSchemaForPartitions) + MPartition mpart, boolean isAcidTable, GetPartitionsArgs... args) throws MetaException { if (mpart == null) { return null; } - Map params = convertMap(mpart.getParameters()); + catName = normalizeIdentifier(catName); + dbName = normalizeIdentifier(dbName); + tblName = normalizeIdentifier(tblName); + Map params = convertMap(mpart.getParameters(), args); + boolean noFS = args != null && args.length == 1 ? args[0].isSkipColumnSchemaForPartition() : false; Partition p = new Partition(convertList(mpart.getValues()), dbName, tblName, mpart.getCreateTime(), mpart.getLastAccessTime(), - convertToStorageDescriptor(mpart.getSd(), skipColSchemaForPartitions, isAcidTable), params); + convertToStorageDescriptor(mpart.getSd(), noFS, isAcidTable), params); p.setCatName(catName); if(mpart.getWriteId()>0) { p.setWriteId(mpart.getWriteId()); @@ -3238,7 +3245,7 @@ private boolean dropPartitionCommon(MPartition part) throws MetaException, @Override public List getPartitions(String catName, String dbName, String tableName, - int maxParts, boolean skipColumnSchemaForPartition) throws MetaException, NoSuchObjectException { + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { List results = Collections.emptyList(); boolean success = false; @@ -3246,7 +3253,7 @@ public List getPartitions(String catName, String dbName, String table try { openTransaction(); - results = getPartitionsInternal(catName, dbName, tableName, maxParts, true, true, skipColumnSchemaForPartition); + results = getPartitionsInternal(catName, dbName, tableName, true, true, args); success = commitTransaction(); } finally { if (!success) { @@ -3297,23 +3304,19 @@ public Map getPartitionLocations(String catName, String dbName, return partLocations; } - protected List getPartitionsInternal(String catName, String dbName, String tblName, final int maxParts, - boolean allowSql, boolean allowJdo) throws MetaException, NoSuchObjectException { - return getPartitionsInternal(catName, dbName, tblName, maxParts, allowSql, allowJdo, false); - } - - private List getPartitionsInternal(String catName, String dbName, String tblName, final int maxParts, - boolean allowSql, boolean allowJdo, boolean skipColumnSchemaForPartition) + protected List getPartitionsInternal(String catName, String dbName, String tblName, + boolean allowSql, boolean allowJdo, GetPartitionsArgs args) throws MetaException, NoSuchObjectException { return new GetListHelper(catName, dbName, tblName, allowSql, allowJdo) { @Override protected List getSqlResult(GetHelper> ctx) throws MetaException { - return directSql.getPartitions(catName, dbName, tblName, maxParts, skipColumnSchemaForPartition); + return directSql.getPartitions(catName, dbName, tblName, args); } @Override protected List getJdoResult(GetHelper> ctx) throws MetaException { try { - return convertToParts(listMPartitions(catName, dbName, tblName, maxParts), skipColumnSchemaForPartition); + return convertToParts(catName, dbName, tblName, + listMPartitions(catName, dbName, tblName, args.getMax()), false, args); } catch (Exception e) { LOG.error("Failed to convert to parts", e); throw new MetaException(e.getMessage()); @@ -3322,40 +3325,6 @@ protected List getJdoResult(GetHelper> ctx) throws Me }.run(false); } - @Override - public List getPartitionsWithAuth(String catName, String dbName, String tblName, - short max, String userName, List groupNames, boolean skipColumnSchemaForPartition) - throws MetaException, InvalidObjectException { - boolean success = false; - - try { - openTransaction(); - List mparts = listMPartitions(catName, dbName, tblName, max); - List parts = new ArrayList<>(mparts.size()); - if (CollectionUtils.isNotEmpty(mparts)) { - for (MPartition mpart : mparts) { - MTable mtbl = mpart.getTable(); - Partition part = convertToPart(mpart, false, skipColumnSchemaForPartition); - parts.add(part); - - if ("TRUE".equalsIgnoreCase(mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) { - String partName = Warehouse.makePartName(this.convertToFieldSchemas(mtbl - .getPartitionKeys()), part.getValues()); - PrincipalPrivilegeSet partAuth = this.getPartitionPrivilegeSet(catName, dbName, - tblName, partName, userName, groupNames); - part.setPrivileges(partAuth); - } - } - } - success = commitTransaction(); - return parts; - } catch (Exception e) { - throw new MetaException(e.getMessage()); - } finally { - rollbackAndCleanup(success, null); - } - } - @Override public Partition getPartitionWithAuth(String catName, String dbName, String tblName, List partVals, String user_name, List group_names) @@ -3370,7 +3339,8 @@ public Partition getPartitionWithAuth(String catName, String dbName, String tblN + partVals.toString()); } MTable mtbl = mpart.getTable(); - Partition part = convertToPart(mpart, false); + + Partition part = convertToPart(catName, dbName, tblName, mpart, TxnUtils.isAcidTable(mtbl.getParameters())); if ("TRUE".equalsIgnoreCase(mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) { String partName = Warehouse.makePartName(this.convertToFieldSchemas(mtbl .getPartitionKeys()), partVals); @@ -3388,32 +3358,12 @@ public Partition getPartitionWithAuth(String catName, String dbName, String tblN } } - private List convertToParts(List mparts, - boolean skipColumnSchemaForPartition) throws MetaException { - return convertToParts(mparts, null, skipColumnSchemaForPartition); - } - - private List convertToParts(List src, List dest, - boolean skipColumnSchemaForPartition) throws MetaException { - if (src == null) { - return dest; - } - if (dest == null) { - dest = new ArrayList<>(src.size()); - } - for (MPartition mp : src) { - dest.add(convertToPart(mp, false, skipColumnSchemaForPartition)); - Deadline.checkTimeout(); - } - return dest; - } - private List convertToParts(String catName, String dbName, String tblName, - List mparts, boolean isAcidTable, boolean skipColumnSchemaForPartition) + List mparts, boolean isAcidTable, GetPartitionsArgs args) throws MetaException { List parts = new ArrayList<>(mparts.size()); for (MPartition mp : mparts) { - parts.add(convertToPart(catName, dbName, tblName, mp, isAcidTable, skipColumnSchemaForPartition)); + parts.add(convertToPart(catName, dbName, tblName, mp, isAcidTable, args)); Deadline.checkTimeout(); } return parts; @@ -3862,7 +3812,7 @@ public int getNumPartitionsByPs(String catName, String dbName, String tblName, L * has types of String, and if resultsCol is null, the types are MPartition. */ private Collection getPartitionPsQueryResults(String catName, String dbName, String tableName, List part_vals, - short max_parts, String resultsCol) throws Exception { + int max_parts, String resultsCol) throws Exception { Preconditions.checkState(this.currentTransaction.isActive()); @@ -3904,8 +3854,8 @@ private Collection getPartitionPsQueryResults(String catName, String dbN * doesn't support partition privileges. */ private boolean canTryDirectSQL(List partVals) { - if (partVals.isEmpty()) { - return false; + if (partVals == null || partVals.isEmpty()) { + return true; } for (String val : partVals) { if (val != null && !val.isEmpty()) { @@ -3917,42 +3867,48 @@ private boolean canTryDirectSQL(List partVals) { @Override public List listPartitionsPsWithAuth(String catName, String db_name, String tbl_name, - List part_vals, short max_parts, String userName, List groupNames, - boolean skipColSchemaForPartitions) throws MetaException, InvalidObjectException, NoSuchObjectException { + GetPartitionsArgs args) throws MetaException, InvalidObjectException, NoSuchObjectException { List partitions = new ArrayList<>(); boolean success = false; try { openTransaction(); - + LOG.debug("executing listPartitionNamesPsWithAuth"); MTable mtbl = getMTable(catName, db_name, tbl_name); if (mtbl == null) { throw new NoSuchObjectException( TableName.getQualified(catName, db_name, tbl_name) + " table not found"); } + int max_parts = args.getMax(); + String userName = args.getUserName(); + List groupNames = args.getGroupNames(); + List part_vals = args.getPart_vals(); boolean getauth = null != userName && null != groupNames && "TRUE".equalsIgnoreCase( mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE")); - if (!getauth && canTryDirectSQL(part_vals)) { + + if (canTryDirectSQL(part_vals)) { LOG.info( "Redirecting to directSQL enabled API: db: {} tbl: {} partVals: {}", - db_name, tbl_name, Joiner.on(',').join(part_vals)); - return getPartitions(catName, db_name, tbl_name, max_parts, skipColSchemaForPartitions); + db_name, tbl_name, part_vals); + partitions = getPartitions(catName, db_name, tbl_name, args); + } else { + Collection parts = getPartitionPsQueryResults(catName, db_name, tbl_name, + part_vals, max_parts, null); + boolean isAcidTable = TxnUtils.isAcidTable(mtbl.getParameters()); + for (Object o : parts) { + Partition part = convertToPart(catName, db_name, tbl_name, (MPartition) o, isAcidTable, args); + partitions.add(part); + } } - LOG.debug("executing listPartitionNamesPsWithAuth"); - Collection parts = getPartitionPsQueryResults(catName, db_name, tbl_name, - part_vals, max_parts, null); - for (Object o : parts) { - Partition part = convertToPart((MPartition) o, false, skipColSchemaForPartitions); - //set auth privileges - if (getauth) { + if (getauth) { + for (Partition part : partitions) { String partName = Warehouse.makePartName(this.convertToFieldSchemas(mtbl .getPartitionKeys()), part.getValues()); PrincipalPrivilegeSet partAuth = getPartitionPrivilegeSet(catName, db_name, tbl_name, partName, userName, groupNames); part.setPrivileges(partAuth); } - partitions.add(part); } success = commitTransaction(); } catch (InvalidObjectException | NoSuchObjectException | MetaException e) { @@ -4074,41 +4030,39 @@ private List listMPartitionsWithProjection(List fieldNames, @Override public List getPartitionsByNames(String catName, String dbName, String tblName, - List partNames, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { - return getPartitionsByNamesInternal(catName, dbName, tblName, partNames, true, true, skipColSchemaForPartitions); + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { + return getPartitionsByNamesInternal(catName, dbName, tblName, true, true, args); } protected List getPartitionsByNamesInternal(String catName, String dbName, - String tblName, final List partNames, boolean allowSql, boolean allowJdo, - boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { + String tblName, boolean allowSql, boolean allowJdo, + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { return new GetListHelper(catName, dbName, tblName, allowSql, allowJdo) { @Override protected List getSqlResult(GetHelper> ctx) throws MetaException { - return directSql.getPartitionsViaSqlFilter(catName, dbName, tblName, partNames, skipColSchemaForPartitions); + return directSql.getPartitionsViaPartNames(catName, dbName, tblName, args); } @Override protected List getJdoResult( GetHelper> ctx) throws MetaException, NoSuchObjectException { - return getPartitionsViaOrmFilter(catName, dbName, tblName, partNames, false, skipColSchemaForPartitions); + return getPartitionsViaOrmFilter(catName, dbName, tblName, false, args); } }.run(false); } @Override - public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, - String defaultPartitionName, short maxParts, List result, - boolean skipColSchemaForPartitions) throws TException { - return getPartitionsByExprInternal(catName, dbName, tblName, expr, defaultPartitionName, maxParts, - result, true, true, skipColSchemaForPartitions); + public boolean getPartitionsByExpr(String catName, String dbName, String tblName, + List result, GetPartitionsArgs args) throws TException { + return getPartitionsByExprInternal(catName, dbName, tblName, result, true, true, args); } - protected boolean getPartitionsByExprInternal(String catName, String dbName, String tblName, final byte[] expr, - final String defaultPartitionName, final short maxParts, List result, - boolean allowSql, boolean allowJdo, boolean skipColSchemaForPartitions) throws TException { + protected boolean getPartitionsByExprInternal(String catName, String dbName, String tblName, + List result, boolean allowSql, boolean allowJdo, GetPartitionsArgs args) throws TException { assert result != null; + byte[] expr = args.getExpr(); final ExpressionTree exprTree = expr.length != 0 ? PartFilterExprUtil.makeExpressionTree( - expressionProxy, expr, getDefaultPartitionName(defaultPartitionName), conf) : ExpressionTree.EMPTY_TREE; + expressionProxy, expr, getDefaultPartitionName(args.getDefaultPartName()), conf) : ExpressionTree.EMPTY_TREE; final AtomicBoolean hasUnknownPartitions = new AtomicBoolean(false); catName = normalizeIdentifier(catName); @@ -4125,17 +4079,18 @@ protected List getSqlResult(GetHelper> ctx) throws Me if (exprTree != null) { SqlFilterForPushdown filter = new SqlFilterForPushdown(); if (directSql.generateSqlFilterForPushdown(catName, dbName, tblName, partitionKeys, - exprTree, defaultPartitionName, filter)) { + exprTree, args.getDefaultPartName(), filter)) { String catalogName = (catName != null) ? catName : getDefaultCatalog(conf); - return directSql.getPartitionsViaSqlFilter(catalogName, dbName, tblName, filter, null, - isAcidTable, skipColSchemaForPartitions); + return directSql.getPartitionsViaSqlFilter(catalogName, dbName, tblName, filter, + isAcidTable, args); } } // We couldn't do SQL filter pushdown. Get names via normal means. List partNames = new LinkedList<>(); hasUnknownPartitions.set(getPartitionNamesPrunedByExprNoTxn( - catName, dbName, tblName, partitionKeys, expr, defaultPartitionName, maxParts, partNames)); - return directSql.getPartitionsViaSqlFilter(catName, dbName, tblName, partNames, skipColSchemaForPartitions); + catName, dbName, tblName, partitionKeys, expr, args.getDefaultPartName(), (short) args.getMax(), partNames)); + GetPartitionsArgs newArgs = new GetPartitionsArgs.GetPartitionsArgsBuilder(args).partNames(partNames).build(); + return directSql.getPartitionsViaPartNames(catName, dbName, tblName, newArgs); } @Override @@ -4145,15 +4100,15 @@ protected List getJdoResult( List result = null; if (exprTree != null) { result = getPartitionsViaOrmFilter(catName, dbName, tblName, exprTree, - maxParts, false, partitionKeys, skipColSchemaForPartitions); + false, partitionKeys, isAcidTable, args); } if (result == null) { // We couldn't do JDOQL filter pushdown. Get names via normal means. List partNames = new ArrayList<>(); hasUnknownPartitions.set(getPartitionNamesPrunedByExprNoTxn( - catName, dbName, tblName, partitionKeys, expr, defaultPartitionName, maxParts, partNames)); - result = getPartitionsViaOrmFilter(catName, dbName, tblName, partNames, - isAcidTable, skipColSchemaForPartitions); + catName, dbName, tblName, partitionKeys, expr, args.getDefaultPartName(), (short) args.getMax(), partNames)); + GetPartitionsArgs newArgs = new GetPartitionsArgs.GetPartitionsArgsBuilder(args).partNames(partNames).build(); + result = getPartitionsViaOrmFilter(catName, dbName, tblName, isAcidTable, newArgs); } return result; } @@ -4201,16 +4156,15 @@ private boolean getPartitionNamesPrunedByExprNoTxn(String catName, String dbName * Gets partition names from the table via ORM (JDOQL) filter pushdown. * @param tblName The table. * @param tree The expression tree from which JDOQL filter will be made. - * @param maxParts Maximum number of partitions to return. * @param isValidatedFilter Whether the filter was pre-validated for JDOQL pushdown by a client * (old hive client or non-hive one); if it was and we fail to create a filter, we will throw. - * @param skipColSchemaForPartitions skip column schema for partitions + * @param args additional arguments for getting partitions * @return Resulting partitions. Can be null if isValidatedFilter is false, and * there was error deriving the JDO filter. */ private List getPartitionsViaOrmFilter(String catName, String dbName, String tblName, ExpressionTree tree, - short maxParts, boolean isValidatedFilter, List partitionKeys, - boolean skipColSchemaForPartitions) throws MetaException { + boolean isValidatedFilter, List partitionKeys, boolean isAcidTable, + GetPartitionsArgs args) throws MetaException { Map params = new HashMap<>(); String jdoFilter = makeQueryFilterString(catName, dbName, tblName, tree, params, isValidatedFilter, partitionKeys); @@ -4219,9 +4173,9 @@ private List getPartitionsViaOrmFilter(String catName, String dbName, return null; } try (QueryWrapper query = new QueryWrapper(pm.newQuery(MPartition.class, jdoFilter))) { - if (maxParts >= 0) { + if (args.getMax() >= 0) { // User specified a row limit, set it on the Query - query.setRange(0, maxParts); + query.setRange(0, args.getMax()); } String parameterDeclaration = makeParameterDeclarationStringObj(params); query.declareParameters(parameterDeclaration); @@ -4230,7 +4184,8 @@ private List getPartitionsViaOrmFilter(String catName, String dbName, LOG.debug("Done executing query for getPartitionsViaOrmFilter"); pm.retrieveAll(mparts); // TODO: why is this inconsistent with what we get by names? LOG.debug("Done retrieving all objects for getPartitionsViaOrmFilter"); - List results = convertToParts(mparts, skipColSchemaForPartitions); + List results = + convertToParts(catName, dbName, tblName, mparts, isAcidTable, args); return results; } } @@ -4259,14 +4214,13 @@ private Integer getNumPartitionsViaOrmFilter(String catName, String dbName, Stri * Gets partition names from the table via ORM (JDOQL) name filter. * @param dbName Database name. * @param tblName Table name. - * @param partNames Partition names to get the objects for. * @param isAcidTable True if the table is ACID - * @param skipColSchemaForPartitions skip column schema for partitions + * @param args additional arguments for getting partitions * @return Resulting partitions. */ private List getPartitionsViaOrmFilter(String catName, String dbName, String tblName, - List partNames, boolean isAcidTable, boolean skipColSchemaForPartitions) throws MetaException { - + boolean isAcidTable, GetPartitionsArgs args) throws MetaException { + List partNames = args.getPartNames(); if (partNames.isEmpty()) { return Collections.emptyList(); } @@ -4283,7 +4237,7 @@ public List run(List input) throws MetaException { List mparts = (List) query.executeWithMap(queryWithParams.getRight()); List partitions = convertToParts(catName, dbName, tblName, mparts, - isAcidTable, skipColSchemaForPartitions); + isAcidTable, args); return partitions; } @@ -4372,9 +4326,8 @@ private Pair> getPartQueryWithParams( @Override public List getPartitionsByFilter(String catName, String dbName, String tblName, - String filter, short maxParts, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { - return getPartitionsByFilterInternal(catName, dbName, tblName, filter, maxParts, - true, true, skipColSchemaForPartitions); + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { + return getPartitionsByFilterInternal(catName, dbName, tblName, true, true, args); } /** Helper class for getting stuff w/transaction, direct SQL, perf logging, etc. */ @@ -4719,8 +4672,8 @@ protected Integer getJdoResult( } protected List getPartitionsByFilterInternal( - String catName, String dbName, String tblName, String filter, final short maxParts, - boolean allowSql, boolean allowJdo, boolean skipColSchemaForPartitions) + String catName, String dbName, String tblName, + boolean allowSql, boolean allowJdo, GetPartitionsArgs args) throws MetaException, NoSuchObjectException { catName = normalizeIdentifier(catName); @@ -4730,6 +4683,7 @@ protected List getPartitionsByFilterInternal( MTable mTable = ensureGetMTable(catName, dbName, tblName); List partitionKeys = convertToFieldSchemas(mTable.getPartitionKeys()); boolean isAcidTable = TxnUtils.isAcidTable(mTable.getParameters()); + String filter = args.getFilter(); final ExpressionTree tree = (filter != null && !filter.isEmpty()) ? PartFilterExprUtil.parseFilterTree(filter) : ExpressionTree.EMPTY_TREE; return new GetListHelper(catName, dbName, tblName, allowSql, allowJdo) { @@ -4742,15 +4696,14 @@ protected boolean canUseDirectSql(GetHelper> ctx) throws MetaExc @Override protected List getSqlResult(GetHelper> ctx) throws MetaException { - return directSql.getPartitionsViaSqlFilter(catName, dbName, tblName, filter, - (maxParts < 0) ? null : (int)maxParts, isAcidTable, skipColSchemaForPartitions); + return directSql.getPartitionsViaSqlFilter(catName, dbName, tblName, filter, isAcidTable, args); } @Override protected List getJdoResult( GetHelper> ctx) throws MetaException, NoSuchObjectException { - return getPartitionsViaOrmFilter(catName, dbName, tblName, tree, maxParts, true, - partitionKeys, skipColSchemaForPartitions); + return getPartitionsViaOrmFilter(catName, dbName, tblName, tree, true, + partitionKeys, isAcidTable, args); } }.run(false); } @@ -4773,8 +4726,11 @@ public List getPartitionSpecsByFilterAndProjection(final Table table, } if (fieldList == null || fieldList.isEmpty()) { // no fields are requested. Fallback to regular getPartitions implementation to return all the fields - return getPartitionsInternal(table.getCatName(), table.getDbName(), table.getTableName(), -1, - true, true); + GetPartitionsArgs.GetPartitionsArgsBuilder argsBuilder = new GetPartitionsArgs.GetPartitionsArgsBuilder() + .excludeParamKeyPattern(inputExcludePattern) + .includeParamKeyPattern(inputIncludePattern); + return getPartitionsInternal(table.getCatName(), table.getDbName(), table.getTableName(), + true, true, argsBuilder.build()); } // anonymous class below requires final String objects @@ -4864,7 +4820,11 @@ protected List getJdoResult( params.put("t3", normalizeIdentifier(catName)); } try { - return convertToParts(listMPartitionsWithProjection(fieldNames, jdoFilter, params), false); + List mparts = listMPartitionsWithProjection(fieldNames, jdoFilter, params); + return convertToParts(catName, dbName, tblName, mparts, false, new GetPartitionsArgs.GetPartitionsArgsBuilder() + .excludeParamKeyPattern(excludeParamKeyPattern) + .includeParamKeyPattern(includeParamKeyPattern) + .build()); } catch (MetaException me) { throw me; } catch (Exception e) { @@ -5251,7 +5211,7 @@ private Partition alterPartitionNoTxn(String catName, String dbname, } oldCd.t = oldCD; - return convertToPart(oldp, false); + return convertToPart(catName, dbname, name, oldp, TxnUtils.isAcidTable(table.getParameters())); } @Override @@ -10127,8 +10087,8 @@ public Map updatePartitionColumnStatistics(Table table, MTable m List statsObjs = colStats.getStatsObj(); ColumnStatisticsDesc statsDesc = colStats.getStatsDesc(); String catName = statsDesc.isSetCatName() ? statsDesc.getCatName() : getDefaultCatalog(conf); - Partition partition = convertToPart(getMPartition( - catName, statsDesc.getDbName(), statsDesc.getTableName(), partVals, mTable), false); + Partition partition = convertToPart(catName, statsDesc.getDbName(), statsDesc.getTableName(), getMPartition( + catName, statsDesc.getDbName(), statsDesc.getTableName(), partVals, mTable), TxnUtils.isAcidTable(table)); List colNames = new ArrayList<>(); for(ColumnStatisticsObj statsObj : statsObjs) { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java index 23d0187d1075..5dead2d425da 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java @@ -113,8 +113,8 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger; import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse; import org.apache.hadoop.hive.metastore.api.WriteEventInfo; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.model.MTable; -import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties; import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.metastore.properties.PropertyStore; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo; @@ -500,9 +500,11 @@ boolean dropPartition(String catName, String dbName, String tableName, String pa * @throws MetaException error access the RDBMS. * @throws NoSuchObjectException no such table exists */ + @Deprecated default List getPartitions(String catName, String dbName, String tableName, int max) throws MetaException, NoSuchObjectException { - return getPartitions(catName, dbName, tableName, max, false); + return getPartitions(catName, dbName, tableName, new GetPartitionsArgs + .GetPartitionsArgsBuilder().max(max).build()); } /** @@ -510,14 +512,13 @@ default List getPartitions(String catName, String dbName, * @param catName catalog name. * @param dbName database name. * @param tableName table name - * @param max maximum number of partitions, or -1 to get all partitions. - * @param skipColumnSchemaForPartition boolean flag to skip column schema for partition + * @param args additional arguments for getting partitions * @return list of partitions * @throws MetaException error access the RDBMS. * @throws NoSuchObjectException no such table exists */ - List getPartitions(String catName, String dbName, - String tableName, int max, boolean skipColumnSchemaForPartition) throws MetaException, NoSuchObjectException; + List getPartitions(String catName, String dbName, String tableName, + GetPartitionsArgs args) throws MetaException, NoSuchObjectException; /** * Get the location for every partition of a given table. If a partition location is a child of @@ -764,10 +765,14 @@ List alterPartitions(String catName, String db_name, String tbl_name, * @throws MetaException Error accessing the RDBMS or processing the filter. * @throws NoSuchObjectException no such table. */ + @Deprecated default List getPartitionsByFilter( String catName, String dbName, String tblName, String filter, short maxParts) throws MetaException, NoSuchObjectException { - return getPartitionsByFilter(catName, dbName, tblName, filter, maxParts, false); + return getPartitionsByFilter(catName, dbName, tblName, new GetPartitionsArgs + .GetPartitionsArgsBuilder() + .filter(filter).max(maxParts) + .build()); } /** @@ -775,15 +780,13 @@ default List getPartitionsByFilter( * @param catName catalog name * @param dbName database name * @param tblName table name - * @param filter SQL where clause filter - * @param maxParts maximum number of partitions to return, or -1 for all. - * @param skipColSchemaForPartitions skips the column schema for partition + * @param args additional arguments for getting partitions * @return list of partition objects matching the criteria * @throws MetaException Error accessing the RDBMS or processing the filter. * @throws NoSuchObjectException no such table. */ List getPartitionsByFilter( - String catName, String dbName, String tblName, String filter, short maxParts, boolean skipColSchemaForPartitions) + String catName, String dbName, String tblName, GetPartitionsArgs args) throws MetaException, NoSuchObjectException; /** @@ -829,11 +832,14 @@ List getPartitionSpecsByFilterAndProjection(Table table, * @return true if the result contains unknown partitions. * @throws TException error executing the expression */ + @Deprecated default boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, String defaultPartitionName, short maxParts, List result) throws TException { - return getPartitionsByExpr(catName, dbName, tblName, expr, defaultPartitionName, - maxParts, result, false); + return getPartitionsByExpr(catName, dbName, tblName, result, new GetPartitionsArgs + .GetPartitionsArgsBuilder() + .expr(expr).defaultPartName(defaultPartitionName).max(maxParts) + .build()); } /** @@ -841,16 +847,12 @@ default boolean getPartitionsByExpr(String catName, String dbName, String tblNam * @param catName catalog name. * @param dbName database name * @param tblName table name - * @param expr an already parsed Hive expression - * @param defaultPartitionName default name of a partition - * @param maxParts maximum number of partitions to return, or -1 for all - * @param result list to place resulting partitions in - * @param skipColSchemaForPartitions skip column schema for partitions + * @param args additional arguments for getting partitions * @return true if the result contains unknown partitions. * @throws TException error executing the expression */ boolean getPartitionsByExpr(String catName, String dbName, String tblName, - byte[] expr, String defaultPartitionName, short maxParts, List result, boolean skipColSchemaForPartitions) + List result, GetPartitionsArgs args) throws TException; /** @@ -906,7 +908,8 @@ int getNumPartitionsByPs(String catName, String dbName, String tblName, List getPartitionsByNames(String catName, String dbName, String tblName, List partNames) throws MetaException, NoSuchObjectException { - return getPartitionsByNames(catName, dbName, tblName, partNames, false); + return getPartitionsByNames(catName, dbName, tblName, new GetPartitionsArgs + .GetPartitionsArgsBuilder().partNames(partNames).build()); } /** @@ -914,15 +917,13 @@ default List getPartitionsByNames(String catName, String dbName, Stri * @param catName catalog name. * @param dbName database name. * @param tblName table name. - * @param partNames list of partition names. These are names not values, so they will include - * both the key and the value. - * @param skipColumnSchemaForPartition boolean flag to skip column schema for partition + * @param args additional arguments for getting partitions * @return list of matching partitions * @throws MetaException error accessing the RDBMS. * @throws NoSuchObjectException No such table. */ List getPartitionsByNames(String catName, String dbName, String tblName, - List partNames, boolean skipColumnSchemaForPartition) throws MetaException, NoSuchObjectException; + GetPartitionsArgs args) throws MetaException, NoSuchObjectException; Table markPartitionForEvent(String catName, String dbName, String tblName, Map partVals, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException; @@ -1154,30 +1155,16 @@ Partition getPartitionWithAuth(String catName, String dbName, String tblName, * @throws NoSuchObjectException no such table exists * @throws InvalidObjectException error fetching privilege information. */ + @Deprecated default List getPartitionsWithAuth(String catName, String dbName, String tblName, short maxParts, String userName, List groupNames) throws MetaException, NoSuchObjectException, InvalidObjectException { - return getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, groupNames, false); + return listPartitionsPsWithAuth(catName, dbName, tblName, + new GetPartitionsArgs.GetPartitionsArgsBuilder() + .max(maxParts).userName(userName).groupNames(groupNames) + .build()); } - /** - * Fetch some or all partitions for a table, along with privilege information for a particular - * user. - * @param catName catalog name. - * @param dbName database name. - * @param tblName table name. - * @param maxParts maximum number of partitions to fetch, -1 for all partitions. - * @param userName user to get privilege information for. - * @param groupNames groups to get privilege information for. - * @return list of partitions. - * @throws MetaException error access the RDBMS. - * @throws NoSuchObjectException no such table exists - * @throws InvalidObjectException error fetching privilege information. - */ - List getPartitionsWithAuth(String catName, String dbName, - String tblName, short maxParts, String userName, List groupNames, boolean isColSchemaRequired) - throws MetaException, NoSuchObjectException, InvalidObjectException; - /** * Lists partition names that match a given partial specification * @param catName catalog name. @@ -1220,11 +1207,14 @@ List listPartitionNamesPs(String catName, String db_name, String tbl_nam * @throws NoSuchObjectException No such table exists * @throws InvalidObjectException error access privilege information */ + @Deprecated default List listPartitionsPsWithAuth(String catName, String db_name, String tbl_name, List part_vals, short max_parts, String userName, List groupNames) throws MetaException, InvalidObjectException, NoSuchObjectException { - return listPartitionsPsWithAuth(catName, db_name, tbl_name, part_vals, max_parts, - userName, groupNames, false); + return listPartitionsPsWithAuth(catName, db_name, tbl_name, new GetPartitionsArgs + .GetPartitionsArgsBuilder() + .part_vals(part_vals).max(max_parts).userName(userName).groupNames(groupNames) + .build()); } /** @@ -1235,23 +1225,14 @@ default List listPartitionsPsWithAuth(String catName, String db_name, * The name of the database which has the partitions * @param tbl_name * The name of the table which has the partitions - * @param part_vals - * A partial list of values for partitions in order of the table's partition keys - * Entries can be empty if you need to specify latter partitions. - * @param max_parts - * The maximum number of partitions to return - * @param userName - * The user name for the partition for authentication privileges - * @param groupNames - * The groupNames for the partition for authentication privileges + * @param args additional arguments for getting partitions * @return A list of partitions that match the partial spec. * @throws MetaException error access RDBMS * @throws NoSuchObjectException No such table exists * @throws InvalidObjectException error access privilege information */ List listPartitionsPsWithAuth(String catName, String db_name, String tbl_name, - List part_vals, short max_parts, String userName, List groupNames, - boolean skipColSchemaForPartitions) throws MetaException, InvalidObjectException, NoSuchObjectException; + GetPartitionsArgs args) throws MetaException, InvalidObjectException, NoSuchObjectException; /** Persists the given column statistics object to the metastore * @param colStats object to persist diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java index c589fd7ba2e9..876f8ff2720c 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java @@ -60,6 +60,7 @@ import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.api.Package; import org.apache.hadoop.hive.metastore.cache.SharedCache.StatsType; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregator; import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregatorFactory; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; @@ -1473,20 +1474,20 @@ public Table getTable(String catName, String dbName, String tblName, String vali sharedCache.removePartitionsFromCache(catName, dbName, tblName, partVals); } - @Override public List getPartitions(String catName, String dbName, String tblName, int max, boolean skipColumnSchemaForPartition) + @Override public List getPartitions(String catName, String dbName, String tblName, GetPartitionsArgs args) throws MetaException, NoSuchObjectException { catName = normalizeIdentifier(catName); dbName = StringUtils.normalizeIdentifier(dbName); tblName = StringUtils.normalizeIdentifier(tblName); if (!shouldCacheTable(catName, dbName, tblName) || (canUseEvents && rawStore.isActiveTransaction())) { - return rawStore.getPartitions(catName, dbName, tblName, max); + return rawStore.getPartitions(catName, dbName, tblName, args); } Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName); if (tbl == null) { // The table containing the partitions is not yet loaded in cache - return rawStore.getPartitions(catName, dbName, tblName, max); + return rawStore.getPartitions(catName, dbName, tblName, args); } - List parts = sharedCache.listCachedPartitions(catName, dbName, tblName, max); + List parts = sharedCache.listCachedPartitions(catName, dbName, tblName, args.getMax()); return parts; } @@ -1697,9 +1698,9 @@ private boolean getPartitionNamesPrunedByExprNoTxn(Table table, byte[] expr, Str @Override // TODO: implement using SharedCache - public List getPartitionsByFilter(String catName, String dbName, String tblName, String filter, - short maxParts, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { - return rawStore.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts, skipColSchemaForPartitions); + public List getPartitionsByFilter(String catName, String dbName, String tblName, GetPartitionsArgs args) + throws MetaException, NoSuchObjectException { + return rawStore.getPartitionsByFilter(catName, dbName, tblName, args); } @Override @@ -1711,24 +1712,23 @@ public List getPartitionsByFilter(String catName, String dbName, Stri return rawStore.getPartitionSpecsByFilterAndProjection(table, projectionSpec, filterSpec); } - @Override public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, - String defaultPartitionName, short maxParts, List result, boolean skipColSchemaForPartitions) throws TException { + @Override public boolean getPartitionsByExpr(String catName, String dbName, String tblName, + List result, GetPartitionsArgs args) throws TException { catName = StringUtils.normalizeIdentifier(catName); dbName = StringUtils.normalizeIdentifier(dbName); tblName = StringUtils.normalizeIdentifier(tblName); if (!shouldCacheTable(catName, dbName, tblName) || (canUseEvents && rawStore.isActiveTransaction())) { - return rawStore.getPartitionsByExpr(catName, dbName, tblName, expr, defaultPartitionName, maxParts, result - , skipColSchemaForPartitions); + return rawStore.getPartitionsByExpr(catName, dbName, tblName, result, args); } List partNames = new LinkedList<>(); Table table = sharedCache.getTableFromCache(catName, dbName, tblName); if (table == null) { // The table is not yet loaded in cache - return rawStore.getPartitionsByExpr(catName, dbName, tblName, expr, defaultPartitionName, maxParts, result - , skipColSchemaForPartitions); + return rawStore.getPartitionsByExpr(catName, dbName, tblName, result, args); } boolean hasUnknownPartitions = - getPartitionNamesPrunedByExprNoTxn(table, expr, defaultPartitionName, maxParts, partNames, sharedCache); + getPartitionNamesPrunedByExprNoTxn(table, args.getExpr(), args.getDefaultPartName(), + (short) args.getMax(), partNames, sharedCache); for (String partName : partNames) { Partition part = sharedCache.getPartitionFromCache(catName, dbName, tblName, partNameToVals(partName)); part.unsetPrivileges(); @@ -1774,20 +1774,20 @@ public List getPartitionsByFilter(String catName, String dbName, Stri } @Override public List getPartitionsByNames(String catName, String dbName, String tblName, - List partNames, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { catName = StringUtils.normalizeIdentifier(catName); dbName = StringUtils.normalizeIdentifier(dbName); tblName = StringUtils.normalizeIdentifier(tblName); if (!shouldCacheTable(catName, dbName, tblName) || (canUseEvents && rawStore.isActiveTransaction())) { - return rawStore.getPartitionsByNames(catName, dbName, tblName, partNames, skipColSchemaForPartitions); + return rawStore.getPartitionsByNames(catName, dbName, tblName, args); } Table table = sharedCache.getTableFromCache(catName, dbName, tblName); if (table == null) { // The table is not yet loaded in cache - return rawStore.getPartitionsByNames(catName, dbName, tblName, partNames, skipColSchemaForPartitions); + return rawStore.getPartitionsByNames(catName, dbName, tblName, args); } List partitions = new ArrayList<>(); - for (String partName : partNames) { + for (String partName : args.getPartNames()) { Partition part = sharedCache.getPartitionFromCache(catName, dbName, tblName, partNameToVals(partName)); if (part != null) { partitions.add(part); @@ -1959,34 +1959,6 @@ public List getPartitionsByFilter(String catName, String dbName, Stri return p; } - @Override public List getPartitionsWithAuth(String catName, String dbName, String tblName, short maxParts, - String userName, List groupNames, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException, InvalidObjectException { - catName = StringUtils.normalizeIdentifier(catName); - dbName = StringUtils.normalizeIdentifier(dbName); - tblName = StringUtils.normalizeIdentifier(tblName); - if (!shouldCacheTable(catName, dbName, tblName) || (canUseEvents && rawStore.isActiveTransaction())) { - return rawStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, groupNames, skipColSchemaForPartitions); - } - Table table = sharedCache.getTableFromCache(catName, dbName, tblName); - if (table == null) { - // The table is not yet loaded in cache - return rawStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, groupNames, skipColSchemaForPartitions); - } - List partitions = new ArrayList<>(); - int count = 0; - for (Partition part : sharedCache.listCachedPartitions(catName, dbName, tblName, maxParts)) { - if (maxParts == -1 || count < maxParts) { - String partName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues()); - PrincipalPrivilegeSet privs = - getPartitionPrivilegeSet(catName, dbName, tblName, partName, userName, groupNames); - part.setPrivileges(privs); - partitions.add(part); - count++; - } - } - return partitions; - } - @Override public List listPartitionNamesPs(String catName, String dbName, String tblName, List partSpecs, short maxParts) throws MetaException, NoSuchObjectException { catName = StringUtils.normalizeIdentifier(catName); @@ -2021,30 +1993,31 @@ public int getNumPartitionsByPs(String catName, String dbName, String tblName, L } @Override public List listPartitionsPsWithAuth(String catName, String dbName, String tblName, - List partSpecs, short maxParts, String userName, List groupNames, boolean skipColSchemaForPartitions) - throws MetaException, InvalidObjectException, NoSuchObjectException { + GetPartitionsArgs args) throws MetaException, InvalidObjectException, NoSuchObjectException { catName = StringUtils.normalizeIdentifier(catName); dbName = StringUtils.normalizeIdentifier(dbName); tblName = StringUtils.normalizeIdentifier(tblName); if (!shouldCacheTable(catName, dbName, tblName) || (canUseEvents && rawStore.isActiveTransaction())) { - return rawStore.listPartitionsPsWithAuth(catName, dbName, tblName, partSpecs, maxParts, userName, groupNames - , skipColSchemaForPartitions); + return rawStore.listPartitionsPsWithAuth(catName, dbName, tblName, args); } Table table = sharedCache.getTableFromCache(catName, dbName, tblName); if (table == null) { // The table is not yet loaded in cache - return rawStore.listPartitionsPsWithAuth(catName, dbName, tblName, partSpecs, maxParts, userName, groupNames - , skipColSchemaForPartitions); + return rawStore.listPartitionsPsWithAuth(catName, dbName, tblName, args); } - String partNameMatcher = getPartNameMatcher(table, partSpecs); + String partNameMatcher = null; + if (args.getPart_vals() != null && !args.getPart_vals().isEmpty()) { + partNameMatcher = getPartNameMatcher(table, args.getPart_vals()); + } + int maxParts = args.getMax(); List partitions = new ArrayList<>(); List allPartitions = sharedCache.listCachedPartitions(catName, dbName, tblName, maxParts); int count = 0; for (Partition part : allPartitions) { String partName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues()); - if (partName.matches(partNameMatcher) && (maxParts == -1 || count < maxParts)) { + if ((partNameMatcher == null || partName.matches(partNameMatcher)) && (maxParts == -1 || count < maxParts)) { PrincipalPrivilegeSet privs = - getPartitionPrivilegeSet(catName, dbName, tblName, partName, userName, groupNames); + getPartitionPrivilegeSet(catName, dbName, tblName, partName, args.getUserName(), args.getGroupNames()); part.setPrivileges(privs); partitions.add(part); count++; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/client/builder/GetPartitionsArgs.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/client/builder/GetPartitionsArgs.java new file mode 100644 index 000000000000..6eb6df1d6d7f --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/client/builder/GetPartitionsArgs.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.client.builder; + +import java.util.List; + +public class GetPartitionsArgs { + private String filter; + private byte[] expr; + private String defaultPartName; + private int max; + private List partNames; + private List part_vals; + private String userName; + private List groupNames; + private String includeParamKeyPattern; + private String excludeParamKeyPattern; + private boolean skipColumnSchemaForPartition; + + private GetPartitionsArgs() { + + } + + public String getFilter() { + return filter; + } + + public byte[] getExpr() { + return expr; + } + + public String getDefaultPartName() { + return defaultPartName; + } + + public int getMax() { + return max; + } + + public List getPartNames() { + return partNames; + } + + public List getPart_vals() { + return part_vals; + } + + public String getUserName() { + return userName; + } + + public List getGroupNames() { + return groupNames; + } + + public String getIncludeParamKeyPattern() { + return includeParamKeyPattern; + } + + public String getExcludeParamKeyPattern() { + return excludeParamKeyPattern; + } + + public boolean isSkipColumnSchemaForPartition() { + return skipColumnSchemaForPartition; + } + + public static class GetPartitionsArgsBuilder { + private String filter; + private byte[] expr; + private String defaultPartName; + private int max = -1; + private List partNames; + private List part_vals; + private String userName; + private List groupNames; + private String includeParamKeyPattern; + private String excludeParamKeyPattern; + private boolean skipColumnSchemaForPartition = false; + + public GetPartitionsArgsBuilder() { + + } + + public GetPartitionsArgsBuilder(GetPartitionsArgs args) { + this.filter = args.filter; + this.expr = args.expr; + this.defaultPartName = args.defaultPartName; + this.max = args.max; + this.partNames = args.partNames; + this.part_vals = args.part_vals; + this.userName = args.userName; + this.groupNames = args.groupNames; + this.includeParamKeyPattern = args.includeParamKeyPattern; + this.excludeParamKeyPattern = args.excludeParamKeyPattern; + this.skipColumnSchemaForPartition = args.skipColumnSchemaForPartition; + } + + public GetPartitionsArgsBuilder filter(String filter) { + this.filter = filter; + return this; + } + + public GetPartitionsArgsBuilder expr(byte[] expr) { + this.expr = expr; + return this; + } + + public GetPartitionsArgsBuilder defaultPartName(String defaultPartName) { + this.defaultPartName = defaultPartName; + return this; + } + + public GetPartitionsArgsBuilder max(int max) { + this.max = max; + return this; + } + + public GetPartitionsArgsBuilder partNames(List partNames) { + this.partNames = partNames; + return this; + } + + public GetPartitionsArgsBuilder part_vals(List part_vals) { + this.part_vals = part_vals; + return this; + } + + public GetPartitionsArgsBuilder userName(String userName) { + this.userName = userName; + return this; + } + + public GetPartitionsArgsBuilder groupNames(List groupNames) { + this.groupNames = groupNames; + return this; + } + + public GetPartitionsArgsBuilder includeParamKeyPattern(String includeParamKeyPattern) { + this.includeParamKeyPattern = includeParamKeyPattern; + return this; + } + + public GetPartitionsArgsBuilder excludeParamKeyPattern(String excludeParamKeyPattern) { + this.excludeParamKeyPattern = excludeParamKeyPattern; + return this; + } + + public GetPartitionsArgsBuilder skipColumnSchemaForPartition(boolean skipColumnSchemaForPartition) { + this.skipColumnSchemaForPartition = skipColumnSchemaForPartition; + return this; + } + + public GetPartitionsArgs build() { + GetPartitionsArgs additionalArgs = new GetPartitionsArgs(); + additionalArgs.filter = filter; + additionalArgs.expr = expr; + additionalArgs.defaultPartName = defaultPartName; + additionalArgs.max = max; + additionalArgs.partNames = partNames; + additionalArgs.part_vals = part_vals; + additionalArgs.userName = userName; + additionalArgs.groupNames = groupNames; + additionalArgs.includeParamKeyPattern = includeParamKeyPattern; + additionalArgs.excludeParamKeyPattern = excludeParamKeyPattern; + additionalArgs.skipColumnSchemaForPartition = skipColumnSchemaForPartition; + return additionalArgs; + } + } +} diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java index 6b5eb47ef3b9..4c574d68c73d 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java @@ -113,6 +113,7 @@ import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse; import org.apache.hadoop.hive.metastore.api.WriteEventInfo; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.model.MTable; import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo; @@ -355,9 +356,9 @@ public List getPartitions(String catName, String dbName, String table } @Override - public List getPartitions(String catName, String dbName, String tblName, int max, - boolean skipColumnSchemaForPartition) throws MetaException, NoSuchObjectException { - return objectStore.getPartitions(catName, dbName, tblName, max, skipColumnSchemaForPartition); + public List getPartitions(String catName, String dbName, String tblName, + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { + return objectStore.getPartitions(catName, dbName, tblName, args); } @Override @@ -471,8 +472,8 @@ public List getPartitionsByFilter(String catName, String dbName, Stri @Override public List getPartitionsByFilter(String catName, String dbName, String tblName, - String filter, short maxParts, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { - return objectStore.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts, skipColSchemaForPartitions); + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { + return objectStore.getPartitionsByFilter(catName, dbName, tblName, args); } @Override @@ -508,8 +509,8 @@ public List getPartitionsByNames(String catName, String dbName, Strin @Override public List getPartitionsByNames(String catName, String dbName, String tblName, - List partNames, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { - return objectStore.getPartitionsByNames(catName, dbName, tblName, partNames, skipColSchemaForPartitions); + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { + return objectStore.getPartitionsByNames(catName, dbName, tblName, args); } @Override @@ -520,10 +521,9 @@ public boolean getPartitionsByExpr(String catName, String dbName, String tblName } @Override - public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, - String defaultPartitionName, short maxParts, List result, boolean skipColSchemaForPartitions) throws TException { - return objectStore.getPartitionsByExpr(catName, - dbName, tblName, expr, defaultPartitionName, maxParts, result, skipColSchemaForPartitions); + public boolean getPartitionsByExpr(String catName, String dbName, String tblName, + List result, GetPartitionsArgs args) throws TException { + return objectStore.getPartitionsByExpr(catName, dbName, tblName, result, args); } @Override @@ -716,15 +716,6 @@ public List getPartitionsWithAuth(String catName, String dbName, Stri groupNames); } - @Override - public List getPartitionsWithAuth(String catName, String dbName, String tblName, short maxParts, - String userName, List groupNames, boolean isColumnSchemaRequired) throws MetaException, NoSuchObjectException, - InvalidObjectException { - - return objectStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, - groupNames, isColumnSchemaRequired); - } - @Override public List listPartitionNamesPs(String catName, String dbName, String tblName, List partVals, short maxParts) @@ -741,11 +732,9 @@ public List listPartitionsPsWithAuth(String catName, String dbName, S } @Override - public List listPartitionsPsWithAuth(String catName, String dbName, String tblName, - List partVals, short maxParts, String userName, List groupNames, boolean skipColSchemaForPartitions) + public List listPartitionsPsWithAuth(String catName, String dbName, String tblName, GetPartitionsArgs args) throws MetaException, InvalidObjectException, NoSuchObjectException { - return objectStore.listPartitionsPsWithAuth(catName, dbName, tblName, partVals, maxParts, - userName, groupNames, skipColSchemaForPartitions); + return objectStore.listPartitionsPsWithAuth(catName, dbName, tblName, args); } @Override diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java index 7b318101bced..a12b989c7307 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java @@ -127,6 +127,7 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.model.MTable; import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; @@ -367,8 +368,8 @@ public List getPartitions(String catName, String dbName, String table } @Override - public List getPartitions(String catName, String dbName, String tblName, int max, - boolean skipColumnSchemaForPartition) throws MetaException, NoSuchObjectException { + public List getPartitions(String catName, String dbName, String tblName, + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { return Collections.emptyList(); } @@ -489,7 +490,7 @@ public List getPartitionsByFilter(String catName, String dbName, Stri @Override public List getPartitionsByFilter(String catName, String dbName, String tblName, - String filter, short maxParts, boolean skipColSchemaForPartitions) + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { return Collections.emptyList(); } @@ -510,7 +511,7 @@ public List getPartitionsByNames(String catName, String dbName, Strin @Override public List getPartitionsByNames(String catName, String dbName, String tblName, - List partNames, boolean skipColSchemaForPartitions) throws MetaException, NoSuchObjectException { + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { return Collections.emptyList(); } @@ -521,8 +522,8 @@ public boolean getPartitionsByExpr(String catName, String dbName, String tblName } @Override - public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, - String defaultPartitionName, short maxParts, List result, boolean skipColSchemaForPartitions) throws TException { + public boolean getPartitionsByExpr(String catName, String dbName, String tblName, + List result, GetPartitionsArgs args) throws TException { return false; } @@ -748,13 +749,6 @@ public List getPartitionsWithAuth(String catName, String dbName, Stri return Collections.emptyList(); } - @Override - public List getPartitionsWithAuth(String catName, String dbName, String tblName, short maxParts, - String userName, List groupNames, boolean isColumnSchemaRequired) - throws MetaException, NoSuchObjectException, InvalidObjectException { - return Collections.emptyList(); - } - @Override public List listPartitionNamesPs(String catName, String db_name, String tbl_name, List part_vals, short max_parts) throws MetaException, NoSuchObjectException { @@ -772,7 +766,7 @@ public List listPartitionsPsWithAuth(String catName, String db_name, @Override public List listPartitionsPsWithAuth(String catName, String db_name, String tbl_name, - List part_vals, short max_parts, String userName, List groupNames, boolean skipColSchemaForPartitions) + GetPartitionsArgs args) throws MetaException, InvalidObjectException, NoSuchObjectException { return Collections.emptyList(); } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjectionAndFilterSpecs.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjectionAndFilterSpecs.java index 7e6372735858..7a37f6032b4a 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjectionAndFilterSpecs.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjectionAndFilterSpecs.java @@ -229,8 +229,12 @@ public void testPartitionProjectionEmptySpec() throws Throwable { Assert.assertEquals(origPartition.getLastAccessTime(), retPartition.getLastAccessTime()); Assert.assertEquals(origPartition.getSd().getLocation(), sharedSD.getLocation() + retPartition.getRelativePath()); - validateMap(origPartition.getParameters(), retPartition.getParameters()); - validateList(origPartition.getValues(), retPartition.getValues()); + Assert.assertFalse("excluded parameter key is found in the response", + retPartition.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key1")); + Assert.assertFalse("excluded parameter key is found in the response", + retPartition.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key2")); + Assert.assertEquals("Additional parameters returned", + 3, retPartition.getParameters().size()); } } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java index 5c6ad05839b8..5746a9526499 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java @@ -64,6 +64,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder; import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder; import org.apache.hadoop.hive.metastore.client.builder.HiveObjectPrivilegeBuilder; import org.apache.hadoop.hive.metastore.client.builder.HiveObjectRefBuilder; @@ -645,7 +646,7 @@ public void testDirectSQLDropPartitionsCacheInSession() List partitions; try(AutoCloseable c =deadline()) { partitions = objectStore.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, - 10, false, true); + false, true, new GetPartitionsArgs.GetPartitionsArgsBuilder().max(10).build()); } Assert.assertEquals(3, partitions.size()); @@ -656,7 +657,8 @@ public void testDirectSQLDropPartitionsCacheInSession() } try (AutoCloseable c = deadline()) { // query the partitions with JDO, checking the cache is not causing any problem - partitions = objectStore.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10, false, true); + partitions = objectStore.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, false, true, + new GetPartitionsArgs.GetPartitionsArgsBuilder().max(10).build()); } Assert.assertEquals(1, partitions.size()); } @@ -671,16 +673,17 @@ public void testDirectSQLDropPartitionsCacheCrossSession() objectStore2.setConf(conf); createPartitionedTable(false, false); + GetPartitionsArgs args = new GetPartitionsArgs.GetPartitionsArgsBuilder().max(10).build(); // query the partitions with JDO in the 1st session List partitions; try (AutoCloseable c = deadline()) { - partitions = objectStore.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10, false, true); + partitions = objectStore.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, false, true, args); } Assert.assertEquals(3, partitions.size()); // query the partitions with JDO in the 2nd session try (AutoCloseable c = deadline()) { - partitions = objectStore2.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10, false, true); + partitions = objectStore2.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, false, true, args); } Assert.assertEquals(3, partitions.size()); @@ -693,7 +696,7 @@ public void testDirectSQLDropPartitionsCacheCrossSession() // query the partitions with JDO in the 2nd session, checking the cache is not causing any // problem try (AutoCloseable c = deadline()) { - partitions = objectStore2.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10, false, true); + partitions = objectStore2.getPartitionsInternal(DEFAULT_CATALOG_NAME, DB1, TABLE1, false, true, args); } Assert.assertEquals(1, partitions.size()); } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java index ed747356b2db..8d209679c152 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.ClassUtils; import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.hadoop.hive.metastore.client.builder.GetPartitionsArgs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; @@ -51,12 +52,12 @@ public VerifyingObjectStore() { @Override public List getPartitionsByFilter(String catName, String dbName, String tblName, - String filter, short maxParts) + GetPartitionsArgs args) throws MetaException, NoSuchObjectException { List sqlResults = getPartitionsByFilterInternal( - catName, dbName, tblName, filter, maxParts, true, false, false); + catName, dbName, tblName, true, false, args); List ormResults = getPartitionsByFilterInternal( - catName, dbName, tblName, filter, maxParts, false, true, false); + catName, dbName, tblName, false, true, args); verifyLists(sqlResults, ormResults, Partition.class); return sqlResults; } @@ -64,22 +65,23 @@ public List getPartitionsByFilter(String catName, String dbName, Stri @Override public List getPartitionsByNames(String catName, String dbName, String tblName, List partNames) throws MetaException, NoSuchObjectException { + GetPartitionsArgs args = new GetPartitionsArgs.GetPartitionsArgsBuilder().partNames(partNames).build(); List sqlResults = getPartitionsByNamesInternal( - catName, dbName, tblName, partNames, true, false, false); + catName, dbName, tblName, true, false, args); List ormResults = getPartitionsByNamesInternal( - catName, dbName, tblName, partNames, false, true, false); + catName, dbName, tblName, false, true, args); verifyLists(sqlResults, ormResults, Partition.class); return sqlResults; } @Override - public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr, - String defaultPartitionName, short maxParts, List result) throws TException { + public boolean getPartitionsByExpr(String catName, String dbName, String tblName, List result, + GetPartitionsArgs args) throws TException { List ormParts = new LinkedList<>(); boolean sqlResult = getPartitionsByExprInternal( - catName, dbName, tblName, expr, defaultPartitionName, maxParts, result, true, false, false); + catName, dbName, tblName, result, true, false, args); boolean ormResult = getPartitionsByExprInternal( - catName, dbName, tblName, expr, defaultPartitionName, maxParts, ormParts, false, true, false); + catName, dbName, tblName, ormParts, false, true, args); if (sqlResult != ormResult) { String msg = "The unknown flag is different - SQL " + sqlResult + ", ORM " + ormResult; LOG.error(msg); @@ -91,10 +93,10 @@ public boolean getPartitionsByExpr(String catName, String dbName, String tblName @Override public List getPartitions( - String catName, String dbName, String tableName, int maxParts) throws MetaException, NoSuchObjectException { + String catName, String dbName, String tableName, GetPartitionsArgs args) throws MetaException, NoSuchObjectException { openTransaction(); - List sqlResults = getPartitionsInternal(catName, dbName, tableName, maxParts, true, false); - List ormResults = getPartitionsInternal(catName, dbName, tableName, maxParts, false, true); + List sqlResults = getPartitionsInternal(catName, dbName, tableName, true, false, args); + List ormResults = getPartitionsInternal(catName, dbName, tableName, false, true, args); verifyLists(sqlResults, ormResults, Partition.class); commitTransaction(); return sqlResults; diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java index d06407342fdc..91ba1c311f33 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java @@ -167,6 +167,11 @@ protected void addPartition(IMetaStoreClient client, Table table, List v client.add_partition(partitionBuilder.build(metaStore.getConf())); } + protected void addPartitions(IMetaStoreClient client, List partitions) + throws TException{ + client.add_partitions(partitions); + } + private Table createTable3PartCols1PartGeneric(IMetaStoreClient client, boolean authOn) throws TException { Table t = createTestTable(client, DB_NAME, TABLE_NAME, Lists.newArrayList("yyyy", "mm", @@ -228,7 +233,7 @@ private void assertPartitionsHaveCorrectValues(List partitions, } } - private void assertCorrectPartitionNames(List names, + protected void assertCorrectPartitionNames(List names, List> testValues, ListpartCols) throws Exception { assertEquals(testValues.size(), names.size()); @@ -266,7 +271,9 @@ private void assertCorrectPartitionValuesResponse(List> testValues, } } + protected void assertPartitionsHaveCorrectParams(List partitions) { + } /** * Testing listPartitions(String,String,short) -> @@ -277,9 +284,11 @@ public void testListPartitionsAll() throws Exception { List> testValues = createTable4PartColsParts(client).testValues; List partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1); assertPartitionsHaveCorrectValues(partitions, testValues); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(0, 1)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short) 0); assertTrue(partitions.isEmpty()); @@ -300,10 +309,12 @@ public void testGetPartitionsRequest() throws Exception { req.setMaxParts((short)-1); PartitionsResponse res = client.getPartitionsRequest(req); assertPartitionsHaveCorrectValues(res.getPartitions(), testValues); + assertPartitionsHaveCorrectParams(res.getPartitions()); req.setMaxParts((short)1); res = client.getPartitionsRequest(req); assertPartitionsHaveCorrectValues(res.getPartitions(), testValues.subList(0, 1)); + assertPartitionsHaveCorrectParams(res.getPartitions()); req.setMaxParts((short)0); res = client.getPartitionsRequest(req); @@ -385,11 +396,13 @@ public void testListPartitionsByValues() throws Exception { assertEquals(2, partitions.size()); assertEquals(testValues.get(2), partitions.get(0).getValues()); assertEquals(testValues.get(3), partitions.get(1).getValues()); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitions(DB_NAME, TABLE_NAME, Lists.newArrayList("2017", "11"), (short)-1); assertEquals(1, partitions.size()); assertEquals(testValues.get(3), partitions.get(0).getValues()); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitions(DB_NAME, TABLE_NAME, Lists.newArrayList("20177", "11"), (short)-1); @@ -537,11 +550,13 @@ public void testListPartitionsWithAuth() throws Exception { assertEquals(4, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short)2, USER_NAME, groups); assertEquals(2, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(0, 2)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); } @Test(expected = MetaException.class) @@ -558,10 +573,12 @@ public void testListPartitionsWithAuthLowMaxParts() throws Exception { getClient().listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short) 2, USER_NAME, Lists.newArrayList(GROUP)); assertTrue(partitions.size() == 2); partitions.forEach(p -> assertAuthInfoReturned(USER_NAME, GROUP, p)); + assertPartitionsHaveCorrectParams(partitions); partitions = getClient().listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short) -1, USER_NAME, Lists.newArrayList(GROUP)); assertTrue(partitions.size() == 4); partitions.forEach(p -> assertAuthInfoReturned(USER_NAME, GROUP, p)); + assertPartitionsHaveCorrectParams(partitions); } @Test @@ -573,6 +590,7 @@ public void testListPartitionsWithAuthNoPrivilegesSet() throws Exception { assertEquals(4, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues); partitions.forEach(partition -> assertNull(partition.getPrivileges())); + assertPartitionsHaveCorrectParams(partitions); } @Test(expected = NoSuchObjectException.class) @@ -621,13 +639,15 @@ public void testListPartitionsWithAuthNullTblName() throws Exception { @Test public void testListPartitionsWithAuthNullUser() throws Exception { createTable4PartColsPartsAuthOn(client); - client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short)-1, null, Lists.newArrayList()); + assertPartitionsHaveCorrectParams(client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short)-1, + null, Lists.newArrayList())); } @Test public void testListPartitionsWithAuthNullGroup() throws Exception { createTable4PartColsPartsAuthOn(client); - client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short)-1, "user0", null); + assertPartitionsHaveCorrectParams(client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (short)-1, + "user0", null)); } /** @@ -654,6 +674,7 @@ public void testListPartitionsWithAuthRequestByValues() throws Exception { assertEquals(1, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(3, 4)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); req.setPartVals(Lists .newArrayList("2017")); @@ -662,6 +683,7 @@ public void testListPartitionsWithAuthRequestByValues() throws Exception { assertEquals(2, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(2, 4)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); req.setMaxParts((short)1); res = client.listPartitionsWithAuthInfoRequest(req); @@ -669,6 +691,7 @@ public void testListPartitionsWithAuthRequestByValues() throws Exception { assertEquals(1, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(2, 3)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); req.setMaxParts((short)-1); req.setPartVals(Lists @@ -692,18 +715,21 @@ public void testListPartitionsWithAuthByValues() throws Exception { assertEquals(1, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(3, 4)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, Lists .newArrayList("2017"), (short)-1, USER_NAME, groups); assertEquals(2, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(2, 4)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, Lists .newArrayList("2017"), (short)1, USER_NAME, groups); assertEquals(1, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(2, 3)); partitions.forEach(partition -> assertAuthInfoReturned(USER_NAME, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, Lists .newArrayList("2013"), (short)-1, USER_NAME, groups); @@ -751,6 +777,7 @@ public void testListPartitionsWithAuthByValuesNoPrivilegesSet() throws Exception assertEquals(1, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(3, 4)); partitions.forEach(partition -> assertAuthInfoReturned(user, groups.get(0), partition)); + assertPartitionsHaveCorrectParams(partitions); } @Test(expected = NoSuchObjectException.class) @@ -817,6 +844,7 @@ public void testListPartitionsWithAuthByValuesNullUser() throws Exception { List partitions = client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, Lists .newArrayList("2017", "11", "27"), (short)-1, null, Lists.newArrayList()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(3, 4)); + assertPartitionsHaveCorrectParams(partitions); } @Test @@ -825,6 +853,7 @@ public void testListPartitionsWithAuthByValuesNullGroup() throws Exception { List partitions = client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, Lists .newArrayList("2017", "11", "27"), (short)-1, "", null); assertPartitionsHaveCorrectValues(partitions, partValues.subList(3, 4)); + assertPartitionsHaveCorrectParams(partitions); } @@ -841,15 +870,18 @@ public void testListPartitionsByFilter() throws Exception { "yyyy=\"2017\" OR " + "mm=\"02\"", (short)-1); assertEquals(3, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(1, 4)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME, "yyyy=\"2017\" OR " + "mm=\"02\"", (short)2); assertEquals(2, partitions.size()); assertPartitionsHaveCorrectValues(partitions, partValues.subList(1, 3)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME, "yyyy=\"2017\" OR " + "mm=\"02\"", (short)0); assertTrue(partitions.isEmpty()); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME, "yyyy=\"2017\" AND mm=\"99\"", (short)-1); @@ -875,26 +907,32 @@ public void testListPartitionsByFilterCaseInsensitive() throws Exception { List partitions = client.listPartitionsByFilter(DB_NAME, tableName, "yYyY=\"2017\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(0, 3)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "yYyY=\"2017\" AND mOnTh=\"may\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(2, 3)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "yYyY!=\"2017\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(3, 5)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "mOnTh=\"september\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(4, 5)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "mOnTh like \"m%\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(0, 4)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "yYyY=\"2018\" AND mOnTh like \"m%\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(3, 4)); + assertPartitionsHaveCorrectParams(partitions); client.dropTable(DB_NAME, tableName); } @@ -917,6 +955,7 @@ public void testListPartitionsByFilterCaseSensitive() throws Exception { List partitions = client.listPartitionsByFilter(DB_NAME, tableName, "month=\"mArCh\"", (short) -1); Assert.assertTrue(partitions.isEmpty()); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "yyyy=\"2017\" AND month=\"May\"", (short) -1); @@ -925,12 +964,13 @@ public void testListPartitionsByFilterCaseSensitive() throws Exception { partitions = client.listPartitionsByFilter(DB_NAME, tableName, "yyyy=\"2017\" AND month!=\"mArCh\"", (short) -1); assertPartitionsHaveCorrectValues(partitions, testValues.subList(0, 3)); + assertPartitionsHaveCorrectParams(partitions); partitions = client.listPartitionsByFilter(DB_NAME, tableName, "month like \"M%\"", (short) -1); Assert.assertTrue(partitions.isEmpty()); client.dropTable(DB_NAME, tableName); - + assertPartitionsHaveCorrectParams(partitions); } @Test(expected = MetaException.class) @@ -997,6 +1037,7 @@ public void testListPartitionsByFilterNullFilter() throws Exception { List partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME, null, (short)-1); assertEquals(4, partitions.size()); + assertPartitionsHaveCorrectParams(partitions); } @Test @@ -1004,6 +1045,7 @@ public void testListPartitionsByFilterEmptyFilter() throws Exception { createTable4PartColsParts(client); List partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME, "", (short)-1); assertEquals(4, partitions.size()); + assertPartitionsHaveCorrectParams(partitions); } @@ -1608,16 +1650,18 @@ public void otherCatalog() throws TException { .addValue("a" + i) .build(metaStore.getConf()); } - client.add_partitions(Arrays.asList(parts)); + addPartitions(client, Arrays.asList(parts)); List fetched = client.listPartitions(catName, dbName, tableName, -1); Assert.assertEquals(parts.length, fetched.size()); Assert.assertEquals(catName, fetched.get(0).getCatName()); + assertPartitionsHaveCorrectParams(fetched); fetched = client.listPartitions(catName, dbName, tableName, Collections.singletonList("a0"), -1); Assert.assertEquals(1, fetched.size()); Assert.assertEquals(catName, fetched.get(0).getCatName()); + assertPartitionsHaveCorrectParams(fetched); PartitionSpecProxy proxy = client.listPartitionSpecs(catName, dbName, tableName, -1); Assert.assertEquals(parts.length, proxy.size()); @@ -1626,6 +1670,7 @@ public void otherCatalog() throws TException { fetched = client.listPartitionsByFilter(catName, dbName, tableName, "partcol=\"a0\"", -1); Assert.assertEquals(1, fetched.size()); Assert.assertEquals(catName, fetched.get(0).getCatName()); + assertPartitionsHaveCorrectParams(fetched); proxy = client.listPartitionSpecsByFilter(catName, dbName, tableName, "partcol=\"a0\"", -1); Assert.assertEquals(1, proxy.size()); From b12f84480a4535c1173187b71e44a1bcfbf0edc5 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Sat, 4 Nov 2023 16:01:34 +0530 Subject: [PATCH 038/179] HIVE-27779: Iceberg: Drop partition support (#4785) (Sourabh Badhya reviewed by Denys Kuzmenko) --- .../iceberg/mr/hive/HiveIcebergMetaHook.java | 67 ++- .../mr/hive/HiveIcebergStorageHandler.java | 50 +- .../iceberg/mr/hive/IcebergTableUtil.java | 14 + .../queries/positive/iceberg_drop_partition.q | 79 +++ .../positive/iceberg_drop_partition.q.out | 530 ++++++++++++++++++ .../apache/hadoop/hive/ql/metadata/Hive.java | 33 +- .../hive/ql/metadata/HiveStorageHandler.java | 10 + .../hadoop/hive/ql/parse/ParseUtils.java | 4 +- .../hadoop/hive/metastore/HiveMetaHook.java | 13 + .../hive/metastore/HiveMetaStoreClient.java | 14 +- .../hadoop/hive/metastore/HMSHandler.java | 3 +- 11 files changed, 797 insertions(+), 20 deletions(-) create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition.q.out diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index ad711d16c40e..f174bd7b2fe6 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.PartitionDropOptions; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.EnvironmentContext; @@ -51,11 +52,16 @@ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.ql.ddl.table.AlterTableType; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.PartitionTransform; import org.apache.hadoop.hive.ql.parse.TransformSpec; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionStateUtil; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; @@ -70,9 +76,14 @@ import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.PartitionsTable; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Snapshot; @@ -89,10 +100,12 @@ import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.hive.CachedClientPool; import org.apache.iceberg.hive.HiveSchemaUtil; import org.apache.iceberg.hive.HiveTableOperations; import org.apache.iceberg.hive.MetastoreLock; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; @@ -106,9 +119,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructProjection; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,7 +144,7 @@ public class HiveIcebergMetaHook implements HiveMetaHook { AlterTableType.ADDCOLS, AlterTableType.REPLACE_COLUMNS, AlterTableType.RENAME_COLUMN, AlterTableType.ADDPROPS, AlterTableType.DROPPROPS, AlterTableType.SETPARTITIONSPEC, AlterTableType.UPDATE_COLUMNS, AlterTableType.RENAME, AlterTableType.EXECUTE, AlterTableType.CREATE_BRANCH, - AlterTableType.CREATE_TAG, AlterTableType.DROP_BRANCH, AlterTableType.DROP_TAG); + AlterTableType.CREATE_TAG, AlterTableType.DROP_BRANCH, AlterTableType.DROPPARTITION, AlterTableType.DROP_TAG); private static final List MIGRATION_ALLOWED_SOURCE_FORMATS = ImmutableList.of( FileFormat.PARQUET.name().toLowerCase(), FileFormat.ORC.name().toLowerCase(), @@ -997,6 +1012,56 @@ public void postGetTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) { } } + @Override + public void preDropPartitions(org.apache.hadoop.hive.metastore.api.Table hmsTable, + EnvironmentContext context, + List> partExprs) + throws MetaException { + Table icebergTbl = IcebergTableUtil.getTable(conf, hmsTable); + DeleteFiles deleteFiles = icebergTbl.newDelete(); + List expressions = partExprs.stream().map(partExpr -> { + ExprNodeDesc exprNodeDesc = SerializationUtilities + .deserializeObjectWithTypeInformation(partExpr.getRight(), true); + SearchArgument sarg = ConvertAstToSearchArg.create(conf, (ExprNodeGenericFuncDesc) exprNodeDesc); + return HiveIcebergFilterFactory.generateFilterExpression(sarg); + }).collect(Collectors.toList()); + PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils + .createMetadataTableInstance(icebergTbl, MetadataTableType.PARTITIONS); + List partitionList = Lists.newArrayList(); + Expression finalExp = Expressions.alwaysFalse(); + PartitionSpec pSpec = icebergTbl.spec(); + for (int index = 0; index < expressions.size(); index++) { + finalExp = Expressions.or(finalExp, expressions.get(index)); + } + ResidualEvaluator resEval = ResidualEvaluator.of(icebergTbl.spec(), finalExp, false); + try (CloseableIterable fileScanTasks = partitionsTable.newScan().planFiles()) { + fileScanTasks.forEach(task -> + partitionList.addAll(Sets.newHashSet(CloseableIterable.transform(task.asDataTask().rows(), row -> { + StructProjection data = row.get(0, StructProjection.class); + return IcebergTableUtil.toPartitionData(data, pSpec.partitionType()); + })).stream() + .filter(partitionData -> resEval.residualFor(partitionData).isEquivalentTo(Expressions.alwaysTrue())) + .collect(Collectors.toSet()))); + + Expression partitionSetFilter = Expressions.alwaysFalse(); + for (PartitionData partitionData : partitionList) { + Expression partFilter = Expressions.alwaysTrue(); + for (int index = 0; index < pSpec.fields().size(); index++) { + PartitionField field = icebergTbl.spec().fields().get(index); + partFilter = Expressions.and( + partFilter, Expressions.equal(field.name(), partitionData.get(index, Object.class))); + } + partitionSetFilter = Expressions.or(partitionSetFilter, partFilter); + } + + deleteFiles.deleteFromRowFilter(partitionSetFilter); + deleteFiles.commit(); + } catch (IOException e) { + throw new MetaException(String.format("Error while fetching the partitions due to: %s", e)); + } + context.putToProperties(HiveMetaStoreClient.SKIP_DROP_PARTITION, "true"); + } + private class PreAlterTableProperties { private String tableLocation; private String format; diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index c11663e3a659..0afdd79ba55b 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -82,10 +82,12 @@ import org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader; import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.metadata.DummyPartition; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler; import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec; @@ -165,6 +167,7 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.expressions.StrictMetricsEvaluator; import org.apache.iceberg.hadoop.HadoopConfigurable; +import org.apache.iceberg.hive.HiveSchemaUtil; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; @@ -1133,8 +1136,11 @@ public List sortColumns(org.apache.hadoop.hive.ql.metadata.Table hm } Schema schema = table.schema(); + List hiveSchema = HiveSchemaUtil.convert(schema); + Map colNameToColType = hiveSchema.stream() + .collect(Collectors.toMap(FieldSchema::getName, FieldSchema::getType)); return table.sortOrder().fields().stream().map(s -> new FieldSchema(schema.findColumnName(s.sourceId()), - schema.findType(s.sourceId()).toString(), + colNameToColType.get(schema.findColumnName(s.sourceId())), String.format("Transform: %s, Sort direction: %s, Null sort order: %s", s.transform().toString(), s.direction().name(), s.nullOrder().name()))).collect(Collectors.toList()); } @@ -1932,4 +1938,46 @@ public boolean canPerformMetadataDelete(org.apache.hadoop.hive.ql.metadata.Table return false; } } + + @Override + public List getPartitionKeys(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { + Table icebergTable = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + Schema schema = icebergTable.schema(); + List hiveSchema = HiveSchemaUtil.convert(schema); + Map colNameToColType = hiveSchema.stream() + .collect(Collectors.toMap(FieldSchema::getName, FieldSchema::getType)); + return icebergTable.spec().fields().stream().map(partField -> + new FieldSchema(schema.findColumnName(partField.sourceId()), + colNameToColType.get(schema.findColumnName(partField.sourceId())), + String.format("Transform: %s", partField.transform().toString()))).collect(Collectors.toList()); + } + + @Override + public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, ExprNodeDesc desc) + throws SemanticException { + Table icebergTable = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + PartitionSpec pSpec = icebergTable.spec(); + PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils + .createMetadataTableInstance(icebergTable, MetadataTableType.PARTITIONS); + SearchArgument sarg = ConvertAstToSearchArg.create(conf, (ExprNodeGenericFuncDesc) desc); + Expression expression = HiveIcebergFilterFactory.generateFilterExpression(sarg); + Set partitionList = Sets.newHashSet(); + ResidualEvaluator resEval = ResidualEvaluator.of(pSpec, expression, false); + try (CloseableIterable fileScanTasks = partitionsTable.newScan().planFiles()) { + fileScanTasks.forEach(task -> + partitionList.addAll(Sets.newHashSet(CloseableIterable.transform(task.asDataTask().rows(), row -> { + StructProjection data = row.get(PART_IDX, StructProjection.class); + return IcebergTableUtil.toPartitionData(data, pSpec.partitionType()); + })).stream() + .filter(partitionData -> resEval.residualFor(partitionData).isEquivalentTo(Expressions.alwaysTrue())) + .collect(Collectors.toSet()))); + + + return partitionList.stream() + .map(partitionData -> new DummyPartition(hmsTable, pSpec.partitionToPath(partitionData))) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new SemanticException(String.format("Error while fetching the partitions due to: %s", e)); + } + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java index aacbf4c4e3d7..e22c5cfcf362 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java @@ -36,9 +36,11 @@ import org.apache.hadoop.hive.ql.session.SessionStateUtil; import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.ManageSnapshots; +import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdatePartitionSpec; @@ -47,6 +49,7 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.types.Types; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -292,4 +295,15 @@ public static void performMetadataDelete(Table icebergTable, String branchName, } deleteFiles.deleteFromRowFilter(exp).commit(); } + + public static PartitionData toPartitionData(StructLike key, Types.StructType keyType) { + PartitionData data = new PartitionData(keyType); + for (int i = 0; i < keyType.fields().size(); i++) { + Object val = key.get(i, keyType.fields().get(i).type().typeId().javaClass()); + if (val != null) { + data.set(i, val); + } + } + return data; + } } diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition.q new file mode 100644 index 000000000000..aa0fbae9420c --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_drop_partition.q @@ -0,0 +1,79 @@ +-- SORT_QUERY_RESULTS +create table ice_int (a int, b string) partitioned by spec (a) stored by iceberg; +insert into table ice_int values (1, 'ABC'); +insert into table ice_int values (2, 'DEF'); +insert into table ice_int values (3, 'ABC'); +insert into table ice_int values (4, 'DEF'); +insert into table ice_int values (1, 'ABC'); +insert into table ice_int values (2, 'DEF'); + +alter table ice_int drop partition (a <= 2), partition (a >= 3); + +select * from ice_int; + +create table ice_str (a int, b string) partitioned by spec (b) stored by iceberg; +insert into table ice_str values (1, 'ABC'); +insert into table ice_str values (2, 'DEF'); +insert into table ice_str values (3, 'ABC'); +insert into table ice_str values (4, 'DEF'); +insert into table ice_str values (1, 'ABC'); +insert into table ice_str values (2, 'XYZ'); + +alter table ice_str drop partition (b != 'ABC'), partition (b == 'XYZ'); + +select * from ice_str; + +create table ice_int_double_part (a int, b int, c int) partitioned by spec (a, b) stored by iceberg; +insert into table ice_int_double_part values (1, 2, 3); +insert into table ice_int_double_part values (2, 3, 4); +insert into table ice_int_double_part values (3, 4, 5); +insert into table ice_int_double_part values (4, 5, 6); +insert into table ice_int_double_part values (1, 2, 4); +insert into table ice_int_double_part values (2, 1, 5); + +alter table ice_int_double_part drop partition (a <= 2, b <= 1), partition (a >= 3, b != 4); + +select * from ice_int_double_part; + +create table ice_date_int_double_part (a date, b int, c int) partitioned by spec (a, b) stored by iceberg; +insert into table ice_date_int_double_part values ('2022-02-07', 2, 3); +insert into table ice_date_int_double_part values ('2022-08-07', 3, 4); +insert into table ice_date_int_double_part values ('2022-10-05', 4, 5); +insert into table ice_date_int_double_part values ('2022-01-17', 5, 6); +insert into table ice_date_int_double_part values ('2022-04-08', 2, 4); +insert into table ice_date_int_double_part values ('2023-02-07', 1, 5); + +alter table ice_date_int_double_part drop partition (a <= '2023-02-07', b <= 1), partition (a >= '2022-08-07', b >= 2); + +select * from ice_date_int_double_part; + +create table ice_date_double_double_part (a date, b double, c int) partitioned by spec (a, b) stored by iceberg; +insert into table ice_date_double_double_part values ('2022-02-07', 2.75, 3); +insert into table ice_date_double_double_part values ('2022-08-07', 3.25, 4); +insert into table ice_date_double_double_part values ('2022-10-05', 4.23, 5); +insert into table ice_date_double_double_part values ('2022-01-17', 5.67, 6); +insert into table ice_date_double_double_part values ('2022-04-08', 2.45, 4); +insert into table ice_date_double_double_part values ('2023-02-07', 1.08, 5); + +alter table ice_date_double_double_part drop partition (a <= '2023-02-07', b <= 1.09), partition (a >= '2022-08-07', b >= 2.78); + +select * from ice_date_int_double_part; + +create table ice_date_bigint_double_part (a date, b bigint, c int) partitioned by spec (a, b) stored by iceberg; +insert into table ice_date_bigint_double_part values ('2022-02-07', 267859937678997886, 3); +insert into table ice_date_bigint_double_part values ('2022-08-07', 325678599459970774, 4); +insert into table ice_date_bigint_double_part values ('2022-10-05', 423789504756478599, 5); +insert into table ice_date_bigint_double_part values ('2022-01-17', 567890387564883960, 6); +insert into table ice_date_bigint_double_part values ('2022-04-08', 245789600487678594, 4); +insert into table ice_date_bigint_double_part values ('2023-02-07', 108789600487566478, 5); + +alter table ice_date_bigint_double_part drop partition (a <= '2023-02-07', b <= 109000000000000000L), partition (a >= '2022-08-07', b >= 278000000000000000L); + +select * from ice_date_bigint_double_part; + +drop table ice_int; +drop table ice_str; +drop table ice_int_double_part; +drop table ice_date_int_double_part; +drop table ice_date_double_double_part; +drop table ice_date_bigint_double_part; diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition.q.out new file mode 100644 index 000000000000..078d918cd0ec --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_drop_partition.q.out @@ -0,0 +1,530 @@ +PREHOOK: query: create table ice_int (a int, b string) partitioned by spec (a) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_int +POSTHOOK: query: create table ice_int (a int, b string) partitioned by spec (a) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_int +PREHOOK: query: insert into table ice_int values (1, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int +POSTHOOK: query: insert into table ice_int values (1, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int +PREHOOK: query: insert into table ice_int values (2, 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int +POSTHOOK: query: insert into table ice_int values (2, 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int +PREHOOK: query: insert into table ice_int values (3, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int +POSTHOOK: query: insert into table ice_int values (3, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int +PREHOOK: query: insert into table ice_int values (4, 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int +POSTHOOK: query: insert into table ice_int values (4, 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int +PREHOOK: query: insert into table ice_int values (1, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int +POSTHOOK: query: insert into table ice_int values (1, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int +PREHOOK: query: insert into table ice_int values (2, 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int +POSTHOOK: query: insert into table ice_int values (2, 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int +PREHOOK: query: alter table ice_int drop partition (a <= 2), partition (a >= 3) +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@ice_int +PREHOOK: Output: default@ice_int@a=1 +PREHOOK: Output: default@ice_int@a=2 +PREHOOK: Output: default@ice_int@a=3 +PREHOOK: Output: default@ice_int@a=4 +POSTHOOK: query: alter table ice_int drop partition (a <= 2), partition (a >= 3) +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@ice_int +POSTHOOK: Output: default@ice_int@a=1 +POSTHOOK: Output: default@ice_int@a=2 +POSTHOOK: Output: default@ice_int@a=3 +POSTHOOK: Output: default@ice_int@a=4 +PREHOOK: query: select * from ice_int +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_int +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_int +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_int +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: create table ice_str (a int, b string) partitioned by spec (b) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_str +POSTHOOK: query: create table ice_str (a int, b string) partitioned by spec (b) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_str +PREHOOK: query: insert into table ice_str values (1, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str +POSTHOOK: query: insert into table ice_str values (1, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str +PREHOOK: query: insert into table ice_str values (2, 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str +POSTHOOK: query: insert into table ice_str values (2, 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str +PREHOOK: query: insert into table ice_str values (3, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str +POSTHOOK: query: insert into table ice_str values (3, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str +PREHOOK: query: insert into table ice_str values (4, 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str +POSTHOOK: query: insert into table ice_str values (4, 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str +PREHOOK: query: insert into table ice_str values (1, 'ABC') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str +POSTHOOK: query: insert into table ice_str values (1, 'ABC') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str +PREHOOK: query: insert into table ice_str values (2, 'XYZ') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_str +POSTHOOK: query: insert into table ice_str values (2, 'XYZ') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_str +PREHOOK: query: alter table ice_str drop partition (b != 'ABC'), partition (b == 'XYZ') +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@ice_str +PREHOOK: Output: default@ice_str@b=DEF +PREHOOK: Output: default@ice_str@b=XYZ +POSTHOOK: query: alter table ice_str drop partition (b != 'ABC'), partition (b == 'XYZ') +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@ice_str +POSTHOOK: Output: default@ice_str@b=DEF +POSTHOOK: Output: default@ice_str@b=XYZ +PREHOOK: query: select * from ice_str +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_str +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_str +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_str +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 ABC +1 ABC +3 ABC +PREHOOK: query: create table ice_int_double_part (a int, b int, c int) partitioned by spec (a, b) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: create table ice_int_double_part (a int, b int, c int) partitioned by spec (a, b) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: insert into table ice_int_double_part values (1, 2, 3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: insert into table ice_int_double_part values (1, 2, 3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: insert into table ice_int_double_part values (2, 3, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: insert into table ice_int_double_part values (2, 3, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: insert into table ice_int_double_part values (3, 4, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: insert into table ice_int_double_part values (3, 4, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: insert into table ice_int_double_part values (4, 5, 6) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: insert into table ice_int_double_part values (4, 5, 6) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: insert into table ice_int_double_part values (1, 2, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: insert into table ice_int_double_part values (1, 2, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: insert into table ice_int_double_part values (2, 1, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: insert into table ice_int_double_part values (2, 1, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: alter table ice_int_double_part drop partition (a <= 2, b <= 1), partition (a >= 3, b != 4) +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@ice_int_double_part +PREHOOK: Output: default@ice_int_double_part@a=2/b=1 +PREHOOK: Output: default@ice_int_double_part@a=4/b=5 +POSTHOOK: query: alter table ice_int_double_part drop partition (a <= 2, b <= 1), partition (a >= 3, b != 4) +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@ice_int_double_part +POSTHOOK: Output: default@ice_int_double_part@a=2/b=1 +POSTHOOK: Output: default@ice_int_double_part@a=4/b=5 +PREHOOK: query: select * from ice_int_double_part +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_int_double_part +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_int_double_part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_int_double_part +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 2 3 +1 2 4 +2 3 4 +3 4 5 +PREHOOK: query: create table ice_date_int_double_part (a date, b int, c int) partitioned by spec (a, b) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: create table ice_date_int_double_part (a date, b int, c int) partitioned by spec (a, b) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: insert into table ice_date_int_double_part values ('2022-02-07', 2, 3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: insert into table ice_date_int_double_part values ('2022-02-07', 2, 3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: insert into table ice_date_int_double_part values ('2022-08-07', 3, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: insert into table ice_date_int_double_part values ('2022-08-07', 3, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: insert into table ice_date_int_double_part values ('2022-10-05', 4, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: insert into table ice_date_int_double_part values ('2022-10-05', 4, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: insert into table ice_date_int_double_part values ('2022-01-17', 5, 6) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: insert into table ice_date_int_double_part values ('2022-01-17', 5, 6) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: insert into table ice_date_int_double_part values ('2022-04-08', 2, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: insert into table ice_date_int_double_part values ('2022-04-08', 2, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: insert into table ice_date_int_double_part values ('2023-02-07', 1, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: insert into table ice_date_int_double_part values ('2023-02-07', 1, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: alter table ice_date_int_double_part drop partition (a <= '2023-02-07', b <= 1), partition (a >= '2022-08-07', b >= 2) +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@ice_date_int_double_part +PREHOOK: Output: default@ice_date_int_double_part@a=2022-08-07/b=3 +PREHOOK: Output: default@ice_date_int_double_part@a=2022-10-05/b=4 +PREHOOK: Output: default@ice_date_int_double_part@a=2023-02-07/b=1 +POSTHOOK: query: alter table ice_date_int_double_part drop partition (a <= '2023-02-07', b <= 1), partition (a >= '2022-08-07', b >= 2) +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@ice_date_int_double_part +POSTHOOK: Output: default@ice_date_int_double_part@a=2022-08-07/b=3 +POSTHOOK: Output: default@ice_date_int_double_part@a=2022-10-05/b=4 +POSTHOOK: Output: default@ice_date_int_double_part@a=2023-02-07/b=1 +PREHOOK: query: select * from ice_date_int_double_part +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date_int_double_part +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_date_int_double_part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date_int_double_part +POSTHOOK: Output: hdfs://### HDFS PATH ### +2022-01-17 5 6 +2022-02-07 2 3 +2022-04-08 2 4 +PREHOOK: query: create table ice_date_double_double_part (a date, b double, c int) partitioned by spec (a, b) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: create table ice_date_double_double_part (a date, b double, c int) partitioned by spec (a, b) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: insert into table ice_date_double_double_part values ('2022-02-07', 2.75, 3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: insert into table ice_date_double_double_part values ('2022-02-07', 2.75, 3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: insert into table ice_date_double_double_part values ('2022-08-07', 3.25, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: insert into table ice_date_double_double_part values ('2022-08-07', 3.25, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: insert into table ice_date_double_double_part values ('2022-10-05', 4.23, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: insert into table ice_date_double_double_part values ('2022-10-05', 4.23, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: insert into table ice_date_double_double_part values ('2022-01-17', 5.67, 6) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: insert into table ice_date_double_double_part values ('2022-01-17', 5.67, 6) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: insert into table ice_date_double_double_part values ('2022-04-08', 2.45, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: insert into table ice_date_double_double_part values ('2022-04-08', 2.45, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: insert into table ice_date_double_double_part values ('2023-02-07', 1.08, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: insert into table ice_date_double_double_part values ('2023-02-07', 1.08, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: alter table ice_date_double_double_part drop partition (a <= '2023-02-07', b <= 1.09), partition (a >= '2022-08-07', b >= 2.78) +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@ice_date_double_double_part +PREHOOK: Output: default@ice_date_double_double_part@a=2022-08-07/b=3.25 +PREHOOK: Output: default@ice_date_double_double_part@a=2022-10-05/b=4.23 +PREHOOK: Output: default@ice_date_double_double_part@a=2023-02-07/b=1.08 +POSTHOOK: query: alter table ice_date_double_double_part drop partition (a <= '2023-02-07', b <= 1.09), partition (a >= '2022-08-07', b >= 2.78) +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@ice_date_double_double_part +POSTHOOK: Output: default@ice_date_double_double_part@a=2022-08-07/b=3.25 +POSTHOOK: Output: default@ice_date_double_double_part@a=2022-10-05/b=4.23 +POSTHOOK: Output: default@ice_date_double_double_part@a=2023-02-07/b=1.08 +PREHOOK: query: select * from ice_date_int_double_part +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date_int_double_part +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_date_int_double_part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date_int_double_part +POSTHOOK: Output: hdfs://### HDFS PATH ### +2022-01-17 5 6 +2022-02-07 2 3 +2022-04-08 2 4 +PREHOOK: query: create table ice_date_bigint_double_part (a date, b bigint, c int) partitioned by spec (a, b) stored by iceberg +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: create table ice_date_bigint_double_part (a date, b bigint, c int) partitioned by spec (a, b) stored by iceberg +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: insert into table ice_date_bigint_double_part values ('2022-02-07', 267859937678997886, 3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: insert into table ice_date_bigint_double_part values ('2022-02-07', 267859937678997886, 3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: insert into table ice_date_bigint_double_part values ('2022-08-07', 325678599459970774, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: insert into table ice_date_bigint_double_part values ('2022-08-07', 325678599459970774, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: insert into table ice_date_bigint_double_part values ('2022-10-05', 423789504756478599, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: insert into table ice_date_bigint_double_part values ('2022-10-05', 423789504756478599, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: insert into table ice_date_bigint_double_part values ('2022-01-17', 567890387564883960, 6) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: insert into table ice_date_bigint_double_part values ('2022-01-17', 567890387564883960, 6) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: insert into table ice_date_bigint_double_part values ('2022-04-08', 245789600487678594, 4) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: insert into table ice_date_bigint_double_part values ('2022-04-08', 245789600487678594, 4) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: insert into table ice_date_bigint_double_part values ('2023-02-07', 108789600487566478, 5) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: insert into table ice_date_bigint_double_part values ('2023-02-07', 108789600487566478, 5) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date_bigint_double_part +PREHOOK: query: alter table ice_date_bigint_double_part drop partition (a <= '2023-02-07', b <= 109000000000000000L), partition (a >= '2022-08-07', b >= 278000000000000000L) +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@ice_date_bigint_double_part +PREHOOK: Output: default@ice_date_bigint_double_part@a=2022-08-07/b=325678599459970774 +PREHOOK: Output: default@ice_date_bigint_double_part@a=2022-10-05/b=423789504756478599 +PREHOOK: Output: default@ice_date_bigint_double_part@a=2023-02-07/b=108789600487566478 +POSTHOOK: query: alter table ice_date_bigint_double_part drop partition (a <= '2023-02-07', b <= 109000000000000000L), partition (a >= '2022-08-07', b >= 278000000000000000L) +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@ice_date_bigint_double_part +POSTHOOK: Output: default@ice_date_bigint_double_part@a=2022-08-07/b=325678599459970774 +POSTHOOK: Output: default@ice_date_bigint_double_part@a=2022-10-05/b=423789504756478599 +POSTHOOK: Output: default@ice_date_bigint_double_part@a=2023-02-07/b=108789600487566478 +PREHOOK: query: select * from ice_date_bigint_double_part +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date_bigint_double_part +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_date_bigint_double_part +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date_bigint_double_part +POSTHOOK: Output: hdfs://### HDFS PATH ### +2022-01-17 567890387564883960 6 +2022-02-07 267859937678997886 3 +2022-04-08 245789600487678594 4 +PREHOOK: query: drop table ice_int +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_int +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_int +POSTHOOK: query: drop table ice_int +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_int +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_int +PREHOOK: query: drop table ice_str +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_str +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_str +POSTHOOK: query: drop table ice_str +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_str +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_str +PREHOOK: query: drop table ice_int_double_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_int_double_part +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_int_double_part +POSTHOOK: query: drop table ice_int_double_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_int_double_part +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_int_double_part +PREHOOK: query: drop table ice_date_int_double_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_date_int_double_part +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_int_double_part +POSTHOOK: query: drop table ice_date_int_double_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_date_int_double_part +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_int_double_part +PREHOOK: query: drop table ice_date_double_double_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_date_double_double_part +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_double_double_part +POSTHOOK: query: drop table ice_date_double_double_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_date_double_double_part +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_double_double_part +PREHOOK: query: drop table ice_date_bigint_double_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@ice_date_bigint_double_part +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date_bigint_double_part +POSTHOOK: query: drop table ice_date_bigint_double_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@ice_date_bigint_double_part +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date_bigint_double_part diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 51133a3636c5..5022b77fc368 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -4491,25 +4491,30 @@ public boolean getPartitionsByExpr(Table tbl, ExprNodeDesc expr, HiveConf conf, perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_BY_EXPR); try { Preconditions.checkNotNull(partitions); - byte[] exprBytes = SerializationUtilities.serializeObjectWithTypeInformation(expr); String defaultPartitionName = HiveConf.getVar(conf, ConfVars.DEFAULTPARTITIONNAME); - List msParts = - new ArrayList<>(); - ValidWriteIdList validWriteIdList = null; + if (tbl.getStorageHandler() != null && tbl.getStorageHandler().alwaysUnpartitioned()) { + partitions.addAll(tbl.getStorageHandler().getPartitionsByExpr(tbl, expr)); + return false; + } else { + byte[] exprBytes = SerializationUtilities.serializeObjectWithTypeInformation(expr); + List msParts = + new ArrayList<>(); + ValidWriteIdList validWriteIdList = null; - PartitionsByExprRequest req = buildPartitionByExprRequest(tbl, exprBytes, defaultPartitionName, conf, - null); + PartitionsByExprRequest req = buildPartitionByExprRequest(tbl, exprBytes, defaultPartitionName, conf, + null); - if (AcidUtils.isTransactionalTable(tbl)) { - validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); - req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); - req.setId(tbl.getTTable().getId()); - } + if (AcidUtils.isTransactionalTable(tbl)) { + validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); + req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(tbl.getTTable().getId()); + } - boolean hasUnknownParts = getMSC().listPartitionsSpecByExpr(req, msParts); - partitions.addAll(convertFromPartSpec(msParts.iterator(), tbl)); + boolean hasUnknownParts = getMSC().listPartitionsSpecByExpr(req, msParts); + partitions.addAll(convertFromPartSpec(msParts.iterator(), tbl)); - return hasUnknownParts; + return hasUnknownParts; + } } finally { perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_BY_EXPR, "HS2-cache"); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index e05ecfb5a50b..4548b067364a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -728,4 +728,14 @@ default boolean canPerformMetadataDelete(org.apache.hadoop.hive.ql.metadata.Tabl SearchArgument searchArgument) { return false; } + default List getPartitionKeys(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { + throw new UnsupportedOperationException("Storage handler does not support getting partition keys " + + "for a table."); + } + + default List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, ExprNodeDesc desc) + throws SemanticException { + throw new UnsupportedOperationException("Storage handler does not support getting partitions by expression " + + "for a table."); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java index 9eefe23d1470..a246e3af2a84 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java @@ -565,7 +565,9 @@ public static Map> getFullPartitionSpecs( CommonTree ast, Table table, Configuration conf, boolean canGroupExprs) throws SemanticException { String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME); Map colTypes = new HashMap<>(); - for (FieldSchema fs : table.getPartitionKeys()) { + List partitionKeys = table.getStorageHandler() != null && table.getStorageHandler().alwaysUnpartitioned() ? + table.getStorageHandler().getPartitionKeys(table) : table.getPartitionKeys(); + for (FieldSchema fs : partitionKeys) { colTypes.put(fs.getName().toLowerCase(), fs.getType()); } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java index 5986015a5138..8a0970a474b3 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.metastore; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hive.metastore.api.EnvironmentContext; @@ -197,4 +198,16 @@ default boolean createHMSTableInHook() { default void postGetTable(Table table) { // Do nothing } + + /** + * Called before dropping the partitions from the table in the metastore during ALTER TABLE DROP PARTITION. + * @param table table whose partition needs to be dropped + * @param context context of the operation + * @param partExprs List of partition expressions + * @throws MetaException + */ + default void preDropPartitions(Table table, + EnvironmentContext context, List> partExprs) throws MetaException { + // Do nothing + } } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index 71a8dfb62008..116d2a415982 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -120,6 +120,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable { public static final String MANUALLY_INITIATED_COMPACTION = "manual"; public static final String TRUNCATE_SKIP_DATA_DELETION = "truncateSkipDataDeletion"; + public static final String SKIP_DROP_PARTITION = "dropPartitionSkip"; public static final String RENAME_PARTITION_MAKE_COPY = "renamePartitionMakeCopy"; /** @@ -1931,6 +1932,16 @@ public List dropPartitions(String catName, String dbName, String tblN PartitionDropOptions options) throws TException { RequestPartsSpec rps = new RequestPartsSpec(); List exprs = new ArrayList<>(partExprs.size()); + Table table = getTable(catName, dbName, tblName); + HiveMetaHook hook = getHook(table); + EnvironmentContext context = new EnvironmentContext(); + if (hook != null) { + hook.preDropPartitions(table, context, partExprs); + } + if (context.getProperties() != null && + Boolean.parseBoolean(context.getProperties().get(SKIP_DROP_PARTITION))) { + return Lists.newArrayList(); + } for (Pair partExpr : partExprs) { DropPartitionsExpr dpe = new DropPartitionsExpr(); dpe.setExpr(partExpr.getRight()); @@ -1944,10 +1955,9 @@ public List dropPartitions(String catName, String dbName, String tblN req.setNeedResult(options.returnResults); req.setIfExists(options.ifExists); - EnvironmentContext context = null; if (options.purgeData) { LOG.info("Dropped partitions will be purged!"); - context = getEnvironmentContextWithIfPurgeSet(); + context.putToProperties("ifPurge", "true"); } if (options.writeId != null) { context = Optional.ofNullable(context).orElse(new EnvironmentContext()); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index feb9701fac49..dfabec77ac5f 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -5013,7 +5013,8 @@ static boolean isMustPurge(EnvironmentContext envContext, Table tbl) { // 3. Either // 3.1. User has specified PURGE from the commandline, and if not, // 3.2. User has set the table to auto-purge. - return ((envContext != null) && Boolean.parseBoolean(envContext.getProperties().get("ifPurge"))) + return (envContext != null && envContext.getProperties() != null + && Boolean.parseBoolean(envContext.getProperties().get("ifPurge"))) || MetaStoreUtils.isSkipTrash(tbl.getParameters()); } From 39099d4c3fba46cd9d6d755c55fae262257ef6bc Mon Sep 17 00:00:00 2001 From: rtrivedi12 <32664785+rtrivedi12@users.noreply.github.com> Date: Mon, 6 Nov 2023 05:54:48 +0530 Subject: [PATCH 039/179] HIVE-27113: Increasing default for hive.thrift.client.max.message.size to 2147483647 bytes (#4704). (Riju Trivedi, Reviewed by Ayush Saxena, John Sherman, Naveen Gangam) --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 8 ++++---- .../cli/thrift/RetryingThriftCLIServiceClient.java | 3 ++- .../apache/hadoop/hive/metastore/HiveMetaStoreClient.java | 3 ++- .../apache/hadoop/hive/metastore/conf/MetastoreConf.java | 7 ++++--- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index a8378f097d39..3e3737ad7574 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2944,10 +2944,10 @@ public static enum ConfVars { HIVE_STATS_MAX_NUM_STATS("hive.stats.max.num.stats", (long) 10000, "When the number of stats to be updated is huge, this value is used to control the number of \n" + " stats to be sent to HMS for update."), - HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE("hive.thrift.client.max.message.size", "1gb", - new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true), - "Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " + - "library. The upper limit is 2147483648 bytes (or 2gb)."), + HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE("hive.thrift.client.max.message.size", "2147483647b", + new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true), + "Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " + + "library. The upper limit is 2147483647 bytes."), // Concurrency HIVE_SUPPORT_CONCURRENCY("hive.support.concurrency", false, "Whether Hive supports concurrency control or not. \n" + diff --git a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java index 9079c652d0a8..e75922287ecc 100644 --- a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java +++ b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java @@ -311,7 +311,8 @@ protected synchronized TTransport connect(HiveConf conf) throws HiveSQLException String host = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST); int port = conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT); int maxThriftMessageSize = (int) conf.getSizeVar(HiveConf.ConfVars.HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE); - LOG.info("Connecting to " + host + ":" + port); + LOG.info("Connecting to {}:{} using a thrift max message of size: {}", + host, port, maxThriftMessageSize); transport = HiveAuthUtils.getSocketTransport(host, port, 0, maxThriftMessageSize); ((TSocket) transport).setTimeout((int) conf.getTimeVar(HiveConf.ConfVars.SERVER_READ_SOCKET_TIMEOUT, diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index 116d2a415982..9b7cbb82df67 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -612,7 +612,8 @@ public void renamePartition(String catName, String dbname, String tableName, Lis } private T configureThriftMaxMessageSize(T transport) { - int maxThriftMessageSize = (int) MetastoreConf.getSizeVar(conf, ConfVars.THRIFT_METASTORE_CLIENT_MAX_MESSAGE_SIZE); + int maxThriftMessageSize = (int) MetastoreConf.getSizeVar( + conf, ConfVars.THRIFT_METASTORE_CLIENT_MAX_MESSAGE_SIZE); if (maxThriftMessageSize > 0) { if (transport.getConfiguration() == null) { LOG.warn("TTransport {} is returning a null Configuration, Thrift max message size is not getting configured", diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index 7208867eaa4f..65449d4362c3 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -1496,9 +1496,10 @@ public enum ConfVars { " corresponding service discovery servers e.g. a zookeeper. Otherwise they are " + "used as URIs for remote metastore."), THRIFT_METASTORE_CLIENT_MAX_MESSAGE_SIZE("metastore.thrift.client.max.message.size", - "hive.thrift.client.max.message.size", "1gb", new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true), - "Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " + - "library. The upper limit is 2147483648 bytes (or 2gb)."), + "hive.thrift.client.max.message.size", "2147483647b", + new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true), + "Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " + + "library. The upper limit is 2147483647 bytes"), THRIFT_SERVICE_DISCOVERY_MODE("metastore.service.discovery.mode", "hive.metastore.service.discovery.mode", "", From 698194183a2cb4ee6bc528ee71462cbae2c2c7ad Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Thu, 2 Nov 2023 12:52:41 +0100 Subject: [PATCH 040/179] HIVE-27846: HIVE-27849: Tests under hive-unit module are not running (Stamatis Zampetakis reviewed by Ayush Saxena) 1. Drop jupiter-engine dependency from hive-unit module preventing tests from running. 2. Remove redundant jupiter annotations from TestStorageSchemaReader to avoid dependencies to jupiter API. 3. Revert ivy upgrade cause since it leads to failures in replication tests: * TestReplAcrossInstancesWithJsonMessageFormat; * TestReplicationScenariosAcrossInstances tests. 4. Disable TestCrudCompactorOnTez#secondCompactionShouldBeRefusedBeforeEnqueueing test till HIVE-27848 is resolved. Close apache/hive#4850 --- itests/hive-unit/pom.xml | 5 ----- .../hadoop/hive/metastore/TestStorageSchemaReader.java | 6 ++---- .../hive/ql/txn/compactor/TestCrudCompactorOnTez.java | 2 ++ pom.xml | 2 +- 4 files changed, 5 insertions(+), 10 deletions(-) diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index 09d4f5712f90..29ece2167223 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -208,11 +208,6 @@ junit test - - org.junit.jupiter - junit-jupiter-engine - test - org.apache.hadoop hadoop-hdfs diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestStorageSchemaReader.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestStorageSchemaReader.java index c6e1f1a4b0d1..fadd793bc23a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestStorageSchemaReader.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestStorageSchemaReader.java @@ -36,8 +36,6 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import java.util.HashMap; import java.util.List; @@ -59,7 +57,7 @@ public class TestStorageSchemaReader { Map jdbcTableParams = new HashMap<>(); Map jdbcSerdeParams = new HashMap<>(); - @Before @BeforeEach public void setUp() throws Exception { + @Before public void setUp() throws Exception { dbName = "sampleDb"; hiveConf = new HiveConf(this.getClass()); new DatabaseBuilder().setName(dbName).create(new HiveMetaStoreClient(hiveConf), hiveConf); @@ -83,7 +81,7 @@ public class TestStorageSchemaReader { jdbcTableParams.put("storage_handler", "org.apache.hive.storage.jdbc.JdbcStorageHandler"); } - @After @AfterEach public void tearDown() throws Exception { + @After public void tearDown() throws Exception { new HiveMetaStoreClient(hiveConf).dropDatabase(dbName, true, true, true); } diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java index e72811e103bf..295bbdd09f79 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java @@ -77,6 +77,7 @@ import org.apache.orc.TypeDescription; import org.apache.orc.impl.RecordReaderImpl; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -601,6 +602,7 @@ public void testCompactionShouldNotFailOnPartitionsWithBooleanField() throws Exc } @Test + @Ignore("HIVE-27848") public void secondCompactionShouldBeRefusedBeforeEnqueueing() throws Exception { conf.setBoolVar(HiveConf.ConfVars.COMPACTOR_CRUD_QUERY_BASED, true); diff --git a/pom.xml b/pom.xml index bf8a5d3e5fd9..c163ce1d3a29 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 4.5.13 4.4.13 - 2.5.2 + 2.5.1 2.13.5 2.3.4 2.4.1 From 3b2e8a9f4a120e269abccdb93a649cf3132ae5de Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 7 Nov 2023 07:34:06 +0530 Subject: [PATCH 041/179] HIVE-27789: Iceberg: Add a way to expire snapshots with retain last. (#4835). (Ayush Saxena, reviewed by Denys Kuzmenko, zhangbutao) --- .../mr/hive/HiveIcebergStorageHandler.java | 11 ++++++++++ .../hive/TestHiveIcebergExpireSnapshots.java | 21 +++++++++++++++++++ .../hadoop/hive/ql/parse/AlterClauseParser.g | 2 ++ .../execute/AlterTableExecuteAnalyzer.java | 8 ++++++- .../hive/ql/parse/AlterTableExecuteSpec.java | 16 ++++++++++++++ 5 files changed, 57 insertions(+), 1 deletion(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 0afdd79ba55b..538c7316779c 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -867,6 +867,8 @@ private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnap expireSnapshotsSpec.getTimestampMillis(), deleteExecutorService); } else if (expireSnapshotsSpec.isExpireByIds()) { expireSnapshotByIds(icebergTable, expireSnapshotsSpec.getIdsToExpire(), deleteExecutorService); + } else if (expireSnapshotsSpec.isExpireByRetainLast()) { + expireSnapshotRetainLast(icebergTable, expireSnapshotsSpec.getNumRetainLast(), deleteExecutorService); } else { expireSnapshotOlderThanTimestamp(icebergTable, expireSnapshotsSpec.getTimestampMillis(), deleteExecutorService); } @@ -877,6 +879,15 @@ private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnap } } + private void expireSnapshotRetainLast(Table icebergTable, int numRetainLast, ExecutorService deleteExecutorService) { + ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots(); + expireSnapshots.retainLast(numRetainLast); + if (deleteExecutorService != null) { + expireSnapshots.executeDeleteWith(deleteExecutorService); + } + expireSnapshots.commit(); + } + private void expireSnapshotByTimestampRange(Table icebergTable, Long fromTimestamp, Long toTimestamp, ExecutorService deleteExecutorService) { ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots(); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java index a851578ee6c6..8fd808f4fed1 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java @@ -28,6 +28,8 @@ import org.junit.Assert; import org.junit.Test; +import static org.apache.iceberg.TableProperties.MAX_SNAPSHOT_AGE_MS; + /** * Tests covering the rollback feature */ @@ -82,4 +84,23 @@ public void testExpireSnapshotsWithTimestampRange() throws IOException, Interrup table.refresh(); Assert.assertEquals(6, IterableUtils.size(table.snapshots())); } + + @Test + public void testExpireSnapshotsWithRetainLast() throws IOException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "source"); + Table table = testTables.createTableWithVersions(shell, identifier.name(), + HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 10); + // No snapshot should expire, since the max snapshot age to expire is by default 5 days + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE EXPIRE_SNAPSHOTS RETAIN LAST 5"); + table.refresh(); + Assert.assertEquals(10, IterableUtils.size(table.snapshots())); + + // Change max snapshot age to expire to 1 ms & re-execute, this time it should retain only 5 + shell.executeStatement( + "ALTER TABLE " + identifier.name() + " SET TBLPROPERTIES('" + MAX_SNAPSHOT_AGE_MS + "'='1')"); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE EXPIRE_SNAPSHOTS RETAIN LAST 5"); + table.refresh(); + Assert.assertEquals(5, IterableUtils.size(table.snapshots())); + } } diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g index 28d60d6262f2..a184b41e0f44 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g @@ -485,6 +485,8 @@ alterStatementSuffixExecute -> ^(TOK_ALTERTABLE_EXECUTE KW_CHERRY_PICK $snapshotId) | KW_EXECUTE KW_EXPIRE_SNAPSHOTS KW_BETWEEN (fromTimestamp=StringLiteral) KW_AND (toTimestamp=StringLiteral) -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS $fromTimestamp $toTimestamp) + | KW_EXECUTE KW_EXPIRE_SNAPSHOTS KW_RETAIN KW_LAST numToRetain=Number + -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS KW_RETAIN $numToRetain) ; alterStatementSuffixDropBranch diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java index 8d4a902b56b4..cdd6f035d4d3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java @@ -54,6 +54,7 @@ import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.SET_CURRENT_SNAPSHOT; import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.RollbackSpec.RollbackType.TIME; import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.RollbackSpec.RollbackType.VERSION; +import static org.apache.hadoop.hive.ql.parse.HiveLexer.KW_RETAIN; /** * Analyzer for ALTER TABLE ... EXECUTE commands. @@ -141,7 +142,12 @@ private static AlterTableExecuteDesc getExpireSnapshotDesc(TableName tableName, SessionState.get().getConf().getLocalTimeZone(); ASTNode firstNode = (ASTNode) children.get(1); String firstNodeText = PlanUtils.stripQuotes(firstNode.getText().trim()); - if (children.size() == 3) { + if (firstNode.getType() == KW_RETAIN) { + ASTNode numRetainLastNode = (ASTNode) children.get(2); + String numToRetainText = PlanUtils.stripQuotes(numRetainLastNode.getText()); + int numToRetain = Integer.parseInt(numToRetainText); + spec = new AlterTableExecuteSpec(EXPIRE_SNAPSHOT, new ExpireSnapshotsSpec(numToRetain)); + } else if (children.size() == 3) { ASTNode secondNode = (ASTNode) children.get(2); String secondNodeText = PlanUtils.stripQuotes(secondNode.getText().trim()); TimestampTZ fromTime = TimestampTZUtil.parse(firstNodeText, timeZone); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java index b3c8edd9d876..2b7ca285e1ca 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java @@ -113,6 +113,8 @@ public static class ExpireSnapshotsSpec { private long fromTimestampMillis = -1L; + private int numRetainLast = -1; + public ExpireSnapshotsSpec(long timestampMillis) { this.timestampMillis = timestampMillis; } @@ -126,6 +128,10 @@ public ExpireSnapshotsSpec(long fromTimestampMillis, long toTimestampMillis) { this.timestampMillis = toTimestampMillis; } + public ExpireSnapshotsSpec(int numRetainLast) { + this.numRetainLast = numRetainLast; + } + public Long getTimestampMillis() { return timestampMillis; } @@ -138,6 +144,10 @@ public String[] getIdsToExpire() { return idsToExpire; } + public int getNumRetainLast() { + return numRetainLast; + } + public boolean isExpireByIds() { return idsToExpire != null; } @@ -146,6 +156,10 @@ public boolean isExpireByTimestampRange() { return timestampMillis != -1 && fromTimestampMillis != -1; } + public boolean isExpireByRetainLast() { + return numRetainLast != -1; + } + @Override public String toString() { MoreObjects.ToStringHelper stringHelper = MoreObjects.toStringHelper(this); @@ -153,6 +167,8 @@ public String toString() { stringHelper.add("fromTimestampMillis", fromTimestampMillis).add("toTimestampMillis", timestampMillis); } else if (isExpireByIds()) { stringHelper.add("idsToExpire", Arrays.toString(idsToExpire)); + } else if (isExpireByRetainLast()) { + stringHelper.add("numRetainLast", numRetainLast); } else { stringHelper.add("timestampMillis", timestampMillis); } From 9387a2cc48e54ff27b7809d6ce67c61f66f16f7b Mon Sep 17 00:00:00 2001 From: Soumyakanti Das Date: Tue, 7 Nov 2023 01:34:02 -0800 Subject: [PATCH 042/179] HIVE-27531: Unparse identifiers in show create table output (Soumyakanti Das, reviewed by Krisztian Kasa) --- .../test/results/positive/ctlt_iceberg.q.out | 6 +- .../results/positive/iceberg_v2_deletes.q.out | 2 +- .../positive/show_create_iceberg_table.q.out | 30 +- .../create/show/ShowCreateTableOperation.java | 3 +- .../hadoop/hive/ql/exec/DDLPlanUtils.java | 204 +++-- .../hadoop/hive/ql/exec/ExplainTask.java | 7 +- .../hadoop/hive/ql/metadata/HiveUtils.java | 4 + .../clientpositive/partition_explain_ddl.q | 18 +- .../clientpositive/show_create_table.q | 4 +- .../llap/constraints_explain_ddl.q.out | 798 +++++++++--------- .../llap/create_table_explain_ddl.q.out | 126 +-- .../llap/explain_ddl_nested_part.q.out | 366 ++++---- .../llap/partition_explain_ddl.q.out | 392 ++++++--- .../clientpositive/llap/quotedid_basic.q.out | 4 +- .../llap/quotedid_basic_standard.q.out | 8 +- .../llap/show_create_table.q.out | 38 +- .../llap/show_create_table_alter.q.out | 30 +- .../llap/show_create_table_db_table.q.out | 4 +- .../clientpositive/llap/table_storage.q.out | 20 +- .../llap/views_explain_ddl.q.out | 36 +- 20 files changed, 1136 insertions(+), 964 deletions(-) diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out index e93f606a141c..d61952f476fb 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out @@ -113,7 +113,7 @@ CREATE EXTERNAL TABLE `emp_iceberg`( `id` int, `company` string) PARTITIONED BY SPEC ( -company) +`company`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY @@ -150,7 +150,7 @@ CREATE EXTERNAL TABLE `emp_like1`( `id` int, `company` string) PARTITIONED BY SPEC ( -company) +`company`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY @@ -216,7 +216,7 @@ CREATE EXTERNAL TABLE `emp_like2`( `id` int, `company` string) PARTITIONED BY SPEC ( -company) +`company`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out index d21ccfe732b4..18c9a4794a6b 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out @@ -267,7 +267,7 @@ CREATE EXTERNAL TABLE `icepart01`( `id` int, `part` int) PARTITIONED BY SPEC ( -part) +`part`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out index c6e0cddcdef3..5770b87e7fce 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out @@ -69,13 +69,13 @@ CREATE EXTERNAL TABLE `ice_t_transform`( `bucket_field` int, `identity_field` int) PARTITIONED BY SPEC ( -YEAR(year_field), -MONTH(month_field), -DAY(day_field), -HOUR(hour_field), -TRUNCATE(2, truncate_field), -BUCKET(2, bucket_field), -identity_field) +YEAR(`year_field`), +MONTH(`month_field`), +DAY(`day_field`), +HOUR(`hour_field`), +TRUNCATE(2, `truncate_field`), +BUCKET(2, `bucket_field`), +`identity_field`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY @@ -124,13 +124,13 @@ CREATE EXTERNAL TABLE `ice_t_transform_prop`( `bucket_field` int, `identity_field` int) PARTITIONED BY SPEC ( -YEAR(year_field), -MONTH(month_field), -DAY(day_field), -HOUR(hour_field), -TRUNCATE(2, truncate_field), -BUCKET(2, bucket_field), -identity_field) +YEAR(`year_field`), +MONTH(`month_field`), +DAY(`day_field`), +HOUR(`hour_field`), +TRUNCATE(2, `truncate_field`), +BUCKET(2, `bucket_field`), +`identity_field`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY @@ -173,7 +173,7 @@ CREATE EXTERNAL TABLE `ice_t_identity_part`( `a` int, `b` string) PARTITIONED BY SPEC ( -b) +`b`) ROW FORMAT SERDE 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' STORED BY diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/create/show/ShowCreateTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/create/show/ShowCreateTableOperation.java index 456ba0b42bca..7b8cc1f2ba49 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/create/show/ShowCreateTableOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/create/show/ShowCreateTableOperation.java @@ -62,8 +62,7 @@ public int execute() throws HiveException { if (primaryKeyStmt != null) { commands.add(primaryKeyStmt); } - commands.addAll(ddlObj.populateConstraints(table, - new HashSet<>(Collections.singletonList(table.getTableName())))); + commands.addAll(ddlObj.populateConstraints(table)); command = String.join("\n", commands); } outStream.write(command.getBytes(StandardCharsets.UTF_8)); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java index 91a6ed906038..10feb13045ff 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java @@ -60,8 +60,10 @@ import org.apache.hadoop.hive.ql.parse.TransformSpec; import org.apache.hadoop.hive.ql.util.DirectionUtils; import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; @@ -80,8 +82,10 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; +import java.util.stream.Collectors; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE; +import static org.apache.hadoop.hive.ql.metadata.HiveUtils.unparseIdentifier; public class DDLPlanUtils { private static final String EXTERNAL = "external"; @@ -133,8 +137,8 @@ public class DDLPlanUtils { private static final String CREATE_DATABASE_STMT = "CREATE DATABASE IF NOT EXISTS <" + DATABASE_NAME + ">;"; private final String CREATE_TABLE_TEMPLATE = - "CREATE <" + TEMPORARY + "><" + EXTERNAL + ">TABLE `<" + DATABASE_NAME + ">`." - + "`<" + TABLE_NAME + ">`(\n" + + "CREATE <" + TEMPORARY + "><" + EXTERNAL + ">TABLE <" + DATABASE_NAME + ">." + + "<" + TABLE_NAME + ">(\n" + "<" + LIST_COLUMNS + ">)\n" + "<" + COMMENT + ">\n" + "<" + PARTITIONS + ">\n" + @@ -147,8 +151,8 @@ public class DDLPlanUtils { "<" + PROPERTIES + ">)"; private static final String CREATE_VIEW_TEMPLATE = - "CREATE VIEW `<" + DATABASE_NAME + ">`.`<" + TABLE_NAME + - ">`<" + PARTITIONS + "> AS <" + SQL + ">"; + "CREATE VIEW <" + DATABASE_NAME + ">.<" + TABLE_NAME + + "><" + PARTITIONS + "> AS <" + SQL + ">"; private final String CREATE_TABLE_TEMPLATE_LOCATION = "LOCATION\n" + "<" + LOCATION + ">\n"; @@ -230,7 +234,7 @@ public class DDLPlanUtils { + TABLE_NAME + "> PARTITION <" + PARTITION_NAME + "> FOR COLUMN <" + COLUMN_NAME + "> BUT IT IS NOT SUPPORTED YET. THE BASE64 VALUE FOR THE HISTOGRAM IS <" + BASE_64_VALUE + "> "; - + /** * Returns the create database query for a give database name. * @@ -238,13 +242,13 @@ public class DDLPlanUtils { * @return */ public List getCreateDatabaseStmt(Set dbNames) { - List allDbStmt = new ArrayList(); + List allDbStmt = new ArrayList<>(); for (String dbName : dbNames) { if (dbName.equals("default")) { continue; } ST command = new ST(CREATE_DATABASE_STMT); - command.add(DATABASE_NAME, dbName); + command.add(DATABASE_NAME, unparseIdentifier(dbName)); allDbStmt.add(command.render()); } return allDbStmt; @@ -265,20 +269,35 @@ public List getTableColumnNames(Table tbl) { } public String getPartitionActualName(Partition pt) { - Map colTypeMap = getTableColumnsToType(pt.getTable()); + Map columnsToPrimitiveCategoryMap = + getPartitionColumnToPrimitiveCategory(pt); String[] partColsDef = pt.getName().split("/"); List ptParam = new ArrayList<>(); for (String partCol : partColsDef) { String[] colValue = partCol.split("="); - if (colTypeMap.get(colValue[0]).equalsIgnoreCase("string")) { - ptParam.add(colValue[0] + "='" + colValue[1] + "'"); - } else { - ptParam.add(colValue[0] + "=" + colValue[1]); - } + ptParam.add(unparseIdentifier(colValue[0]) + "=" + + TypeInfoUtils.convertStringToLiteralForSQL(colValue[1], columnsToPrimitiveCategoryMap.get(colValue[0]))); } return StringUtils.join(ptParam, ","); } + /** + * Get PrimitiveCategory for all partition columns + * + * @param pt Partition object + * @return Map of partition columns to PrimitiveCategory + */ + private Map getPartitionColumnToPrimitiveCategory(Partition pt) { + Map resultMap = new HashMap<>(); + for (FieldSchema schema: pt.getTable().getPartCols()) { + resultMap.put( + schema.getName(), + ((PrimitiveTypeInfo) TypeInfoUtils.getTypeInfoFromTypeString(schema.getType())).getPrimitiveCategory() + ); + } + return resultMap; + } + public boolean checkIfDefaultPartition(String pt) { if (pt.contains(HIVE_DEFAULT_PARTITION)) { return true; @@ -292,14 +311,13 @@ public boolean checkIfDefaultPartition(String pt) { * * @param pt * @return - * @throws MetaException */ //TODO: Adding/Updating Stats to Default Partition Not Allowed. Need to Fix Later - public String getAlterTableAddPartition(Partition pt) throws MetaException { + public String getAlterTableAddPartition(Partition pt) { Table tb = pt.getTable(); ST command = new ST(ALTER_TABLE_CREATE_PARTITION); - command.add(DATABASE_NAME, tb.getDbName()); - command.add(TABLE_NAME, tb.getTableName()); + command.add(DATABASE_NAME, unparseIdentifier(tb.getDbName())); + command.add(TABLE_NAME, unparseIdentifier(tb.getTableName())); command.add(PARTITION, getPartitionActualName(pt)); if (checkIfDefaultPartition(pt.getName())) { command.add(COMMENT_SQL, "--"); @@ -482,9 +500,9 @@ public String addAllColStats(ColumnStatisticsData columnStatisticsData) { */ public String getAlterTableStmtCol(ColumnStatisticsData columnStatisticsData, String colName, String tblName, String dbName) { ST command = new ST(ALTER_TABLE_UPDATE_STATISTICS_TABLE_COLUMN); - command.add(DATABASE_NAME, dbName); - command.add(TABLE_NAME, tblName); - command.add(COLUMN_NAME, colName); + command.add(DATABASE_NAME, unparseIdentifier(dbName)); + command.add(TABLE_NAME, unparseIdentifier(tblName)); + command.add(COLUMN_NAME, unparseIdentifier(colName)); command.add(TBLPROPERTIES, addAllColStats(columnStatisticsData)); return command.render(); } @@ -510,18 +528,18 @@ public List getAlterTableStmtTableStatsColsAll(Table tbl) String base64BitVector = checkBitVectors(statisticsObj.getStatsData()); if (base64BitVector != null) { ST command = new ST(EXIST_BIT_VECTORS); - command.add(DATABASE_NAME, tbl.getDbName()); - command.add(TABLE_NAME, tbl.getTableName()); - command.add(COLUMN_NAME, statisticsObj.getColName()); + command.add(DATABASE_NAME, unparseIdentifier(tbl.getDbName())); + command.add(TABLE_NAME, unparseIdentifier(tbl.getTableName())); + command.add(COLUMN_NAME, unparseIdentifier(statisticsObj.getColName())); command.add(BASE_64_VALUE, base64BitVector); alterTblStmt.add(command.render()); } String base64Histogram = checkHistogram(statisticsObj.getStatsData()); if (base64Histogram != null) { ST command = new ST(EXIST_HISTOGRAM); - command.add(DATABASE_NAME, tbl.getDbName()); - command.add(TABLE_NAME, tbl.getTableName()); - command.add(COLUMN_NAME, statisticsObj.getColName()); + command.add(DATABASE_NAME, unparseIdentifier(tbl.getDbName())); + command.add(TABLE_NAME, unparseIdentifier(tbl.getTableName())); + command.add(COLUMN_NAME, unparseIdentifier(statisticsObj.getColName())); command.add(BASE_64_VALUE, base64Histogram); alterTblStmt.add(command.render()); } @@ -542,9 +560,9 @@ public List getAlterTableStmtTableStatsColsAll(Table tbl) public String getAlterTableStmtPartitionColStat(ColumnStatisticsData columnStatisticsData, String colName, String tblName, String ptName, String dbName) { ST command = new ST(ALTER_TABLE_UPDATE_STATISTICS_PARTITION_COLUMN); - command.add(DATABASE_NAME, dbName); - command.add(TABLE_NAME, tblName); - command.add(COLUMN_NAME, colName); + command.add(DATABASE_NAME, unparseIdentifier(dbName)); + command.add(TABLE_NAME, unparseIdentifier(tblName)); + command.add(COLUMN_NAME, unparseIdentifier(colName)); command.add(PARTITION_NAME, ptName); command.add(TBLPROPERTIES, addAllColStats(columnStatisticsData)); if (checkIfDefaultPartition(ptName)) { @@ -566,7 +584,7 @@ public List getAlterTableStmtPartitionStatsColsAll(List alterTableStmt = new ArrayList(); + List alterTableStmt = new ArrayList<>(); ColumnStatisticsObj[] columnStatisticsObj = columnStatisticsObjList.toArray(new ColumnStatisticsObj[0]); for (ColumnStatisticsObj statisticsObj : columnStatisticsObj) { alterTableStmt.add(getAlterTableStmtPartitionColStat( @@ -574,20 +592,20 @@ public List getAlterTableStmtPartitionStatsColsAll(List parameters) { public String getAlterTableStmtPartitionStatsBasic(Partition pt) { Map parameters = pt.getParameters(); ST command = new ST(ALTER_TABLE_UPDATE_STATISTICS_PARTITION_BASIC); - command.add(DATABASE_NAME, pt.getTable().getDbName()); - command.add(TABLE_NAME, pt.getTable().getTableName()); + command.add(DATABASE_NAME, unparseIdentifier(pt.getTable().getDbName())); + command.add(TABLE_NAME, unparseIdentifier(pt.getTable().getTableName())); command.add(PARTITION_NAME, getPartitionActualName(pt)); command.add(TBLPROPERTIES, paramToValues(parameters)); if (checkIfDefaultPartition(pt.getName())) { @@ -664,8 +682,8 @@ public List getDDLPlanForPartitionWithStats(Table table, public String getAlterTableStmtTableStatsBasic(Table tbl) { Map parameters = tbl.getParameters(); ST command = new ST(ALTER_TABLE_UPDATE_STATISTICS_TABLE_BASIC); - command.add(TABLE_NAME, tbl.getTableName()); - command.add(DATABASE_NAME, tbl.getDbName()); + command.add(TABLE_NAME, unparseIdentifier(tbl.getTableName())); + command.add(DATABASE_NAME, unparseIdentifier(tbl.getDbName())); command.add(TBLPROPERTIES, paramToValues(parameters)); return command.render(); } @@ -675,17 +693,22 @@ public String getAlterTableStmtPrimaryKeyConstraint(PrimaryKeyInfo pr) { return null; } ST command = new ST(ALTER_TABLE_ADD_PRIMARY_KEY); - command.add(TABLE_NAME, pr.getTableName()); - command.add(DATABASE_NAME, pr.getDatabaseName()); - command.add(CONSTRAINT_NAME, pr.getConstraintName()); - command.add(COL_NAMES, String.join(",", pr.getColNames().values())); + command.add(TABLE_NAME, unparseIdentifier(pr.getTableName())); + command.add(DATABASE_NAME, unparseIdentifier(pr.getDatabaseName())); + command.add(CONSTRAINT_NAME, unparseIdentifier(pr.getConstraintName())); + command.add( + COL_NAMES, + pr.getColNames().values().stream() + .map(x -> unparseIdentifier(x)) + .collect(Collectors.joining(",")) + ); command.add(ENABLE, pr.getEnable()); command.add(VALIDATE, pr.getValidate()); command.add(RELY, pr.getRely()); return command.render(); } - public void getAlterTableStmtForeignKeyConstraint(ForeignKeyInfo fr, List constraints, Set allTableNames) { + public void getAlterTableStmtForeignKeyConstraint(ForeignKeyInfo fr, List constraints) { if (!ForeignKeyInfo.isNotEmpty(fr)) { return; } @@ -693,13 +716,13 @@ public void getAlterTableStmtForeignKeyConstraint(ForeignKeyInfo fr, List Map> uniqueConstraints = uq.getUniqueConstraints(); for (String key : uniqueConstraints.keySet()) { ST command = new ST(ALTER_TABLE_ADD_UNIQUE_CONSTRAINT); - command.add(DATABASE_NAME, uq.getDatabaseName()); - command.add(TABLE_NAME, uq.getTableName()); - command.add(CONSTRAINT_NAME, key); + command.add(DATABASE_NAME, unparseIdentifier(uq.getDatabaseName())); + command.add(TABLE_NAME, unparseIdentifier(uq.getTableName())); + command.add(CONSTRAINT_NAME, unparseIdentifier(key)); List colNames = new ArrayList<>(); for (UniqueConstraint.UniqueConstraintCol col : uniqueConstraints.get(key)) { colNames.add(col.colName); } - command.add(COLUMN_NAME, Joiner.on(",").join(colNames)); + command.add( + COLUMN_NAME, + colNames.stream() + .map(x -> unparseIdentifier(x)) + .collect(Collectors.joining(",")) + ); command.add(ENABLE, uniqueConstraints.get(key).get(0).enable); command.add(VALIDATE, uniqueConstraints.get(key).get(0).validate); command.add(RELY, uniqueConstraints.get(key).get(0).rely); @@ -740,9 +768,9 @@ public void getAlterTableStmtDefaultConstraint(DefaultConstraint dc, Table tb, L List defaultConstraintCols = defaultConstraints.get(constraintName); for (DefaultConstraintCol col : defaultConstraintCols) { ST command = new ST(ALTER_TABLE_ADD_DEFAULT_CONSTRAINT); - command.add(DATABASE_NAME, dc.getTableName()); - command.add(TABLE_NAME, dc.getTableName()); - command.add(COLUMN_NAME, col.colName); + command.add(DATABASE_NAME, unparseIdentifier(dc.getTableName())); + command.add(TABLE_NAME, unparseIdentifier(dc.getTableName())); + command.add(COLUMN_NAME, unparseIdentifier(col.colName)); command.add(COL_TYPE, colType.get(col.colName)); command.add(DEFAULT_VALUE, col.defaultVal); command.add(ENABLE, col.enable); @@ -763,9 +791,9 @@ public void getAlterTableStmtCheckConstraint(CheckConstraint ck, List co if (checkConstraintCols != null && checkConstraintCols.size() > 0) { for (CheckConstraintCol col : checkConstraintCols) { ST command = new ST(ALTER_TABLE_ADD_CHECK_CONSTRAINT); - command.add(DATABASE_NAME, ck.getDatabaseName()); - command.add(TABLE_NAME, ck.getTableName()); - command.add(CONSTRAINT_NAME, constraintName); + command.add(DATABASE_NAME, unparseIdentifier(ck.getDatabaseName())); + command.add(TABLE_NAME, unparseIdentifier(ck.getTableName())); + command.add(CONSTRAINT_NAME, unparseIdentifier(constraintName)); command.add(CHECK_EXPRESSION, col.getCheckExpression()); command.add(ENABLE, col.getEnable()); command.add(VALIDATE, col.getValidate()); @@ -786,11 +814,11 @@ public void getAlterTableStmtNotNullConstraint(NotNullConstraint nc, Table tb, L Map> enableValidateRely = nc.getEnableValidateRely(); for (String constraintName : notNullConstraints.keySet()) { ST command = new ST(ALTER_TABLE_ADD_NOT_NULL_CONSTRAINT); - command.add(DATABASE_NAME, nc.getDatabaseName()); - command.add(TABLE_NAME, nc.getTableName()); - command.add(COLUMN_NAME, notNullConstraints.get(constraintName)); + command.add(DATABASE_NAME, unparseIdentifier(nc.getDatabaseName())); + command.add(TABLE_NAME, unparseIdentifier(nc.getTableName())); + command.add(COLUMN_NAME, unparseIdentifier(notNullConstraints.get(constraintName))); command.add(COL_TYPE, colType.get(notNullConstraints.get(constraintName))); - command.add(CONSTRAINT_NAME, constraintName); + command.add(CONSTRAINT_NAME, unparseIdentifier(constraintName)); command.add(ENABLE, enableValidateRely.get(constraintName).get(0)); command.add(VALIDATE, enableValidateRely.get(constraintName).get(1)); command.add(RELY, enableValidateRely.get(constraintName).get(2)); @@ -803,9 +831,9 @@ public void getAlterTableStmtNotNullConstraint(NotNullConstraint nc, Table tb, L * * @param tb */ - public List populateConstraints(Table tb, Set allTableNames) { + public List populateConstraints(Table tb) { List constraints = new ArrayList<>(); - getAlterTableStmtForeignKeyConstraint(tb.getForeignKeyInfo(), constraints, allTableNames); + getAlterTableStmtForeignKeyConstraint(tb.getForeignKeyInfo(), constraints); getAlterTableStmtUniqueConstraint(tb.getUniqueKeyInfo(), constraints); getAlterTableStmtDefaultConstraint(tb.getDefaultConstraint(), tb, constraints); getAlterTableStmtCheckConstraint(tb.getCheckConstraint(), constraints); @@ -835,9 +863,9 @@ public String getCreateViewCommand(Table table, boolean isRelative) { ST command = new ST(CREATE_VIEW_TEMPLATE); if (!isRelative) { - command.add(DATABASE_NAME, table.getDbName()); + command.add(DATABASE_NAME, unparseIdentifier(table.getDbName())); } - command.add(TABLE_NAME, table.getTableName()); + command.add(TABLE_NAME, unparseIdentifier(table.getTableName())); command.add(PARTITIONS, getPartitionsForView(table)); command.add(SQL, table.getViewExpandedText()); @@ -849,9 +877,9 @@ public String getCreateTableCommand(Table table, boolean isRelative) { ST command = new ST(CREATE_TABLE_TEMPLATE); if (!isRelative) { - command.add(DATABASE_NAME, table.getDbName()); + command.add(DATABASE_NAME, unparseIdentifier(table.getDbName())); } - command.add(TABLE_NAME, table.getTableName()); + command.add(TABLE_NAME, unparseIdentifier(table.getTableName())); command.add(TEMPORARY, getTemporary(table)); command.add(EXTERNAL, getExternal(table)); command.add(LIST_COLUMNS, getColumns(table)); @@ -876,10 +904,10 @@ private String getExternal(Table table) { } private String getColumns(Table table) { - List columnDescs = new ArrayList(); + List columnDescs = new ArrayList<>(); for (FieldSchema column : table.getCols()) { String columnType = formatType(TypeInfoUtils.getTypeInfoFromTypeString(column.getType())); - String columnDesc = " `" + column.getName() + "` " + columnType; + String columnDesc = " " + unparseIdentifier(column.getName()) + " " + columnType; if (column.getComment() != null) { columnDesc += " COMMENT '" + HiveStringUtils.escapeHiveCommand(column.getComment()) + "'"; } @@ -889,7 +917,7 @@ private String getColumns(Table table) { } /** - * Struct fields are identifiers, need to be put between ``. + * Struct fields are identifiers, need to be unparsed. */ public static String formatType(TypeInfo typeInfo) { switch (typeInfo.getCategory()) { @@ -907,7 +935,7 @@ public static String formatType(TypeInfo typeInfo) { String structElementName = structTypeInfo.getAllStructFieldNames().get(i); String structElementType = formatType(structTypeInfo.getAllStructFieldTypeInfos().get(i)); - structFormattedType.append("`" + structElementName + "`:" + structElementType); + structFormattedType.append(unparseIdentifier(structElementName) + ":" + structElementType); } return "struct<" + structFormattedType.toString() + ">"; case LIST: @@ -949,7 +977,7 @@ private String getPartitionsForView(Table table) { } List partitionCols = new ArrayList(); for (String col : table.getPartColNames()) { - partitionCols.add('`' + col + '`'); + partitionCols.add(unparseIdentifier(col)); } return " PARTITIONED ON (" + StringUtils.join(partitionCols, ", ") + ")"; } @@ -962,7 +990,7 @@ private String getPartitions(Table table) { List partitionDescs = new ArrayList(); for (FieldSchema partitionKey : partitionKeys) { - String partitionDesc = " `" + partitionKey.getName() + "` " + partitionKey.getType(); + String partitionDesc = " " + unparseIdentifier(partitionKey.getName()) + " " + partitionKey.getType(); if (partitionKey.getComment() != null) { partitionDesc += " COMMENT '" + HiveStringUtils.escapeHiveCommand(partitionKey.getComment()) + "'"; } @@ -981,11 +1009,11 @@ private String getPartitionsBySpec(Table table) { List partitionTransforms = new ArrayList<>(); for (TransformSpec spec : specs) { if (spec.getTransformType() == TransformSpec.TransformType.IDENTITY) { - partitionTransforms.add(spec.getColumnName()); + partitionTransforms.add(unparseIdentifier(spec.getColumnName())); } else { partitionTransforms.add(spec.getTransformType().name() + "(" + (spec.getTransformParam().isPresent() ? spec.getTransformParam().get() + ", " : "") + - spec.getColumnName() + ")"); + unparseIdentifier(spec.getColumnName()) + ")"); } } return "PARTITIONED BY SPEC ( \n" + StringUtils.join(partitionTransforms, ", \n") + ")"; @@ -998,14 +1026,18 @@ private String getBuckets(Table table) { if (bucketCols.isEmpty()) { return ""; } + List unparsedBucketCols = bucketCols.stream() + .map(x -> unparseIdentifier(x)) + .collect(Collectors.toList()); - String buckets = "CLUSTERED BY ( \n `" + StringUtils.join(bucketCols, "`, \n `") + "`) \n"; + String buckets = "CLUSTERED BY (\n " + StringUtils.join(unparsedBucketCols, ",\n ") + ")\n"; List sortColumns = table.getSortCols(); if (!sortColumns.isEmpty()) { List sortKeys = new ArrayList(); for (Order sortColumn : sortColumns) { - String sortKeyDesc = " " + sortColumn.getCol() + " " + DirectionUtils.codeToText(sortColumn.getOrder()); + String sortKeyDesc = " " + unparseIdentifier(sortColumn.getCol()) + " " + + DirectionUtils.codeToText(sortColumn.getOrder()); sortKeys.add(sortKeyDesc); } buckets += "SORTED BY ( \n" + StringUtils.join(sortKeys, ", \n") + ") \n"; @@ -1021,13 +1053,17 @@ private String getSkewed(Table table) { return ""; } - List columnValuesList = new ArrayList(); + List unparsedSkewedColNames = skewedInfo.getSkewedColNames().stream() + .map(x -> unparseIdentifier(x)) + .collect(Collectors.toList()); + + List columnValuesList = new ArrayList<>(); for (List columnValues : skewedInfo.getSkewedColValues()) { columnValuesList.add("('" + StringUtils.join(columnValues, "','") + "')"); } String skewed = - "SKEWED BY (" + StringUtils.join(skewedInfo.getSkewedColNames(), ",") + ")\n" + + "SKEWED BY (" + StringUtils.join(unparsedSkewedColNames, ",") + ")\n" + " ON (" + StringUtils.join(columnValuesList, ",") + ")"; if (table.isStoredAsSubDirectories()) { skewed += "\n STORED AS DIRECTORIES"; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index 565cf50391e6..a8d4a8cbf53d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -464,9 +464,8 @@ public void addPKandBasicStats(Table tbl, List basicDef, DDLPlanUtils dd basicDef.add(ddlPlanUtils.getAlterTableStmtTableStatsBasic(tbl)); } - public void addConstraints(Table tbl, List constraints, Set allTableNames, - DDLPlanUtils ddlPlanUtils){ - constraints.addAll(ddlPlanUtils.populateConstraints(tbl, allTableNames)); + public void addConstraints(Table tbl, List constraints, DDLPlanUtils ddlPlanUtils){ + constraints.addAll(ddlPlanUtils.populateConstraints(tbl)); } public void addStats(Table table,List alterTableStmt ,Map> tablePartitionsMap, @@ -524,7 +523,7 @@ public void getDDLPlan(PrintStream out) throws Exception { } else { addCreateTableStatement(table, tableCreateStmt, ddlPlanUtils); addPKandBasicStats(table, tableBasicDef, ddlPlanUtils); - addConstraints(table, alterTableStmt, tableMap.keySet(), ddlPlanUtils); + addConstraints(table, alterTableStmt, ddlPlanUtils); addStats(table, alterTableStmt, tablePartitionsMap, ddlPlanUtils); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java index 199c564bd736..d1d197ebece3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java @@ -298,6 +298,10 @@ public static String unparseIdentifier(String identifier, Quotation quotation) { quotation.getQuotationChar()); } + public static String unparseIdentifier(String identifier) { + return unparseIdentifier(identifier, Quotation.BACKTICKS); + } + public static HiveStorageHandler getStorageHandler( Configuration conf, String className) throws HiveException { diff --git a/ql/src/test/queries/clientpositive/partition_explain_ddl.q b/ql/src/test/queries/clientpositive/partition_explain_ddl.q index 7cf4d560ce0b..23a40d872d44 100644 --- a/ql/src/test/queries/clientpositive/partition_explain_ddl.q +++ b/ql/src/test/queries/clientpositive/partition_explain_ddl.q @@ -142,4 +142,20 @@ analyze table db_bdpbase.default_partition_test compute statistics for columns; explain ddl select * from db_bdpbase.default_partition_test; -drop database db_bdpbase cascade; \ No newline at end of file +drop database db_bdpbase cascade; + +-- Add tests to check if partition values are quoted correctly. + +create table test_partitions ( + a int, + b string +) partitioned by ( + c string, + d date, + e int +); + +insert into test_partitions partition (c="str1", d="1970-01-01", e=1) values ('a_1', 'b_1'); +insert into test_partitions partition (c="str2", d="1970-02-02", e=2) values ('a_2', 'b_2'); + +explain ddl select * from test_partitions; diff --git a/ql/src/test/queries/clientpositive/show_create_table.q b/ql/src/test/queries/clientpositive/show_create_table.q index 9ed9c1ac133f..19058a969490 100644 --- a/ql/src/test/queries/clientpositive/show_create_table.q +++ b/ql/src/test/queries/clientpositive/show_create_table.q @@ -1,11 +1,11 @@ CREATE TABLE TEST( col1 varchar(100) NOT NULL COMMENT "comment for column 1", col2 timestamp DEFAULT CURRENT_TIMESTAMP() COMMENT "comment for column 2", - col3 decimal CHECK (col3 + col4 > 1) enable novalidate rely, + `col 3` decimal CHECK (`col 3` + col4 > 1) enable novalidate rely, col4 decimal NOT NULL, col5 varchar(100), primary key(col1, col2) disable novalidate rely, - constraint c3_c4_check CHECK((col3 + col4)/(col3 - col4) > 3) enable novalidate norely, + constraint c3_c4_check CHECK((`col 3` + col4)/(`col 3` - col4) > 3) enable novalidate norely, constraint c4_unique UNIQUE(col4) disable novalidate rely) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' diff --git a/ql/src/test/results/clientpositive/llap/constraints_explain_ddl.q.out b/ql/src/test/results/clientpositive/llap/constraints_explain_ddl.q.out index e393ef9bcad0..d2daeff2d7fa 100644 --- a/ql/src/test/results/clientpositive/llap/constraints_explain_ddl.q.out +++ b/ql/src/test/results/clientpositive/llap/constraints_explain_ddl.q.out @@ -99,8 +99,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.customer_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (c_custkey) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`c_custkey`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -170,8 +170,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.customer_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (c_custkey) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`c_custkey`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -251,8 +251,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dates_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (d_datekey,d_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`dates_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`d_datekey`,`d_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -334,8 +334,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dates_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (d_datekey,d_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`dates_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`d_datekey`,`d_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -480,8 +480,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dates_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (d_datekey,d_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`dates_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`d_datekey`,`d_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -626,8 +626,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dates_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (d_datekey,d_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`dates_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`d_datekey`,`d_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -897,62 +897,62 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.customer_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (c_custkey) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS SET('numRows'='1','rawDataSize'='22' ); -ALTER TABLE default.dates_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (d_datekey,d_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS SET('numRows'='2','rawDataSize'='102' ); -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_address SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_address BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_city SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_custkey SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_custkey BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_mktsegment SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_mktsegment BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_name SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_name BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_nation SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_nation BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_phone SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_phone BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_region SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_region BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_date SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_date BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_datekey SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_datekey BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_daynuminmonth SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_daynuminmonth BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_daynuminweek SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_daynuminweek BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_daynuminyear SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_daynuminyear BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_dayofweek SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_dayofweek BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_holidayfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_holidayfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_id SET('lowValue'='0','highValue'='1','numNulls'='0','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SIDgz8///////wE= -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_lastdayinmonthfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_lastdayinmonthfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_lastdayinweekfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_lastdayinweekfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_month SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_month BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_monthnuminyear SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_monthnuminyear BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_sellingseason SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_sellingseason BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_weekdayfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_weekdayfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_weeknuminyear SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_weeknuminyear BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_year SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_yearmonth SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_yearmonth BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_yearmonthnum SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_yearmonthnum BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`c_custkey`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS SET('numRows'='1','rawDataSize'='22' ); +ALTER TABLE `default`.`dates_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`d_datekey`,`d_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS SET('numRows'='2','rawDataSize'='102' ); +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_address` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_address` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_city` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_custkey` SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_custkey` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_mktsegment` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_mktsegment` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_name` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_name` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_nation` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_nation` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_phone` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_phone` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_region` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_region` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_date` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_date` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_datekey` SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_datekey` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_daynuminmonth` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_daynuminmonth` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_daynuminweek` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_daynuminweek` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_daynuminyear` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_daynuminyear` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_dayofweek` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_dayofweek` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_holidayfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_holidayfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_id` SET('lowValue'='0','highValue'='1','numNulls'='0','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SIDgz8///////wE= +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_lastdayinmonthfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_lastdayinmonthfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_lastdayinweekfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_lastdayinweekfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_month` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_month` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_monthnuminyear` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_monthnuminyear` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_sellingseason` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_sellingseason` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_weekdayfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_weekdayfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_weeknuminyear` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_weeknuminyear` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_year` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_yearmonth` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_yearmonth` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_yearmonthnum` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_yearmonthnum` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA @@ -1107,44 +1107,44 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dates_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (d_datekey,d_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS SET('numRows'='2','rawDataSize'='102' ); -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_date SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_date BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_datekey SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_datekey BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_daynuminmonth SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_daynuminmonth BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_daynuminweek SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_daynuminweek BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_daynuminyear SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_daynuminyear BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_dayofweek SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_dayofweek BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_holidayfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_holidayfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_id SET('lowValue'='0','highValue'='1','numNulls'='0','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SIDgz8///////wE= -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_lastdayinmonthfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_lastdayinmonthfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_lastdayinweekfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_lastdayinweekfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_month SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_month BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_monthnuminyear SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_monthnuminyear BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_sellingseason SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_sellingseason BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_weekdayfl SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_weekdayfl BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_weeknuminyear SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_weeknuminyear BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_year SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_yearmonth SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_yearmonth BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.dates_removal_n0 UPDATE STATISTICS FOR COLUMN d_yearmonthnum SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.dates_removal_n0 FOR COLUMN d_yearmonthnum BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`d_datekey`,`d_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS SET('numRows'='2','rawDataSize'='102' ); +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_date` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_date` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_datekey` SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_datekey` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_daynuminmonth` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_daynuminmonth` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_daynuminweek` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_daynuminweek` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_daynuminyear` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_daynuminyear` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_dayofweek` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_dayofweek` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_holidayfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_holidayfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_id` SET('lowValue'='0','highValue'='1','numNulls'='0','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SIDgz8///////wE= +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_lastdayinmonthfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_lastdayinmonthfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_lastdayinweekfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_lastdayinweekfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_month` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_month` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_monthnuminyear` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_monthnuminyear` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_sellingseason` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_sellingseason` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_weekdayfl` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_weekdayfl` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_weeknuminyear` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_weeknuminyear` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_year` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_yearmonth` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_yearmonth` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`dates_removal_n0` UPDATE STATISTICS FOR COLUMN `d_yearmonthnum` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`dates_removal_n0` FOR COLUMN `d_yearmonthnum` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA @@ -1310,24 +1310,24 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.customer_removal_n0 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (c_custkey) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS SET('numRows'='1','rawDataSize'='22' ); -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_address SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_address BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_city SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_custkey SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_custkey BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_mktsegment SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_mktsegment BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_name SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_name BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_nation SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_nation BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_phone SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_phone BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA -ALTER TABLE default.customer_removal_n0 UPDATE STATISTICS FOR COLUMN c_region SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.customer_removal_n0 FOR COLUMN c_region BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`c_custkey`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS SET('numRows'='1','rawDataSize'='22' ); +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_address` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_address` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_city` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_custkey` SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_custkey` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_mktsegment` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_mktsegment` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_name` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_name` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_nation` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_nation` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_phone` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_phone` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`customer_removal_n0` UPDATE STATISTICS FOR COLUMN `c_region` SET('avgColLen'='0.0','maxColLen'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`customer_removal_n0` FOR COLUMN `c_region` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA @@ -1500,12 +1500,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -1564,12 +1564,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -1629,12 +1629,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -1694,12 +1694,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -1823,12 +1823,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -1954,12 +1954,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -2104,12 +2104,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -2286,12 +2286,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -2354,12 +2354,12 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN key1 SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN key1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= -ALTER TABLE default.dest_g21 UPDATE STATISTICS FOR COLUMN value1 SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); --- BIT VECTORS PRESENT FOR default.dest_g21 FOR COLUMN value1 BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='6','rawDataSize'='28' ); +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `key1` SET('lowValue'='1','highValue'='6','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `key1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwfO+SIGmu+f//////wHC9+jHAf6diLP//////wG/9IJOg97xwAI= +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS FOR COLUMN `value1` SET('numNulls'='2','numDVs'='3','highValue'='4.0','lowValue'='1.0' ); +-- BIT VECTORS PRESENT FOR `default`.`dest_g21` FOR COLUMN `value1` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwvmagwOBp7a0Av65vtz9/////wE= @@ -2513,14 +2513,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.tconst UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); -ALTER TABLE default.tconst CHANGE COLUMN i i int CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN d_year SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN i SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN i BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN j SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN j BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= +ALTER TABLE `default`.`tconst` UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); +ALTER TABLE `default`.`tconst` CHANGE COLUMN `i` `i` int CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `d_year` SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `i` SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `i` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `j` SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `j` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= @@ -2642,14 +2642,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.tconst UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); -ALTER TABLE default.tconst CHANGE COLUMN i i int CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN d_year SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN i SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN i BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN j SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN j BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= +ALTER TABLE `default`.`tconst` UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); +ALTER TABLE `default`.`tconst` CHANGE COLUMN `i` `i` int CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `d_year` SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `i` SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `i` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `j` SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `j` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= @@ -2772,14 +2772,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.tconst UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); -ALTER TABLE default.tconst CHANGE COLUMN i i int CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN d_year SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN i SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN i BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN j SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN j BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= +ALTER TABLE `default`.`tconst` UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); +ALTER TABLE `default`.`tconst` CHANGE COLUMN `i` `i` int CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `d_year` SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `i` SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `i` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `j` SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `j` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= @@ -2901,14 +2901,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.tconst UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); -ALTER TABLE default.tconst CHANGE COLUMN i i int CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN d_year SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN i SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN i BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN j SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN j BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= +ALTER TABLE `default`.`tconst` UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); +ALTER TABLE `default`.`tconst` CHANGE COLUMN `i` `i` int CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `d_year` SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `i` SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `i` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `j` SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `j` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= @@ -3090,14 +3090,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.tconst UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); -ALTER TABLE default.tconst CHANGE COLUMN i i int CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN d_year SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN d_year BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN i SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN i BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== -ALTER TABLE default.tconst UPDATE STATISTICS FOR COLUMN j SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); --- BIT VECTORS PRESENT FOR default.tconst FOR COLUMN j BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= +ALTER TABLE `default`.`tconst` UPDATE STATISTICS SET('numRows'='3','rawDataSize'='25' ); +ALTER TABLE `default`.`tconst` CHANGE COLUMN `i` `i` int CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `d_year` SET('avgColLen'='4.0','maxColLen'='4','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `d_year` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwrjb8gb/vrr2+f////8BgaCT+///////AQ== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `i` SET('lowValue'='1','highValue'='3','numNulls'='0','numDVs'='3' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `i` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAMDwfO+SMG7rGLC0vSOAw== +ALTER TABLE `default`.`tconst` UPDATE STATISTICS FOR COLUMN `j` SET('lowValue'='1','highValue'='3','numNulls'='1','numDVs'='2' ); +-- BIT VECTORS PRESENT FOR `default`.`tconst` FOR COLUMN `j` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAICwfO+SMG7rGI= @@ -3291,9 +3291,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g21 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.dest_g21 ADD CONSTRAINT #### A masked pattern was here #### UNIQUE (key1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.dest_g21 CHANGE COLUMN key1 key1 int CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`dest_g21` ADD CONSTRAINT `#### A masked pattern was here ####` UNIQUE (`key1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g21` CHANGE COLUMN `key1` `key1` int CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; EXPLAIN select key1 from dest_g21 group by key1, value1; @@ -3354,8 +3354,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.dest_g24 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.dest_g24 ADD CONSTRAINT #### A masked pattern was here #### UNIQUE (key1) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`dest_g24` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`dest_g24` ADD CONSTRAINT `#### A masked pattern was here ####` UNIQUE (`key1`) DISABLE NOVALIDATE RELY; EXPLAIN select key1 from dest_g24 group by key1, value1; @@ -3703,13 +3703,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -3987,13 +3987,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -4301,13 +4301,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -4696,13 +4696,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -5218,14 +5218,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.date_dim UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`date_dim` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -5830,13 +5830,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -6122,13 +6122,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -6449,13 +6449,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -6773,13 +6773,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.store_sales ADD CONSTRAINT pk_ss PRIMARY KEY (ss_item_sk,ss_ticket_number) DISABLE NOVALIDATE RELY; -ALTER TABLE default.store_sales UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.customer ADD CONSTRAINT pk_c PRIMARY KEY (c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.store_sales ADD CONSTRAINT ss_c FOREIGN KEY (ss_customer_sk) REFERENCES default.customer(c_customer_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer ADD CONSTRAINT uk1 UNIQUE (c_customer_id) DISABLE NOVALIDATE RELY; -ALTER TABLE default.customer CHANGE COLUMN c_customer_id c_customer_id string CONSTRAINT cid_nn NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `pk_ss` PRIMARY KEY (`ss_item_sk`,`ss_ticket_number`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`store_sales` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`customer` ADD CONSTRAINT `pk_c` PRIMARY KEY (`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`store_sales` ADD CONSTRAINT `ss_c` FOREIGN KEY (`ss_customer_sk`) REFERENCES `default`.`customer`(`c_customer_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` ADD CONSTRAINT `uk1` UNIQUE (`c_customer_id`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`customer` CHANGE COLUMN `c_customer_id` `c_customer_id` string CONSTRAINT `cid_nn` NOT NULL DISABLE NOVALIDATE RELY; @@ -7033,14 +7033,14 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.web_sales ADD CONSTRAINT pk1 PRIMARY KEY (ws_order_number,ws_item_sk) DISABLE NOVALIDATE RELY; -ALTER TABLE default.web_sales UPDATE STATISTICS SET('numRows'='2','rawDataSize'='14' ); -ALTER TABLE default.web_sales UPDATE STATISTICS FOR COLUMN ws_item_sk SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.web_sales FOR COLUMN ws_item_sk BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== -ALTER TABLE default.web_sales UPDATE STATISTICS FOR COLUMN ws_order_number SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.web_sales FOR COLUMN ws_order_number BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== -ALTER TABLE default.web_sales UPDATE STATISTICS FOR COLUMN ws_price SET('numNulls'='0','numDVs'='1','highValue'='1.2000000476837158','lowValue'='1.2000000476837158' ); --- BIT VECTORS PRESENT FOR default.web_sales FOR COLUMN ws_price BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwbaGtAM= +ALTER TABLE `default`.`web_sales` ADD CONSTRAINT `pk1` PRIMARY KEY (`ws_order_number`,`ws_item_sk`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`web_sales` UPDATE STATISTICS SET('numRows'='2','rawDataSize'='14' ); +ALTER TABLE `default`.`web_sales` UPDATE STATISTICS FOR COLUMN `ws_item_sk` SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`web_sales` FOR COLUMN `ws_item_sk` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== +ALTER TABLE `default`.`web_sales` UPDATE STATISTICS FOR COLUMN `ws_order_number` SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`web_sales` FOR COLUMN `ws_order_number` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== +ALTER TABLE `default`.`web_sales` UPDATE STATISTICS FOR COLUMN `ws_price` SET('numNulls'='0','numDVs'='1','highValue'='1.2000000476837158','lowValue'='1.2000000476837158' ); +-- BIT VECTORS PRESENT FOR `default`.`web_sales` FOR COLUMN `ws_price` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwbaGtAM= EXPLAIN select count(distinct ws_order_number) from web_sales; @@ -7372,7 +7372,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table1_n13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table1_n13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7432,7 +7432,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table2_n8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table2_n8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7485,7 +7485,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table3_n1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table3_n1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7539,7 +7539,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table4_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table4_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7592,7 +7592,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table5_n4 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table5_n4` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7646,7 +7646,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table6_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table6_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7700,8 +7700,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table7_n3 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (a) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table7_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table7_n3` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`a`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table7_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7755,7 +7755,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7809,8 +7809,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table9 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (a,b) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table9 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table9` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`a`,`b`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table9` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7864,7 +7864,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table10 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table10` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7919,8 +7919,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table11 ADD CONSTRAINT pk11 PRIMARY KEY (a) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table11 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table11` ADD CONSTRAINT `pk11` PRIMARY KEY (`a`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table11` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -7974,7 +7974,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table12 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table12` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8029,8 +8029,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table13 CHANGE COLUMN a a string CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table13` CHANGE COLUMN `a` `a` string CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; EXPLAIN select * from table13; @@ -8083,8 +8083,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table14 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table14 CHANGE COLUMN a a string CONSTRAINT nn14_1 NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table14` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table14` CHANGE COLUMN `a` `a` string CONSTRAINT `nn14_1` NOT NULL DISABLE NOVALIDATE RELY; EXPLAIN select * from table14; @@ -8137,7 +8137,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table15 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table15` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8191,7 +8191,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table16 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table16` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8245,8 +8245,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table17 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table17 ADD CONSTRAINT uk17_1 UNIQUE (a) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table17` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table17` ADD CONSTRAINT `uk17_1` UNIQUE (`a`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table17; @@ -8300,8 +8300,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table18 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table18 ADD CONSTRAINT uk18_1 UNIQUE (b) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table18` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table18` ADD CONSTRAINT `uk18_1` UNIQUE (`b`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table18; @@ -8354,9 +8354,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table19 ADD CONSTRAINT pk19_1 PRIMARY KEY (b) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table19 ADD CONSTRAINT fk19_2 FOREIGN KEY (a) REFERENCES default.table19(b) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table19` ADD CONSTRAINT `pk19_1` PRIMARY KEY (`b`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table19` ADD CONSTRAINT `fk19_2` FOREIGN KEY (`a`) REFERENCES `default`.`table19`(`b`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table19; @@ -8409,8 +8409,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table20 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table20 ADD CONSTRAINT uk20_1 UNIQUE (b) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table20` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table20` ADD CONSTRAINT `uk20_1` UNIQUE (`b`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table20; @@ -8464,7 +8464,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table21 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table21` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8518,7 +8518,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table22 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table22` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8572,7 +8572,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table1_n13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table1_n13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8629,7 +8629,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table2_n8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table2_n8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8682,7 +8682,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table3_n1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table3_n1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8736,7 +8736,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table4_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table4_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8789,7 +8789,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table5_n4 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table5_n4` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8843,7 +8843,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table6_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table6_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8897,8 +8897,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table7_n3 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (a) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table7_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table7_n3` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`a`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table7_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -8952,7 +8952,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9006,8 +9006,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table9 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (a,b) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table9 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table9` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`a`,`b`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table9` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9061,7 +9061,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table10 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table10` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9116,8 +9116,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table11 ADD CONSTRAINT pk11 PRIMARY KEY (a) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table11 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table11` ADD CONSTRAINT `pk11` PRIMARY KEY (`a`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table11` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9171,7 +9171,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table12 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table12` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9226,8 +9226,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table13 CHANGE COLUMN a a string CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table13` CHANGE COLUMN `a` `a` string CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; EXPLAIN select * from table13; @@ -9280,8 +9280,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table14 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table14 CHANGE COLUMN a a string CONSTRAINT nn14_1 NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table14` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table14` CHANGE COLUMN `a` `a` string CONSTRAINT `nn14_1` NOT NULL DISABLE NOVALIDATE RELY; EXPLAIN select * from table14; @@ -9334,7 +9334,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table15 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table15` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9388,7 +9388,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table16 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table16` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9442,8 +9442,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table17 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table17 ADD CONSTRAINT uk17_1 UNIQUE (a) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table17` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table17` ADD CONSTRAINT `uk17_1` UNIQUE (`a`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table17; @@ -9497,8 +9497,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table18 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table18 ADD CONSTRAINT uk18_1 UNIQUE (b) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table18` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table18` ADD CONSTRAINT `uk18_1` UNIQUE (`b`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table18; @@ -9551,9 +9551,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table19 ADD CONSTRAINT pk19_1 PRIMARY KEY (b) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table19 ADD CONSTRAINT fk19_2 FOREIGN KEY (a) REFERENCES default.table19(b) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table19` ADD CONSTRAINT `pk19_1` PRIMARY KEY (`b`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table19` ADD CONSTRAINT `fk19_2` FOREIGN KEY (`a`) REFERENCES `default`.`table19`(`b`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table19; @@ -9606,8 +9606,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table20 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table20 ADD CONSTRAINT uk20_1 UNIQUE (b) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table20` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table20` ADD CONSTRAINT `uk20_1` UNIQUE (`b`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table20; @@ -9661,7 +9661,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table21 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table21` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9715,7 +9715,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table22 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table22` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9825,7 +9825,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table2_n8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table2_n8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9881,7 +9881,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table3_n1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table3_n1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9935,7 +9935,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table4_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table4_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -9989,7 +9989,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table6_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table6_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10043,7 +10043,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10097,7 +10097,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table16 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table16` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10152,7 +10152,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table18 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table18` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10206,7 +10206,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table2_n8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table2_n8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10262,7 +10262,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table3_n1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table3_n1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10316,7 +10316,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table4_n0 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table4_n0` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10370,7 +10370,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table6_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table6_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10424,7 +10424,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10478,7 +10478,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table16 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table16` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10533,7 +10533,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table18 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table18` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10635,7 +10635,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table2_n8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table2_n8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10691,8 +10691,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table3_n1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table3_n1 ADD CONSTRAINT fk1 FOREIGN KEY (x) REFERENCES default.table2_n8(a) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table3_n1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table3_n1` ADD CONSTRAINT `fk1` FOREIGN KEY (`x`) REFERENCES `default`.`table2_n8`(`a`) DISABLE NOVALIDATE RELY; EXPLAIN select * from table3_n1; @@ -10745,7 +10745,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table6_n3 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table6_n3` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10799,8 +10799,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table8 ADD CONSTRAINT pk8_2 PRIMARY KEY (a,b) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table8 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table8` ADD CONSTRAINT `pk8_2` PRIMARY KEY (`a`,`b`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table8` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10854,7 +10854,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table16 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table16` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10909,7 +10909,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table18 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table18` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -10979,7 +10979,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table12 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table12` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11037,8 +11037,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.table13 CHANGE COLUMN a a string CONSTRAINT #### A masked pattern was here #### NOT NULL DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table13` CHANGE COLUMN `a` `a` string CONSTRAINT `#### A masked pattern was here ####` NOT NULL DISABLE NOVALIDATE RELY; EXPLAIN select * from table13; @@ -11099,7 +11099,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table12 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table12` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11167,7 +11167,7 @@ POSTHOOK: query: explain ddl select * from DbConstraint.Table2 POSTHOOK: type: QUERY POSTHOOK: Input: dbconstraint@table2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS dbconstraint; +CREATE DATABASE IF NOT EXISTS `dbconstraint`; CREATE TABLE `dbconstraint`.`table2`( `a` string, `b` string) @@ -11182,7 +11182,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE dbconstraint.table2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `dbconstraint`.`table2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11224,7 +11224,7 @@ POSTHOOK: query: explain ddl select * from DbConstraint.Table2 POSTHOOK: type: QUERY POSTHOOK: Input: dbconstraint@table2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS dbconstraint; +CREATE DATABASE IF NOT EXISTS `dbconstraint`; CREATE TABLE `dbconstraint`.`table2`( `a` string, `b` string) @@ -11239,7 +11239,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE dbconstraint.table2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `dbconstraint`.`table2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11286,7 +11286,7 @@ POSTHOOK: query: explain ddl select * from DbConstraint.Table2 POSTHOOK: type: QUERY POSTHOOK: Input: dbconstraint@table2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS dbconstraint; +CREATE DATABASE IF NOT EXISTS `dbconstraint`; CREATE TABLE `dbconstraint`.`table2`( `a` string, `b` string) @@ -11301,7 +11301,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE dbconstraint.table2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `dbconstraint`.`table2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11343,7 +11343,7 @@ POSTHOOK: query: explain ddl select * from DbConstraint.Table2 POSTHOOK: type: QUERY POSTHOOK: Input: dbconstraint@table2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS dbconstraint; +CREATE DATABASE IF NOT EXISTS `dbconstraint`; CREATE TABLE `dbconstraint`.`table2`( `a` string, `b` string) @@ -11358,7 +11358,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE dbconstraint.table2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `dbconstraint`.`table2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11405,7 +11405,7 @@ POSTHOOK: query: explain ddl select * from DbConstraint.Table2 POSTHOOK: type: QUERY POSTHOOK: Input: dbconstraint@table2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS dbconstraint; +CREATE DATABASE IF NOT EXISTS `dbconstraint`; CREATE TABLE `dbconstraint`.`table2`( `a` string, `b` string) @@ -11420,7 +11420,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE dbconstraint.table2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `dbconstraint`.`table2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11467,7 +11467,7 @@ POSTHOOK: query: explain ddl select * from DbConstraint.Table2 POSTHOOK: type: QUERY POSTHOOK: Input: dbconstraint@table2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS dbconstraint; +CREATE DATABASE IF NOT EXISTS `dbconstraint`; CREATE TABLE `dbconstraint`.`table2`( `a` string, `b` string) @@ -11482,7 +11482,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE dbconstraint.table2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `dbconstraint`.`table2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -11561,8 +11561,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.table23 ADD CONSTRAINT pk23_1 PRIMARY KEY (b) DISABLE NOVALIDATE RELY; -ALTER TABLE default.table23 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`table23` ADD CONSTRAINT `pk23_1` PRIMARY KEY (`b`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`table23` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); diff --git a/ql/src/test/results/clientpositive/llap/create_table_explain_ddl.q.out b/ql/src/test/results/clientpositive/llap/create_table_explain_ddl.q.out index 5aaa4a586d29..3916165778cf 100644 --- a/ql/src/test/results/clientpositive/llap/create_table_explain_ddl.q.out +++ b/ql/src/test/results/clientpositive/llap/create_table_explain_ddl.q.out @@ -29,7 +29,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -93,7 +93,7 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'c'='3', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -158,7 +158,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -223,7 +223,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -288,7 +288,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -355,7 +355,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -423,7 +423,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE default.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -492,7 +492,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -507,7 +507,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -554,7 +554,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -571,7 +571,7 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'c'='3', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -618,7 +618,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -636,7 +636,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -683,7 +683,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE EXTERNAL TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -701,7 +701,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -748,7 +748,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -766,7 +766,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -813,7 +813,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -833,7 +833,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -880,7 +880,7 @@ POSTHOOK: query: explain ddl select * from alter1_db.alter1 POSTHOOK: type: QUERY POSTHOOK: Input: alter1_db@alter1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter1_db; +CREATE DATABASE IF NOT EXISTS `alter1_db`; CREATE TABLE `alter1_db`.`alter1`( `a` int, `b` int) @@ -901,7 +901,7 @@ TBLPROPERTIES ( 'c'='4', 'd'='3', #### A masked pattern was here #### -ALTER TABLE alter1_db.alter1 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter1_db`.`alter1` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -983,7 +983,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -1056,9 +1056,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE default.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `default`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1133,11 +1133,11 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE default.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-02'); -ALTER TABLE default.alter2 PARTITION (insertdate='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `default`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-02'); +ALTER TABLE `default`.`alter2` PARTITION (`insertdate`='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1218,7 +1218,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -1291,9 +1291,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE default.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `default`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1368,11 +1368,11 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE default.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-02'); -ALTER TABLE default.alter2 PARTITION (insertdate='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `default`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-02'); +ALTER TABLE `default`.`alter2` PARTITION (`insertdate`='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1460,7 +1460,7 @@ POSTHOOK: query: explain ddl select * from alter2 POSTHOOK: type: QUERY POSTHOOK: Input: alter2_db@alter2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter2_db; +CREATE DATABASE IF NOT EXISTS `alter2_db`; CREATE TABLE `alter2_db`.`alter2`( `a` int, `b` int) @@ -1477,7 +1477,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter2_db.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -1533,7 +1533,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: alter2_db@alter2 POSTHOOK: Input: alter2_db@alter2@insertdate=2008-01-01 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter2_db; +CREATE DATABASE IF NOT EXISTS `alter2_db`; CREATE TABLE `alter2_db`.`alter2`( `a` int, `b` int) @@ -1550,9 +1550,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter2_db.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE alter2_db.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE alter2_db.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `alter2_db`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1610,7 +1610,7 @@ POSTHOOK: Input: alter2_db@alter2 POSTHOOK: Input: alter2_db@alter2@insertdate=2008-01-01 POSTHOOK: Input: alter2_db@alter2@insertdate=2008-01-02 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter2_db; +CREATE DATABASE IF NOT EXISTS `alter2_db`; CREATE TABLE `alter2_db`.`alter2`( `a` int, `b` int) @@ -1627,11 +1627,11 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter2_db.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE alter2_db.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE alter2_db.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE alter2_db.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-02'); -ALTER TABLE alter2_db.alter2 PARTITION (insertdate='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `alter2_db`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-02'); +ALTER TABLE `alter2_db`.`alter2` PARTITION (`insertdate`='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1695,7 +1695,7 @@ POSTHOOK: query: explain ddl select * from alter2 POSTHOOK: type: QUERY POSTHOOK: Input: alter2_db@alter2 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter2_db; +CREATE DATABASE IF NOT EXISTS `alter2_db`; CREATE EXTERNAL TABLE `alter2_db`.`alter2`( `a` int, `b` int) @@ -1712,7 +1712,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter2_db.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -1768,7 +1768,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: alter2_db@alter2 POSTHOOK: Input: alter2_db@alter2@insertdate=2008-01-01 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter2_db; +CREATE DATABASE IF NOT EXISTS `alter2_db`; CREATE EXTERNAL TABLE `alter2_db`.`alter2`( `a` int, `b` int) @@ -1785,9 +1785,9 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter2_db.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE alter2_db.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE alter2_db.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `alter2_db`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; @@ -1845,7 +1845,7 @@ POSTHOOK: Input: alter2_db@alter2 POSTHOOK: Input: alter2_db@alter2@insertdate=2008-01-01 POSTHOOK: Input: alter2_db@alter2@insertdate=2008-01-02 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS alter2_db; +CREATE DATABASE IF NOT EXISTS `alter2_db`; CREATE EXTERNAL TABLE `alter2_db`.`alter2`( `a` int, `b` int) @@ -1862,11 +1862,11 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE alter2_db.alter2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE alter2_db.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-01'); -ALTER TABLE alter2_db.alter2 PARTITION (insertdate='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE alter2_db.alter2 ADD IF NOT EXISTS PARTITION (insertdate='2008-01-02'); -ALTER TABLE alter2_db.alter2 PARTITION (insertdate='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-01'); +ALTER TABLE `alter2_db`.`alter2` PARTITION (`insertdate`='2008-01-01') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `alter2_db`.`alter2` ADD IF NOT EXISTS PARTITION (`insertdate`='2008-01-02'); +ALTER TABLE `alter2_db`.`alter2` PARTITION (`insertdate`='2008-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); EXPLAIN select * from alter2; diff --git a/ql/src/test/results/clientpositive/llap/explain_ddl_nested_part.q.out b/ql/src/test/results/clientpositive/llap/explain_ddl_nested_part.q.out index bba587992b91..69624f11a277 100644 --- a/ql/src/test/results/clientpositive/llap/explain_ddl_nested_part.q.out +++ b/ql/src/test/results/clientpositive/llap/explain_ddl_nested_part.q.out @@ -82,30 +82,30 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.test ADD IF NOT EXISTS PARTITION (i=1,j=1); -ALTER TABLE default.test PARTITION (i=1,j=1) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); -ALTER TABLE default.test ADD IF NOT EXISTS PARTITION (i=2,j=2); -ALTER TABLE default.test PARTITION (i=2,j=2) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); -ALTER TABLE default.test ADD IF NOT EXISTS PARTITION (i=3,j=3); -ALTER TABLE default.test PARTITION (i=3,j=3) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); -ALTER TABLE default.test ADD IF NOT EXISTS PARTITION (i=4,j=4); -ALTER TABLE default.test PARTITION (i=4,j=4) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); -ALTER TABLE default.test ADD IF NOT EXISTS PARTITION (i=5,j=5); -ALTER TABLE default.test PARTITION (i=5,j=5) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); --- ALTER TABLE default.test ADD IF NOT EXISTS PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__); --- ALTER TABLE default.test PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__) UPDATE STATISTICS SET('numRows'='2','rawDataSize'='4' ); -ALTER TABLE default.test PARTITION (i=1,j=1) UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test PARTITION i=1,j=1 FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== -ALTER TABLE default.test PARTITION (i=2,j=2) UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='2','highValue'='2','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test PARTITION i=2,j=2 FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxIHguQQ= -ALTER TABLE default.test PARTITION (i=5,j=5) UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='5','highValue'='5','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test PARTITION i=5,j=5 FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwaPu+AE= -ALTER TABLE default.test PARTITION (i=3,j=3) UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test PARTITION i=3,j=3 FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= -ALTER TABLE default.test PARTITION (i=4,j=4) UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4','highValue'='4','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test PARTITION i=4,j=4 FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBhJHj9wE= --- ALTER TABLE default.test PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__) UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='0' ); +ALTER TABLE `default`.`test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`test` ADD IF NOT EXISTS PARTITION (`i`=1,`j`=1); +ALTER TABLE `default`.`test` PARTITION (`i`=1,`j`=1) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +ALTER TABLE `default`.`test` ADD IF NOT EXISTS PARTITION (`i`=2,`j`=2); +ALTER TABLE `default`.`test` PARTITION (`i`=2,`j`=2) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +ALTER TABLE `default`.`test` ADD IF NOT EXISTS PARTITION (`i`=3,`j`=3); +ALTER TABLE `default`.`test` PARTITION (`i`=3,`j`=3) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +ALTER TABLE `default`.`test` ADD IF NOT EXISTS PARTITION (`i`=4,`j`=4); +ALTER TABLE `default`.`test` PARTITION (`i`=4,`j`=4) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +ALTER TABLE `default`.`test` ADD IF NOT EXISTS PARTITION (`i`=5,`j`=5); +ALTER TABLE `default`.`test` PARTITION (`i`=5,`j`=5) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +-- ALTER TABLE `default`.`test` ADD IF NOT EXISTS PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__); +-- ALTER TABLE `default`.`test` PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__) UPDATE STATISTICS SET('numRows'='2','rawDataSize'='4' ); +ALTER TABLE `default`.`test` PARTITION (`i`=1,`j`=1) UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test` PARTITION `i`=1,`j`=1 FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== +ALTER TABLE `default`.`test` PARTITION (`i`=2,`j`=2) UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='2','highValue'='2','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test` PARTITION `i`=2,`j`=2 FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxIHguQQ= +ALTER TABLE `default`.`test` PARTITION (`i`=5,`j`=5) UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='5','highValue'='5','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test` PARTITION `i`=5,`j`=5 FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwaPu+AE= +ALTER TABLE `default`.`test` PARTITION (`i`=3,`j`=3) UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3','highValue'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test` PARTITION `i`=3,`j`=3 FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgq/rqgE= +ALTER TABLE `default`.`test` PARTITION (`i`=4,`j`=4) UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4','highValue'='4','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test` PARTITION `i`=4,`j`=4 FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBhJHj9wE= +-- ALTER TABLE `default`.`test` PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__) UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='0','highValue'='0','numNulls'='2','numDVs'='0' ); @@ -371,165 +371,165 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.test2 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); --- ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=1,j=1,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__'); --- ALTER TABLE default.test2 PARTITION (i=1,j=1,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=1522484,j=1522484,s='1522484',t='1522484'); -ALTER TABLE default.test2 PARTITION (i=1522484,j=1522484,s='1522484',t='1522484') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=1565998,j=1565998,s='1565998',t='1565998'); -ALTER TABLE default.test2 PARTITION (i=1565998,j=1565998,s='1565998',t='1565998') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=1655195,j=1655195,s='1655195',t='1655195'); -ALTER TABLE default.test2 PARTITION (i=1655195,j=1655195,s='1655195',t='1655195') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=1880938,j=1880938,s='1880938',t='1880938'); -ALTER TABLE default.test2 PARTITION (i=1880938,j=1880938,s='1880938',t='1880938') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=1956691,j=1956691,s='1956691',t='1956691'); -ALTER TABLE default.test2 PARTITION (i=1956691,j=1956691,s='1956691',t='1956691') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); --- ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=2,j=2,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__'); --- ALTER TABLE default.test2 PARTITION (i=2,j=2,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=2049769,j=2049769,s='2049769',t='2049769'); -ALTER TABLE default.test2 PARTITION (i=2049769,j=2049769,s='2049769',t='2049769') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=2146641,j=2146641,s='2146641',t='2146641'); -ALTER TABLE default.test2 PARTITION (i=2146641,j=2146641,s='2146641',t='2146641') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=2581458,j=2581458,s='2581458',t='2581458'); -ALTER TABLE default.test2 PARTITION (i=2581458,j=2581458,s='2581458',t='2581458') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3023419,j=3023419,s='3023419',t='3023419'); -ALTER TABLE default.test2 PARTITION (i=3023419,j=3023419,s='3023419',t='3023419') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3073641,j=3073641,s='3073641',t='3073641'); -ALTER TABLE default.test2 PARTITION (i=3073641,j=3073641,s='3073641',t='3073641') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3076080,j=3076080,s='3076080',t='3076080'); -ALTER TABLE default.test2 PARTITION (i=3076080,j=3076080,s='3076080',t='3076080') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3187683,j=3187683,s='3187683',t='3187683'); -ALTER TABLE default.test2 PARTITION (i=3187683,j=3187683,s='3187683',t='3187683') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3356420,j=3356420,s='3356420',t='3356420'); -ALTER TABLE default.test2 PARTITION (i=3356420,j=3356420,s='3356420',t='3356420') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3583871,j=3583871,s='3583871',t='3583871'); -ALTER TABLE default.test2 PARTITION (i=3583871,j=3583871,s='3583871',t='3583871') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=3847717,j=3847717,s='3847717',t='3847717'); -ALTER TABLE default.test2 PARTITION (i=3847717,j=3847717,s='3847717',t='3847717') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=4115101,j=4115101,s='4115101',t='4115101'); -ALTER TABLE default.test2 PARTITION (i=4115101,j=4115101,s='4115101',t='4115101') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=42629,j=42629,s='42629',t='42629'); -ALTER TABLE default.test2 PARTITION (i=42629,j=42629,s='42629',t='42629') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='5' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=4304398,j=4304398,s='4304398',t='4304398'); -ALTER TABLE default.test2 PARTITION (i=4304398,j=4304398,s='4304398',t='4304398') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=4409637,j=4409637,s='4409637',t='4409637'); -ALTER TABLE default.test2 PARTITION (i=4409637,j=4409637,s='4409637',t='4409637') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=4439186,j=4439186,s='4439186',t='4439186'); -ALTER TABLE default.test2 PARTITION (i=4439186,j=4439186,s='4439186',t='4439186') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=4497880,j=4497880,s='4497880',t='4497880'); -ALTER TABLE default.test2 PARTITION (i=4497880,j=4497880,s='4497880',t='4497880') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=4604522,j=4604522,s='4604522',t='4604522'); -ALTER TABLE default.test2 PARTITION (i=4604522,j=4604522,s='4604522',t='4604522') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=5238051,j=5238051,s='5238051',t='5238051'); -ALTER TABLE default.test2 PARTITION (i=5238051,j=5238051,s='5238051',t='5238051') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=5247332,j=5247332,s='5247332',t='5247332'); -ALTER TABLE default.test2 PARTITION (i=5247332,j=5247332,s='5247332',t='5247332') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=5803231,j=5803231,s='5803231',t='5803231'); -ALTER TABLE default.test2 PARTITION (i=5803231,j=5803231,s='5803231',t='5803231') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=6129618,j=6129618,s='6129618',t='6129618'); -ALTER TABLE default.test2 PARTITION (i=6129618,j=6129618,s='6129618',t='6129618') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=6276003,j=6276003,s='6276003',t='6276003'); -ALTER TABLE default.test2 PARTITION (i=6276003,j=6276003,s='6276003',t='6276003') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=6758530,j=6758530,s='6758530',t='6758530'); -ALTER TABLE default.test2 PARTITION (i=6758530,j=6758530,s='6758530',t='6758530') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=7005576,j=7005576,s='7005576',t='7005576'); -ALTER TABLE default.test2 PARTITION (i=7005576,j=7005576,s='7005576',t='7005576') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=7286712,j=7286712,s='7286712',t='7286712'); -ALTER TABLE default.test2 PARTITION (i=7286712,j=7286712,s='7286712',t='7286712') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=7670676,j=7670676,s='7670676',t='7670676'); -ALTER TABLE default.test2 PARTITION (i=7670676,j=7670676,s='7670676',t='7670676') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=7880229,j=7880229,s='7880229',t='7880229'); -ALTER TABLE default.test2 PARTITION (i=7880229,j=7880229,s='7880229',t='7880229') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=812837,j=812837,s='812837',t='812837'); -ALTER TABLE default.test2 PARTITION (i=812837,j=812837,s='812837',t='812837') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=967815,j=967815,s='967815',t='967815'); -ALTER TABLE default.test2 PARTITION (i=967815,j=967815,s='967815',t='967815') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=971079,j=971079,s='971079',t='971079'); -ALTER TABLE default.test2 PARTITION (i=971079,j=971079,s='971079',t='971079') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); -ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=9805326,j=9805326,s='9805326',t='9805326'); -ALTER TABLE default.test2 PARTITION (i=9805326,j=9805326,s='9805326',t='9805326') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); --- ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__,s='3',t='3'); --- ALTER TABLE default.test2 PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__,s='3',t='3') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='2' ); --- ALTER TABLE default.test2 ADD IF NOT EXISTS PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__'); --- ALTER TABLE default.test2 PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='2' ); -ALTER TABLE default.test2 PARTITION (i=1956691,j=1956691,s='1956691',t='1956691') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1956691','highValue'='1956691','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=1956691,j=1956691,s='1956691',t='1956691' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgZ3cVQ== -ALTER TABLE default.test2 PARTITION (i=7005576,j=7005576,s='7005576',t='7005576') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='7005576','highValue'='7005576','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=7005576,j=7005576,s='7005576',t='7005576' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgc7krgc= -ALTER TABLE default.test2 PARTITION (i=7286712,j=7286712,s='7286712',t='7286712') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='7286712','highValue'='7286712','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=7286712,j=7286712,s='7286712',t='7286712' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwperKQ== -ALTER TABLE default.test2 PARTITION (i=1522484,j=1522484,s='1522484',t='1522484') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1522484','highValue'='1522484','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=1522484,j=1522484,s='1522484',t='1522484' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg5zSoAU= -ALTER TABLE default.test2 PARTITION (i=2049769,j=2049769,s='2049769',t='2049769') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='2049769','highValue'='2049769','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=2049769,j=2049769,s='2049769',t='2049769' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfS2sAQ= -ALTER TABLE default.test2 PARTITION (i=3073641,j=3073641,s='3073641',t='3073641') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3073641','highValue'='3073641','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3073641,j=3073641,s='3073641',t='3073641' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBguu6jgQ= -ALTER TABLE default.test2 PARTITION (i=3583871,j=3583871,s='3583871',t='3583871') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3583871','highValue'='3583871','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3583871,j=3583871,s='3583871',t='3583871' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxoKDzwM= -ALTER TABLE default.test2 PARTITION (i=3847717,j=3847717,s='3847717',t='3847717') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3847717','highValue'='3847717','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3847717,j=3847717,s='3847717',t='3847717' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxfvAiwE= -ALTER TABLE default.test2 PARTITION (i=7670676,j=7670676,s='7670676',t='7670676') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='7670676','highValue'='7670676','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=7670676,j=7670676,s='7670676',t='7670676' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgejUtQU= -ALTER TABLE default.test2 PARTITION (i=1655195,j=1655195,s='1655195',t='1655195') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1655195','highValue'='1655195','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=1655195,j=1655195,s='1655195',t='1655195' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwvLYBQ== -ALTER TABLE default.test2 PARTITION (i=6129618,j=6129618,s='6129618',t='6129618') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='6129618','highValue'='6129618','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=6129618,j=6129618,s='6129618',t='6129618' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgqzpngY= -ALTER TABLE default.test2 PARTITION (i=7880229,j=7880229,s='7880229',t='7880229') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='7880229','highValue'='7880229','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=7880229,j=7880229,s='7880229',t='7880229' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgturhwI= -ALTER TABLE default.test2 PARTITION (i=5803231,j=5803231,s='5803231',t='5803231') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='5803231','highValue'='5803231','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=5803231,j=5803231,s='5803231',t='5803231' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgpW7mQI= --- ALTER TABLE default.test2 PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__,s='3',t='3') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='0','highValue'='0','numNulls'='1','numDVs'='0' ); -ALTER TABLE default.test2 PARTITION (i=4439186,j=4439186,s='4439186',t='4439186') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4439186','highValue'='4439186','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=4439186,j=4439186,s='4439186',t='4439186' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgd22gQY= -ALTER TABLE default.test2 PARTITION (i=4497880,j=4497880,s='4497880',t='4497880') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4497880','highValue'='4497880','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=4497880,j=4497880,s='4497880',t='4497880' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwcvXgAE= -ALTER TABLE default.test2 PARTITION (i=2146641,j=2146641,s='2146641',t='2146641') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='2146641','highValue'='2146641','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=2146641,j=2146641,s='2146641',t='2146641' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg9/xOw== -ALTER TABLE default.test2 PARTITION (i=967815,j=967815,s='967815',t='967815') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='967815','highValue'='967815','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=967815,j=967815,s='967815',t='967815' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgom62gM= -ALTER TABLE default.test2 PARTITION (i=4604522,j=4604522,s='4604522',t='4604522') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4604522','highValue'='4604522','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=4604522,j=4604522,s='4604522',t='4604522' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgrrK+AY= -ALTER TABLE default.test2 PARTITION (i=5238051,j=5238051,s='5238051',t='5238051') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='5238051','highValue'='5238051','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=5238051,j=5238051,s='5238051',t='5238051' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBhfjCjwE= -ALTER TABLE default.test2 PARTITION (i=3187683,j=3187683,s='3187683',t='3187683') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3187683','highValue'='3187683','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3187683,j=3187683,s='3187683',t='3187683' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwZn8uAc= -ALTER TABLE default.test2 PARTITION (i=6758530,j=6758530,s='6758530',t='6758530') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='6758530','highValue'='6758530','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=6758530,j=6758530,s='6758530',t='6758530' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgtSr0gQ= -ALTER TABLE default.test2 PARTITION (i=2581458,j=2581458,s='2581458',t='2581458') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='2581458','highValue'='2581458','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=2581458,j=2581458,s='2581458',t='2581458' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgf6ljQQ= -ALTER TABLE default.test2 PARTITION (i=4304398,j=4304398,s='4304398',t='4304398') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4304398','highValue'='4304398','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=4304398,j=4304398,s='4304398',t='4304398' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgf2tnwQ= -ALTER TABLE default.test2 PARTITION (i=812837,j=812837,s='812837',t='812837') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='812837','highValue'='812837','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=812837,j=812837,s='812837',t='812837' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwuja+gY= --- ALTER TABLE default.test2 PARTITION (i=2,j=2,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='2','highValue'='2','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=2,j=2,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxIHguQQ= -ALTER TABLE default.test2 PARTITION (i=3356420,j=3356420,s='3356420',t='3356420') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3356420','highValue'='3356420','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3356420,j=3356420,s='3356420',t='3356420' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwa/logE= --- ALTER TABLE default.test2 PARTITION (i=__HIVE_DEFAULT_PARTITION__,j=__HIVE_DEFAULT_PARTITION__,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='0','highValue'='0','numNulls'='1','numDVs'='0' ); -ALTER TABLE default.test2 PARTITION (i=42629,j=42629,s='42629',t='42629') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='42629','highValue'='42629','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=42629,j=42629,s='42629',t='42629' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBweiP4AY= -ALTER TABLE default.test2 PARTITION (i=5247332,j=5247332,s='5247332',t='5247332') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='5247332','highValue'='5247332','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=5247332,j=5247332,s='5247332',t='5247332' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg5mT4Qc= -ALTER TABLE default.test2 PARTITION (i=1565998,j=1565998,s='1565998',t='1565998') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1565998','highValue'='1565998','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=1565998,j=1565998,s='1565998',t='1565998' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBhKGjdQ== -ALTER TABLE default.test2 PARTITION (i=1880938,j=1880938,s='1880938',t='1880938') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1880938','highValue'='1880938','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=1880938,j=1880938,s='1880938',t='1880938' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgb2svAQ= -ALTER TABLE default.test2 PARTITION (i=9805326,j=9805326,s='9805326',t='9805326') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='9805326','highValue'='9805326','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=9805326,j=9805326,s='9805326',t='9805326' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBw96RggU= -ALTER TABLE default.test2 PARTITION (i=3076080,j=3076080,s='3076080',t='3076080') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3076080','highValue'='3076080','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3076080,j=3076080,s='3076080',t='3076080' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBweDt4QM= -ALTER TABLE default.test2 PARTITION (i=4409637,j=4409637,s='4409637',t='4409637') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4409637','highValue'='4409637','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=4409637,j=4409637,s='4409637',t='4409637' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgsT4sAc= -ALTER TABLE default.test2 PARTITION (i=6276003,j=6276003,s='6276003',t='6276003') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='6276003','highValue'='6276003','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=6276003,j=6276003,s='6276003',t='6276003' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBw8Kw8wQ= --- ALTER TABLE default.test2 PARTITION (i=1,j=1,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=1,j=1,s='__HIVE_DEFAULT_PARTITION__',t='__HIVE_DEFAULT_PARTITION__' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== -ALTER TABLE default.test2 PARTITION (i=3023419,j=3023419,s='3023419',t='3023419') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='3023419','highValue'='3023419','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=3023419,j=3023419,s='3023419',t='3023419' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBge2+4gQ= -ALTER TABLE default.test2 PARTITION (i=4115101,j=4115101,s='4115101',t='4115101') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='4115101','highValue'='4115101','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=4115101,j=4115101,s='4115101',t='4115101' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgaamtAY= -ALTER TABLE default.test2 PARTITION (i=971079,j=971079,s='971079',t='971079') UPDATE STATISTICS FOR COLUMN foo SET('lowValue'='971079','highValue'='971079','numNulls'='0','numDVs'='1' ); --- BIT VECTORS PRESENT FOR default.test2 PARTITION i=971079,j=971079,s='971079',t='971079' FOR COLUMN foo BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg6XR2gc= +ALTER TABLE `default`.`test2` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +-- ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=1,`j`=1,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__'); +-- ALTER TABLE `default`.`test2` PARTITION (`i`=1,`j`=1,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=1522484,`j`=1522484,`s`='1522484',`t`='1522484'); +ALTER TABLE `default`.`test2` PARTITION (`i`=1522484,`j`=1522484,`s`='1522484',`t`='1522484') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=1565998,`j`=1565998,`s`='1565998',`t`='1565998'); +ALTER TABLE `default`.`test2` PARTITION (`i`=1565998,`j`=1565998,`s`='1565998',`t`='1565998') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=1655195,`j`=1655195,`s`='1655195',`t`='1655195'); +ALTER TABLE `default`.`test2` PARTITION (`i`=1655195,`j`=1655195,`s`='1655195',`t`='1655195') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=1880938,`j`=1880938,`s`='1880938',`t`='1880938'); +ALTER TABLE `default`.`test2` PARTITION (`i`=1880938,`j`=1880938,`s`='1880938',`t`='1880938') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=1956691,`j`=1956691,`s`='1956691',`t`='1956691'); +ALTER TABLE `default`.`test2` PARTITION (`i`=1956691,`j`=1956691,`s`='1956691',`t`='1956691') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +-- ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=2,`j`=2,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__'); +-- ALTER TABLE `default`.`test2` PARTITION (`i`=2,`j`=2,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='1' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=2049769,`j`=2049769,`s`='2049769',`t`='2049769'); +ALTER TABLE `default`.`test2` PARTITION (`i`=2049769,`j`=2049769,`s`='2049769',`t`='2049769') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=2146641,`j`=2146641,`s`='2146641',`t`='2146641'); +ALTER TABLE `default`.`test2` PARTITION (`i`=2146641,`j`=2146641,`s`='2146641',`t`='2146641') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=2581458,`j`=2581458,`s`='2581458',`t`='2581458'); +ALTER TABLE `default`.`test2` PARTITION (`i`=2581458,`j`=2581458,`s`='2581458',`t`='2581458') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3023419,`j`=3023419,`s`='3023419',`t`='3023419'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3023419,`j`=3023419,`s`='3023419',`t`='3023419') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3073641,`j`=3073641,`s`='3073641',`t`='3073641'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3073641,`j`=3073641,`s`='3073641',`t`='3073641') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3076080,`j`=3076080,`s`='3076080',`t`='3076080'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3076080,`j`=3076080,`s`='3076080',`t`='3076080') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3187683,`j`=3187683,`s`='3187683',`t`='3187683'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3187683,`j`=3187683,`s`='3187683',`t`='3187683') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3356420,`j`=3356420,`s`='3356420',`t`='3356420'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3356420,`j`=3356420,`s`='3356420',`t`='3356420') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3583871,`j`=3583871,`s`='3583871',`t`='3583871'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3583871,`j`=3583871,`s`='3583871',`t`='3583871') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=3847717,`j`=3847717,`s`='3847717',`t`='3847717'); +ALTER TABLE `default`.`test2` PARTITION (`i`=3847717,`j`=3847717,`s`='3847717',`t`='3847717') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=4115101,`j`=4115101,`s`='4115101',`t`='4115101'); +ALTER TABLE `default`.`test2` PARTITION (`i`=4115101,`j`=4115101,`s`='4115101',`t`='4115101') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=42629,`j`=42629,`s`='42629',`t`='42629'); +ALTER TABLE `default`.`test2` PARTITION (`i`=42629,`j`=42629,`s`='42629',`t`='42629') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='5' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=4304398,`j`=4304398,`s`='4304398',`t`='4304398'); +ALTER TABLE `default`.`test2` PARTITION (`i`=4304398,`j`=4304398,`s`='4304398',`t`='4304398') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=4409637,`j`=4409637,`s`='4409637',`t`='4409637'); +ALTER TABLE `default`.`test2` PARTITION (`i`=4409637,`j`=4409637,`s`='4409637',`t`='4409637') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=4439186,`j`=4439186,`s`='4439186',`t`='4439186'); +ALTER TABLE `default`.`test2` PARTITION (`i`=4439186,`j`=4439186,`s`='4439186',`t`='4439186') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=4497880,`j`=4497880,`s`='4497880',`t`='4497880'); +ALTER TABLE `default`.`test2` PARTITION (`i`=4497880,`j`=4497880,`s`='4497880',`t`='4497880') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=4604522,`j`=4604522,`s`='4604522',`t`='4604522'); +ALTER TABLE `default`.`test2` PARTITION (`i`=4604522,`j`=4604522,`s`='4604522',`t`='4604522') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=5238051,`j`=5238051,`s`='5238051',`t`='5238051'); +ALTER TABLE `default`.`test2` PARTITION (`i`=5238051,`j`=5238051,`s`='5238051',`t`='5238051') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=5247332,`j`=5247332,`s`='5247332',`t`='5247332'); +ALTER TABLE `default`.`test2` PARTITION (`i`=5247332,`j`=5247332,`s`='5247332',`t`='5247332') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=5803231,`j`=5803231,`s`='5803231',`t`='5803231'); +ALTER TABLE `default`.`test2` PARTITION (`i`=5803231,`j`=5803231,`s`='5803231',`t`='5803231') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=6129618,`j`=6129618,`s`='6129618',`t`='6129618'); +ALTER TABLE `default`.`test2` PARTITION (`i`=6129618,`j`=6129618,`s`='6129618',`t`='6129618') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=6276003,`j`=6276003,`s`='6276003',`t`='6276003'); +ALTER TABLE `default`.`test2` PARTITION (`i`=6276003,`j`=6276003,`s`='6276003',`t`='6276003') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=6758530,`j`=6758530,`s`='6758530',`t`='6758530'); +ALTER TABLE `default`.`test2` PARTITION (`i`=6758530,`j`=6758530,`s`='6758530',`t`='6758530') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=7005576,`j`=7005576,`s`='7005576',`t`='7005576'); +ALTER TABLE `default`.`test2` PARTITION (`i`=7005576,`j`=7005576,`s`='7005576',`t`='7005576') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=7286712,`j`=7286712,`s`='7286712',`t`='7286712'); +ALTER TABLE `default`.`test2` PARTITION (`i`=7286712,`j`=7286712,`s`='7286712',`t`='7286712') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=7670676,`j`=7670676,`s`='7670676',`t`='7670676'); +ALTER TABLE `default`.`test2` PARTITION (`i`=7670676,`j`=7670676,`s`='7670676',`t`='7670676') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=7880229,`j`=7880229,`s`='7880229',`t`='7880229'); +ALTER TABLE `default`.`test2` PARTITION (`i`=7880229,`j`=7880229,`s`='7880229',`t`='7880229') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=812837,`j`=812837,`s`='812837',`t`='812837'); +ALTER TABLE `default`.`test2` PARTITION (`i`=812837,`j`=812837,`s`='812837',`t`='812837') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=967815,`j`=967815,`s`='967815',`t`='967815'); +ALTER TABLE `default`.`test2` PARTITION (`i`=967815,`j`=967815,`s`='967815',`t`='967815') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=971079,`j`=971079,`s`='971079',`t`='971079'); +ALTER TABLE `default`.`test2` PARTITION (`i`=971079,`j`=971079,`s`='971079',`t`='971079') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); +ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=9805326,`j`=9805326,`s`='9805326',`t`='9805326'); +ALTER TABLE `default`.`test2` PARTITION (`i`=9805326,`j`=9805326,`s`='9805326',`t`='9805326') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='7' ); +-- ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__,`s`='3',`t`='3'); +-- ALTER TABLE `default`.`test2` PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__,`s`='3',`t`='3') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='2' ); +-- ALTER TABLE `default`.`test2` ADD IF NOT EXISTS PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__'); +-- ALTER TABLE `default`.`test2` PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='1','rawDataSize'='2' ); +ALTER TABLE `default`.`test2` PARTITION (`i`=1956691,`j`=1956691,`s`='1956691',`t`='1956691') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1956691','highValue'='1956691','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=1956691,`j`=1956691,`s`='1956691',`t`='1956691' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgZ3cVQ== +ALTER TABLE `default`.`test2` PARTITION (`i`=7005576,`j`=7005576,`s`='7005576',`t`='7005576') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='7005576','highValue'='7005576','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=7005576,`j`=7005576,`s`='7005576',`t`='7005576' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgc7krgc= +ALTER TABLE `default`.`test2` PARTITION (`i`=7286712,`j`=7286712,`s`='7286712',`t`='7286712') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='7286712','highValue'='7286712','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=7286712,`j`=7286712,`s`='7286712',`t`='7286712' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwperKQ== +ALTER TABLE `default`.`test2` PARTITION (`i`=1522484,`j`=1522484,`s`='1522484',`t`='1522484') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1522484','highValue'='1522484','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=1522484,`j`=1522484,`s`='1522484',`t`='1522484' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg5zSoAU= +ALTER TABLE `default`.`test2` PARTITION (`i`=2049769,`j`=2049769,`s`='2049769',`t`='2049769') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='2049769','highValue'='2049769','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=2049769,`j`=2049769,`s`='2049769',`t`='2049769' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfS2sAQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=3073641,`j`=3073641,`s`='3073641',`t`='3073641') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3073641','highValue'='3073641','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3073641,`j`=3073641,`s`='3073641',`t`='3073641' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBguu6jgQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=3583871,`j`=3583871,`s`='3583871',`t`='3583871') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3583871','highValue'='3583871','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3583871,`j`=3583871,`s`='3583871',`t`='3583871' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxoKDzwM= +ALTER TABLE `default`.`test2` PARTITION (`i`=3847717,`j`=3847717,`s`='3847717',`t`='3847717') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3847717','highValue'='3847717','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3847717,`j`=3847717,`s`='3847717',`t`='3847717' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxfvAiwE= +ALTER TABLE `default`.`test2` PARTITION (`i`=7670676,`j`=7670676,`s`='7670676',`t`='7670676') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='7670676','highValue'='7670676','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=7670676,`j`=7670676,`s`='7670676',`t`='7670676' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgejUtQU= +ALTER TABLE `default`.`test2` PARTITION (`i`=1655195,`j`=1655195,`s`='1655195',`t`='1655195') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1655195','highValue'='1655195','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=1655195,`j`=1655195,`s`='1655195',`t`='1655195' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwvLYBQ== +ALTER TABLE `default`.`test2` PARTITION (`i`=6129618,`j`=6129618,`s`='6129618',`t`='6129618') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='6129618','highValue'='6129618','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=6129618,`j`=6129618,`s`='6129618',`t`='6129618' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgqzpngY= +ALTER TABLE `default`.`test2` PARTITION (`i`=7880229,`j`=7880229,`s`='7880229',`t`='7880229') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='7880229','highValue'='7880229','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=7880229,`j`=7880229,`s`='7880229',`t`='7880229' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgturhwI= +ALTER TABLE `default`.`test2` PARTITION (`i`=5803231,`j`=5803231,`s`='5803231',`t`='5803231') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='5803231','highValue'='5803231','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=5803231,`j`=5803231,`s`='5803231',`t`='5803231' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgpW7mQI= +-- ALTER TABLE `default`.`test2` PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__,`s`='3',`t`='3') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='0','highValue'='0','numNulls'='1','numDVs'='0' ); +ALTER TABLE `default`.`test2` PARTITION (`i`=4439186,`j`=4439186,`s`='4439186',`t`='4439186') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4439186','highValue'='4439186','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=4439186,`j`=4439186,`s`='4439186',`t`='4439186' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgd22gQY= +ALTER TABLE `default`.`test2` PARTITION (`i`=4497880,`j`=4497880,`s`='4497880',`t`='4497880') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4497880','highValue'='4497880','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=4497880,`j`=4497880,`s`='4497880',`t`='4497880' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwcvXgAE= +ALTER TABLE `default`.`test2` PARTITION (`i`=2146641,`j`=2146641,`s`='2146641',`t`='2146641') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='2146641','highValue'='2146641','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=2146641,`j`=2146641,`s`='2146641',`t`='2146641' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg9/xOw== +ALTER TABLE `default`.`test2` PARTITION (`i`=967815,`j`=967815,`s`='967815',`t`='967815') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='967815','highValue'='967815','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=967815,`j`=967815,`s`='967815',`t`='967815' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgom62gM= +ALTER TABLE `default`.`test2` PARTITION (`i`=4604522,`j`=4604522,`s`='4604522',`t`='4604522') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4604522','highValue'='4604522','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=4604522,`j`=4604522,`s`='4604522',`t`='4604522' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgrrK+AY= +ALTER TABLE `default`.`test2` PARTITION (`i`=5238051,`j`=5238051,`s`='5238051',`t`='5238051') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='5238051','highValue'='5238051','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=5238051,`j`=5238051,`s`='5238051',`t`='5238051' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBhfjCjwE= +ALTER TABLE `default`.`test2` PARTITION (`i`=3187683,`j`=3187683,`s`='3187683',`t`='3187683') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3187683','highValue'='3187683','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3187683,`j`=3187683,`s`='3187683',`t`='3187683' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwZn8uAc= +ALTER TABLE `default`.`test2` PARTITION (`i`=6758530,`j`=6758530,`s`='6758530',`t`='6758530') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='6758530','highValue'='6758530','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=6758530,`j`=6758530,`s`='6758530',`t`='6758530' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgtSr0gQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=2581458,`j`=2581458,`s`='2581458',`t`='2581458') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='2581458','highValue'='2581458','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=2581458,`j`=2581458,`s`='2581458',`t`='2581458' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgf6ljQQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=4304398,`j`=4304398,`s`='4304398',`t`='4304398') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4304398','highValue'='4304398','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=4304398,`j`=4304398,`s`='4304398',`t`='4304398' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgf2tnwQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=812837,`j`=812837,`s`='812837',`t`='812837') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='812837','highValue'='812837','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=812837,`j`=812837,`s`='812837',`t`='812837' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwuja+gY= +-- ALTER TABLE `default`.`test2` PARTITION (`i`=2,`j`=2,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='2','highValue'='2','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=2,`j`=2,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBxIHguQQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=3356420,`j`=3356420,`s`='3356420',`t`='3356420') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3356420','highValue'='3356420','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3356420,`j`=3356420,`s`='3356420',`t`='3356420' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwa/logE= +-- ALTER TABLE `default`.`test2` PARTITION (`i`=__HIVE_DEFAULT_PARTITION__,`j`=__HIVE_DEFAULT_PARTITION__,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='0','highValue'='0','numNulls'='1','numDVs'='0' ); +ALTER TABLE `default`.`test2` PARTITION (`i`=42629,`j`=42629,`s`='42629',`t`='42629') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='42629','highValue'='42629','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=42629,`j`=42629,`s`='42629',`t`='42629' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBweiP4AY= +ALTER TABLE `default`.`test2` PARTITION (`i`=5247332,`j`=5247332,`s`='5247332',`t`='5247332') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='5247332','highValue'='5247332','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=5247332,`j`=5247332,`s`='5247332',`t`='5247332' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg5mT4Qc= +ALTER TABLE `default`.`test2` PARTITION (`i`=1565998,`j`=1565998,`s`='1565998',`t`='1565998') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1565998','highValue'='1565998','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=1565998,`j`=1565998,`s`='1565998',`t`='1565998' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBhKGjdQ== +ALTER TABLE `default`.`test2` PARTITION (`i`=1880938,`j`=1880938,`s`='1880938',`t`='1880938') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1880938','highValue'='1880938','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=1880938,`j`=1880938,`s`='1880938',`t`='1880938' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgb2svAQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=9805326,`j`=9805326,`s`='9805326',`t`='9805326') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='9805326','highValue'='9805326','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=9805326,`j`=9805326,`s`='9805326',`t`='9805326' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBw96RggU= +ALTER TABLE `default`.`test2` PARTITION (`i`=3076080,`j`=3076080,`s`='3076080',`t`='3076080') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3076080','highValue'='3076080','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3076080,`j`=3076080,`s`='3076080',`t`='3076080' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBweDt4QM= +ALTER TABLE `default`.`test2` PARTITION (`i`=4409637,`j`=4409637,`s`='4409637',`t`='4409637') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4409637','highValue'='4409637','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=4409637,`j`=4409637,`s`='4409637',`t`='4409637' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgsT4sAc= +ALTER TABLE `default`.`test2` PARTITION (`i`=6276003,`j`=6276003,`s`='6276003',`t`='6276003') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='6276003','highValue'='6276003','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=6276003,`j`=6276003,`s`='6276003',`t`='6276003' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBw8Kw8wQ= +-- ALTER TABLE `default`.`test2` PARTITION (`i`=1,`j`=1,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='1','highValue'='1','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=1,`j`=1,`s`='__HIVE_DEFAULT_PARTITION__',`t`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBwfO+SA== +ALTER TABLE `default`.`test2` PARTITION (`i`=3023419,`j`=3023419,`s`='3023419',`t`='3023419') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='3023419','highValue'='3023419','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=3023419,`j`=3023419,`s`='3023419',`t`='3023419' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBge2+4gQ= +ALTER TABLE `default`.`test2` PARTITION (`i`=4115101,`j`=4115101,`s`='4115101',`t`='4115101') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='4115101','highValue'='4115101','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=4115101,`j`=4115101,`s`='4115101',`t`='4115101' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgaamtAY= +ALTER TABLE `default`.`test2` PARTITION (`i`=971079,`j`=971079,`s`='971079',`t`='971079') UPDATE STATISTICS FOR COLUMN `foo` SET('lowValue'='971079','highValue'='971079','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test2` PARTITION `i`=971079,`j`=971079,`s`='971079',`t`='971079' FOR COLUMN `foo` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBg6XR2gc= diff --git a/ql/src/test/results/clientpositive/llap/partition_explain_ddl.q.out b/ql/src/test/results/clientpositive/llap/partition_explain_ddl.q.out index 1075f45b2fa1..1be1cc023b33 100644 --- a/ql/src/test/results/clientpositive/llap/partition_explain_ddl.q.out +++ b/ql/src/test/results/clientpositive/llap/partition_explain_ddl.q.out @@ -31,7 +31,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.add_part_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`add_part_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); @@ -142,13 +142,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.add_part_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR default.add_part_test PARTITION ds='2010-01-01' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR default.add_part_test PARTITION ds='2010-01-01' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== +ALTER TABLE `default`.`add_part_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-01'); +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `default`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `default`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== @@ -263,21 +263,21 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.add_part_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); -ALTER TABLE default.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02'); -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE default.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-03'); -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='225' ); -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR default.add_part_test PARTITION ds='2010-01-01' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR default.add_part_test PARTITION ds='2010-01-01' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='8.0','maxColLen'='8','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR default.add_part_test PARTITION ds='2010-01-03' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgaOnjASAnPDu//////8BwfPgSP+i5ViAxKHO/P////8BgIjfnQTA44A8gJ2G1///////AcSq3Y7//////wH84vzz/f////8BhYzE6vz/////AfyQ+MsH/4aiEoChh/v//////wHBvOTP//////8B -ALTER TABLE default.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='6.0','maxColLen'='6','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR default.add_part_test PARTITION ds='2010-01-03' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PxLrgqgHA37++Bv2T79H6/////wGAkZX+/v////8BwYzG8AWApP7O+f////8BgdfK4wT/m6bn+/////8B/9jt3wKC8sjC/f////8BvuyTzgPB1Mbz/P////8Bv5KKwgTBi8Ph+f////8BgIaQ5QM= +ALTER TABLE `default`.`add_part_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-01'); +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); +ALTER TABLE `default`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-02'); +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-03'); +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='225' ); +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `default`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `default`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='8.0','maxColLen'='8','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `default`.`add_part_test` PARTITION `ds`='2010-01-03' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgaOnjASAnPDu//////8BwfPgSP+i5ViAxKHO/P////8BgIjfnQTA44A8gJ2G1///////AcSq3Y7//////wH84vzz/f////8BhYzE6vz/////AfyQ+MsH/4aiEoChh/v//////wHBvOTP//////8B +ALTER TABLE `default`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='6.0','maxColLen'='6','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `default`.`add_part_test` PARTITION `ds`='2010-01-03' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PxLrgqgHA37++Bv2T79H6/////wGAkZX+/v////8BwYzG8AWApP7O+f////8BgdfK4wT/m6bn+/////8B/9jt3wKC8sjC/f////8BvuyTzgPB1Mbz/P////8Bv5KKwgTBi8Ph+f////8BgIaQ5QM= @@ -399,7 +399,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: add_part_test_db@add_part_test POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-01 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS add_part_test_db; +CREATE DATABASE IF NOT EXISTS `add_part_test_db`; CREATE TABLE `add_part_test_db`.`add_part_test`( `key` string, `value` string) @@ -416,13 +416,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE add_part_test_db.add_part_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-01' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-01' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== +ALTER TABLE `add_part_test_db`.`add_part_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-01'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== @@ -520,7 +520,7 @@ POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-01 POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-02 POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-03 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS add_part_test_db; +CREATE DATABASE IF NOT EXISTS `add_part_test_db`; CREATE TABLE `add_part_test_db`.`add_part_test`( `key` string, `value` string) @@ -537,21 +537,21 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE add_part_test_db.add_part_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-03'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='225' ); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-01' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-01' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='8.0','maxColLen'='8','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-03' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgaOnjASAnPDu//////8BwfPgSP+i5ViAxKHO/P////8BgIjfnQTA44A8gJ2G1///////AcSq3Y7//////wH84vzz/f////8BhYzE6vz/////AfyQ+MsH/4aiEoChh/v//////wHBvOTP//////8B -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='6.0','maxColLen'='6','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-03' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PxLrgqgHA37++Bv2T79H6/////wGAkZX+/v////8BwYzG8AWApP7O+f////8BgdfK4wT/m6bn+/////8B/9jt3wKC8sjC/f////8BvuyTzgPB1Mbz/P////8Bv5KKwgTBi8Ph+f////8BgIaQ5QM= +ALTER TABLE `add_part_test_db`.`add_part_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-01'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-02'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-03'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='225' ); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='8.0','maxColLen'='8','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-03' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgaOnjASAnPDu//////8BwfPgSP+i5ViAxKHO/P////8BgIjfnQTA44A8gJ2G1///////AcSq3Y7//////wH84vzz/f////8BhYzE6vz/////AfyQ+MsH/4aiEoChh/v//////wHBvOTP//////8B +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='6.0','maxColLen'='6','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-03' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PxLrgqgHA37++Bv2T79H6/////wGAkZX+/v////8BwYzG8AWApP7O+f////8BgdfK4wT/m6bn+/////8B/9jt3wKC8sjC/f////8BvuyTzgPB1Mbz/P////8Bv5KKwgTBi8Ph+f////8BgIaQ5QM= @@ -622,7 +622,7 @@ POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-01 POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-02 POSTHOOK: Input: add_part_test_db@add_part_test@ds=2010-01-03 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS add_part_test_db; +CREATE DATABASE IF NOT EXISTS `add_part_test_db`; CREATE TABLE `add_part_test_db`.`add_part_test`( `key` string, `value` string) @@ -639,21 +639,21 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE add_part_test_db.add_part_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE add_part_test_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-03'); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='225' ); -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-01' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-01') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-01' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS FOR COLUMN value SET('avgColLen'='8.0','maxColLen'='8','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-03' FOR COLUMN value BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgaOnjASAnPDu//////8BwfPgSP+i5ViAxKHO/P////8BgIjfnQTA44A8gJ2G1///////AcSq3Y7//////wH84vzz/f////8BhYzE6vz/////AfyQ+MsH/4aiEoChh/v//////wHBvOTP//////8B -ALTER TABLE add_part_test_db.add_part_test PARTITION (ds='2010-01-03') UPDATE STATISTICS FOR COLUMN key SET('avgColLen'='6.0','maxColLen'='6','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR add_part_test_db.add_part_test PARTITION ds='2010-01-03' FOR COLUMN key BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PxLrgqgHA37++Bv2T79H6/////wGAkZX+/v////8BwYzG8AWApP7O+f////8BgdfK4wT/m6bn+/////8B/9jt3wKC8sjC/f////8BvuyTzgPB1Mbz/P////8Bv5KKwgTBi8Ph+f////8BgIaQ5QM= +ALTER TABLE `add_part_test_db`.`add_part_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-01'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS SET('numRows'='19','rawDataSize'='267' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-02'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-02') UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `add_part_test_db`.`add_part_test` ADD IF NOT EXISTS PARTITION (`ds`='2010-01-03'); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='225' ); +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='7.526315789473684','maxColLen'='8','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTwq6o+gLE0trKAvuXwdz6/////wGAp4ikAcCYgKYEgKv42f3/////AcLt7oMD/orRm///////AcDeotcBwOvPh/r/////AYDyz7kGwM+kvvz/////AYD90bn+/////wGBytzd/v////8Bv9fU1v//////AYDRh8UEwf6S/v3/////Ab+aqtIChMGNvP7/////AQ== +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-01') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='5.526315789473684','maxColLen'='6','numNulls'='0','numDVs'='19' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-01' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBMTga///AODuuFlwpGjgAL7r+CB+/////8BgPGY2AGB8bqnA8TDyvT6/////wG75PK0BMDg56b6/////wGAn/E7wM3HswOA1rnX/P////8BguujpwOB4qzYAv6j2hD/wYu4+v////8BgIKhwAGAge7KA8Cop9b//////wE= +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS FOR COLUMN `value` SET('avgColLen'='8.0','maxColLen'='8','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-03' FOR COLUMN `value` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgaOnjASAnPDu//////8BwfPgSP+i5ViAxKHO/P////8BgIjfnQTA44A8gJ2G1///////AcSq3Y7//////wH84vzz/f////8BhYzE6vz/////AfyQ+MsH/4aiEoChh/v//////wHBvOTP//////8B +ALTER TABLE `add_part_test_db`.`add_part_test` PARTITION (`ds`='2010-01-03') UPDATE STATISTICS FOR COLUMN `key` SET('avgColLen'='6.0','maxColLen'='6','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `add_part_test_db`.`add_part_test` PARTITION `ds`='2010-01-03' FOR COLUMN `key` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PxLrgqgHA37++Bv2T79H6/////wGAkZX+/v////8BwYzG8AWApP7O+f////8BgdfK4wT/m6bn+/////8B/9jt3wKC8sjC/f////8BvuyTzgPB1Mbz/P////8Bv5KKwgTBi8Ph+f////8BgIaQ5QM= @@ -864,7 +864,7 @@ POSTHOOK: Input: db_bdpbase@default_partition_test@sports=Football POSTHOOK: Input: db_bdpbase@default_partition_test@sports=Tennis POSTHOOK: Input: db_bdpbase@default_partition_test@sports=__HIVE_DEFAULT_PARTITION__ #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db_bdpbase; +CREATE DATABASE IF NOT EXISTS `db_bdpbase`; CREATE TABLE `db_bdpbase`.`default_partition_test`( `id` int, `firstname` string, @@ -887,79 +887,79 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db_bdpbase.default_partition_test UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE db_bdpbase.default_partition_test ADD IF NOT EXISTS PARTITION (sports='Badminton'); -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Badminton') UPDATE STATISTICS SET('numRows'='26','rawDataSize'='985' ); -ALTER TABLE db_bdpbase.default_partition_test ADD IF NOT EXISTS PARTITION (sports='Basketball'); -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Basketball') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='588' ); -ALTER TABLE db_bdpbase.default_partition_test ADD IF NOT EXISTS PARTITION (sports='Cricket'); -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Cricket') UPDATE STATISTICS SET('numRows'='14','rawDataSize'='550' ); -ALTER TABLE db_bdpbase.default_partition_test ADD IF NOT EXISTS PARTITION (sports='Football'); -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Football') UPDATE STATISTICS SET('numRows'='23','rawDataSize'='898' ); -ALTER TABLE db_bdpbase.default_partition_test ADD IF NOT EXISTS PARTITION (sports='Tennis'); -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Tennis') UPDATE STATISTICS SET('numRows'='16','rawDataSize'='635' ); --- ALTER TABLE db_bdpbase.default_partition_test ADD IF NOT EXISTS PARTITION (sports='__HIVE_DEFAULT_PARTITION__'); --- ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='6','rawDataSize'='247' ); -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Tennis') UPDATE STATISTICS FOR COLUMN lastname SET('avgColLen'='6.625','maxColLen'='10','numNulls'='0','numDVs'='16' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Tennis' FOR COLUMN lastname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQw9jy4wbApZ6a/v////8B/sPU3fv/////AcCHp9kGgN/fv/r/////AYLFxPn+/////wGBkrK3Av2Ry8cCgc676Pv/////Ab/A89YBxOTiEvzixO/+/////wHD55ygA77R9+T6/////wHFq+mVB7vOpY39/////wE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Tennis') UPDATE STATISTICS FOR COLUMN id SET('lowValue'='1007','highValue'='1086','numNulls'='0','numDVs'='16' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Tennis' FOR COLUMN id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQgsirL7+NuZgFgJSyrf7/////AcDw1/3//////wGAw9/iAoGurff//////wH//tG+/v////8BhMDytf//////Abya/vb+/////wHCpbPM//////8B/v+D1v7/////AcD2p9r//////wGApJdwwLKYgQTByo7U//////8BwdS26vv/////AQ== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Tennis') UPDATE STATISTICS FOR COLUMN firstname SET('avgColLen'='5.875','maxColLen'='10','numNulls'='0','numDVs'='16' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Tennis' FOR COLUMN firstname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQgp+Lswf/6/Gw+/////8BgM2YkAPB0+LF/f////8B/8ulC8DngZ0CgJ+o9v//////AcDR9AWBwp7N+v////8BwZT4/QH/qIfj//////8BgP+t//3/////Ab+a5YgDxoiV3QK8+K/t/v////8Bgdruk/7/////AQ== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Tennis') UPDATE STATISTICS FOR COLUMN country SET('avgColLen'='9.5625','maxColLen'='27','numNulls'='0','numDVs'='16' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Tennis' FOR COLUMN country BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQweClxwTAto6P/P////8Bg5qv8gO9nZOuAYWnjJT8/////wH7y9SkAoD13gGBtJOV/v////8BgqrYtf7/////Af3z25EEgbjb3wK/2Jud//////8BgZbmgvr/////Af+XtN4Dga/pvP//////AcH6mY8B -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Tennis') UPDATE STATISTICS FOR COLUMN city SET('avgColLen'='9.5625','maxColLen'='18','numNulls'='0','numDVs'='16' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Tennis' FOR COLUMN city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQgfSH0gPAg7CJAoCl9ljC0YLK//////8B/picz/7/////AcDlwtP7/////wHAl84twIbP+v//////AYCoomaApeWJBoG1lab6/////wH/sNr5AoDjqJr8/////wHBz+68BcCH7fb9/////wH/+7xK -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Basketball') UPDATE STATISTICS FOR COLUMN lastname SET('avgColLen'='6.2','maxColLen'='8','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Basketball' FOR COLUMN lastname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgZ/tvgSA18jgAoGZwtb+/////wH/zaeIAcDWpIP//////wGC/43R+/////8B/5/ogwGDhYerAfz89K4DgL6l8v7/////AYDwwZj//////wHAlpaV/f////8BgLqSK8GJg9D9/////wHCiZiWAw== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Basketball') UPDATE STATISTICS FOR COLUMN id SET('lowValue'='1001','highValue'='1097','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Basketball' FOR COLUMN id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgYCNxQOA2pic/f////8BxuPc2AP8m7G4A7+Ix+z7/////wG/xNKo/f////8BgLCJswSAxK+U/f////8BgPmo2gHE6pNr/pSPjQK+4MTY+v////8BgIiW4ASEgZT0+/////8B/OCGvQE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Basketball') UPDATE STATISTICS FOR COLUMN firstname SET('avgColLen'='6.466666666666667','maxColLen'='9','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Basketball' FOR COLUMN firstname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgbilpAeC7+ssvtnsxfj/////AcCW6PAFwLLw9f7/////AcHwo4v9/////wH/nYz3AoGH1J/9/////wH/7Jv7/f////8BxNrM2wK85uvN/v////8BgJnvHMCdv9b//////wGAw7/oBcLAloD5/////wE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Basketball') UPDATE STATISTICS FOR COLUMN country SET('avgColLen'='8.666666666666666','maxColLen'='16','numNulls'='0','numDVs'='14' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Basketball' FOR COLUMN country BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4OhdzCtAL8jYW5A8Ce5MX6/////wHAicoqgNKYqAGA+cYigKy/jwGB3qTH/f////8BgNG/hwLDkchH/YOnjf//////Af/knsj9/////wHA+9/oBICtjo7//////wE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Basketball') UPDATE STATISTICS FOR COLUMN city SET('avgColLen'='9.733333333333333','maxColLen'='26','numNulls'='0','numDVs'='15' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Basketball' FOR COLUMN city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgZjVoAbAiv9ng+yLbv3bzYP8/////wGFitRK/MrIowP/zYSG+/////8BwvzCl/7/////Af+jp8UBv+6Ctf//////AYGf/9wDv6/ZgQLCvb65+f////8BvsGr/QKAtqUo -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Football') UPDATE STATISTICS FOR COLUMN lastname SET('avgColLen'='5.6521739130434785','maxColLen'='8','numNulls'='0','numDVs'='23' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Football' FOR COLUMN lastname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXgqW6lwS/7cK2AsGpjOz7/////wGCm5fW//////8BvbT96wTA5rbf/P////8Bgoq+hwO/6qXp/P////8B/8GzqP7/////AcCqq+L//////wGA5PWu/v////8Bwa37xwa/u+SQ/P////8Bwcrsrf3/////Ab+2k+ADgNahiAHAyNff/v////8BwNftnP7/////AYDDkyfB/tKCA4DswM8Bgoeop/3/////Af3g8/oB -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Football') UPDATE STATISTICS FOR COLUMN id SET('lowValue'='1006','highValue'='1100','numNulls'='0','numDVs'='23' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Football' FOR COLUMN id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXgaCXhwXA1P+S/f////8Bg93J8wG91quf/P////8Bg8OW9gG9v/KA//////8Bhbns+AW7mZtJwM2xGcCm2JD7/////wHAzv2WAsDMyob9/////wHCpeL4/v////8Bv4iY4wW/56Sb+f////8BgLbX1QHB2tTlBP+Sqp36/////wGA9fLJAcGhkpcBgIXC0QPBxYbz//////8B/uGZ+vv/////AQ== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Football') UPDATE STATISTICS FOR COLUMN firstname SET('avgColLen'='5.782608695652174','maxColLen'='9','numNulls'='0','numDVs'='23' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Football' FOR COLUMN firstname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXwbS80wPA08+OBMDEmZT6/////wHA7fi1BMLU84kBvs2zyfr/////AcDRiGfCgsLrA4DDgsb9/////wG+ipqK/v////8BwMjJowWC5Jyw//////8Bv5j19Pn/////AYDBosL//////wHA9t7fAb/QhPkDgJCxk/v/////AYKK0Qb+ra2ZAoOA5Pj//////wH9vuKJAcH15av8/////wGA1v/hAg== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Football') UPDATE STATISTICS FOR COLUMN country SET('avgColLen'='10.826086956521738','maxColLen'='33','numNulls'='0','numDVs'='22' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Football' FOR COLUMN country BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBYWw66T/gX/gadGwYr4n/3/////AYG57zXAkP/9+/////8BvbWmigfB/KHo+v////8BwYzq0gS/zsX3/f////8B//i05Pz/////AcG9m/4D//vz0fz/////AcCqpx3AxKqpAcCF+PACgIGGwPv/////AYSQlM0B/fqH6v//////Af+1/M38/////wGCp8si/pO8vQKC4MSEAQ== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Football') UPDATE STATISTICS FOR COLUMN city SET('avgColLen'='8.782608695652174','maxColLen'='20','numNulls'='0','numDVs'='23' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Football' FOR COLUMN city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXw7/xvgW/vaexAoD6oqj6/////wHBqdfw//////8BvuPJ1P7/////AYCN2LgDwaXjsv3/////Af+IxJcGwbyujvv/////Af+ttFnBqYmkA7+Yrqb7/////wHAgr+LBIGHpJP9/////wH/iY3oAYDAj7/8/////wGAie6wAYD20A3Cw5oi/ozl0QKAl+LVAcHRpASBoNS9//////8B --- ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN lastname SET('avgColLen'='6.333333333333333','maxColLen'='8','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='__HIVE_DEFAULT_PARTITION__' FOR COLUMN lastname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGgbePuQfB9sjg+P////8Bw93qogW83dm9/P////8BwYbtqQK/yYTa/v////8B --- ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN id SET('lowValue'='1005','highValue'='1098','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='__HIVE_DEFAULT_PARTITION__' FOR COLUMN id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGgeXMkgKAqeDgAYDOgWfAjve+/P////8Bga6jv///////Af/B69cD --- ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN firstname SET('avgColLen'='7.0','maxColLen'='9','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='__HIVE_DEFAULT_PARTITION__' FOR COLUMN firstname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwe3GFsGq05QBv6XU8P7/////AYHCq/AGwJ7v2Pr/////AcG15yw= --- ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN country SET('avgColLen'='9.333333333333334','maxColLen'='15','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='__HIVE_DEFAULT_PARTITION__' FOR COLUMN country BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwaXO9gaA4928+f////8BgoXl0AP/u9nv/v////8BwP3t9gP/wbzN/P////8B --- ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN city SET('avgColLen'='10.5','maxColLen'='17','numNulls'='0','numDVs'='6' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='__HIVE_DEFAULT_PARTITION__' FOR COLUMN city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwaGlyQKAyt+3AYCs6oQBwt6jv/7/////Af/P9r4C//fig/v/////AQ== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Cricket') UPDATE STATISTICS FOR COLUMN lastname SET('avgColLen'='6.642857142857143','maxColLen'='10','numNulls'='0','numDVs'='14' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Cricket' FOR COLUMN lastname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4Oh/eB3gH6kIqEBoDm49H7/////wGBzfvyAcDCzlTC2eaU/P////8B/q3H8v3/////Af+bxdoFgJL6jPv/////AYG84W//1I1igqC6pwH+kKX+/v////8Bwr362gQ= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Cricket') UPDATE STATISTICS FOR COLUMN id SET('lowValue'='1002','highValue'='1099','numNulls'='0','numDVs'='14' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Cricket' FOR COLUMN id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4Owp+5vgXAgIGC+/////8B/6frowTArIixAcCvrknA3Jmv//////8BgIHmBoGA38n6/////wHAhO5lgs76yAG+9dT6Ar+cjO36/////wGAx7GYBML+3Mj8/////wE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Cricket') UPDATE STATISTICS FOR COLUMN firstname SET('avgColLen'='5.285714285714286','maxColLen'='9','numNulls'='0','numDVs'='14' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Cricket' FOR COLUMN firstname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4OwbS80wOB6fyi/f////8B/4eU6QKB+P2a/f////8Bw7z4qQT8+8KdAsKaipb9/////wH/pLfZ//////8Bg/367/3/////Af72zOj//////wG+8a95gvbxuf3/////Ab7LzcwFwNTv8Pz/////AQ== -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Cricket') UPDATE STATISTICS FOR COLUMN country SET('avgColLen'='9.714285714285714','maxColLen'='18','numNulls'='0','numDVs'='14' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Cricket' FOR COLUMN country BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4OwanPogfEsvOR+/////8B/M6b8AKCibP3/f////8Bvprl2gLB+fLA//////8B/7zUlv7/////AcXwxLz+/////wG7+JuG/v////8BgPWXlAHB1KGNAv/45vkDwvKSh/n/////AYSysbYC -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Cricket') UPDATE STATISTICS FOR COLUMN city SET('avgColLen'='9.5','maxColLen'='18','numNulls'='0','numDVs'='14' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Cricket' FOR COLUMN city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4Owv2KsweAxJ3Y+f////8Bv7aO5QaAo7XI+v////8BwM6p9/3/////AcCGkuoBwYfi6f//////Ab/3+6ICxPu9zwP90YX2//////8BgpXXv/j/////Af3c9qcEgZnkk/z/////Af/BmTE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Badminton') UPDATE STATISTICS FOR COLUMN lastname SET('avgColLen'='6.269230769230769','maxColLen'='9','numNulls'='0','numDVs'='26' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Badminton' FOR COLUMN lastname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoaw5bxWsDKgKgE/pbs5vz/////AYDw18AEguubl/3/////AcHBj579/////wH9pu6ABYKuotL+/////wH+x+S0//////8BgMj+2QLAjorG+f////8BwJm1pwLBgbeMBL/BgaD8/////wHAuJDKA4CXh93//////wHBuPXl+v////8B//SC/wWAweCB//////8BwMTBhv7/////AYC+l/cBwI3g1fr/////AYTay4cBvtbOsAL//sj+A8Hd+ZH4/////wE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Badminton') UPDATE STATISTICS FOR COLUMN id SET('lowValue'='1004','highValue'='1094','numNulls'='0','numDVs'='26' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Badminton' FOR COLUMN id BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoahd3Cdf32590G/6SLj/3/////AYCykT7Av92e/f////8BwYfGqwP/9oKWAYD4z638/////wGBwrxzv+i89wHA0Niv/f////8BgLL2rgGAstnL+/////8BguuQrAH+1vPaBMG99/D5/////wH//4P6BcDSr2WAnKXH/P////8BgKfB3P3/////AYCTgJEEgqb8+Pv/////AcCCvmq+0O7IAoDBhvb+/////wHA0dlC -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Badminton') UPDATE STATISTICS FOR COLUMN firstname SET('avgColLen'='5.615384615384615','maxColLen'='8','numNulls'='0','numDVs'='25' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Badminton' FOR COLUMN firstname BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBkZyJCAxAK5lvTQBMDlqKP5/////wHAl7CAA8CbmRzA8faZ/f////8Bgf/q/f//////Ab/LlfIGgNTwqvz/////AcDcuH6Ay4aWAoDC4QXA6L+q/f////8BwPv+XoDLzxPAzvF9waDs/Pz/////AcCQvbL+/////wHAuJKC//////8Bv8CUgAXAzNZPw+WL8vz/////AYD4gCH9tourAcPksWU= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Badminton') UPDATE STATISTICS FOR COLUMN country SET('avgColLen'='10.038461538461538','maxColLen'='44','numNulls'='0','numDVs'='26' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Badminton' FOR COLUMN country BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoahbLBzAP85PKJ/f////8BwLe62gLD4vSXAYGN3q0B/6Xhi/r/////Ab2osqUFwqq+oQH+tdPa+v////8BgPenYoCwhkKAndeXA4Sn15/+/////wH+le6UAr/epnP/7bOp+v////8BgKHJV4DwkcACg4u68wGB85+S/f////8B/JyLowOAs7kWgYmk8fr/////Af/OrLoDgO721P//////AYGQ3sD7/////wE= -ALTER TABLE db_bdpbase.default_partition_test PARTITION (sports='Badminton') UPDATE STATISTICS FOR COLUMN city SET('avgColLen'='7.961538461538462','maxColLen'='15','numNulls'='0','numDVs'='26' ); --- BIT VECTORS PRESENT FOR db_bdpbase.default_partition_test PARTITION sports='Badminton' FOR COLUMN city BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoagcnkoQHDmv3pA/6E55gChOm3qP//////AbzE6Yj7/////wH/jeLHAYC7qMgEgO/pvP//////AYDl7q35/////wHAnoCTBsDeqIT+/////wGArM7x//////8BwdCEL//10sL+/////wGB/Yxrv/G7V8HV1cT7/////wGA/62HA4C7sfsBwJT/rPr/////AYCFgrIBgbbW6P//////Ab707dv//////wGEkeKsBYLSvzy6mZuX+v////8B +ALTER TABLE `db_bdpbase`.`default_partition_test` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db_bdpbase`.`default_partition_test` ADD IF NOT EXISTS PARTITION (`sports`='Badminton'); +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Badminton') UPDATE STATISTICS SET('numRows'='26','rawDataSize'='985' ); +ALTER TABLE `db_bdpbase`.`default_partition_test` ADD IF NOT EXISTS PARTITION (`sports`='Basketball'); +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Basketball') UPDATE STATISTICS SET('numRows'='15','rawDataSize'='588' ); +ALTER TABLE `db_bdpbase`.`default_partition_test` ADD IF NOT EXISTS PARTITION (`sports`='Cricket'); +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Cricket') UPDATE STATISTICS SET('numRows'='14','rawDataSize'='550' ); +ALTER TABLE `db_bdpbase`.`default_partition_test` ADD IF NOT EXISTS PARTITION (`sports`='Football'); +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Football') UPDATE STATISTICS SET('numRows'='23','rawDataSize'='898' ); +ALTER TABLE `db_bdpbase`.`default_partition_test` ADD IF NOT EXISTS PARTITION (`sports`='Tennis'); +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Tennis') UPDATE STATISTICS SET('numRows'='16','rawDataSize'='635' ); +-- ALTER TABLE `db_bdpbase`.`default_partition_test` ADD IF NOT EXISTS PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__'); +-- ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS SET('numRows'='6','rawDataSize'='247' ); +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Tennis') UPDATE STATISTICS FOR COLUMN `lastname` SET('avgColLen'='6.625','maxColLen'='10','numNulls'='0','numDVs'='16' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Tennis' FOR COLUMN `lastname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQw9jy4wbApZ6a/v////8B/sPU3fv/////AcCHp9kGgN/fv/r/////AYLFxPn+/////wGBkrK3Av2Ry8cCgc676Pv/////Ab/A89YBxOTiEvzixO/+/////wHD55ygA77R9+T6/////wHFq+mVB7vOpY39/////wE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Tennis') UPDATE STATISTICS FOR COLUMN `id` SET('lowValue'='1007','highValue'='1086','numNulls'='0','numDVs'='16' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Tennis' FOR COLUMN `id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQgsirL7+NuZgFgJSyrf7/////AcDw1/3//////wGAw9/iAoGurff//////wH//tG+/v////8BhMDytf//////Abya/vb+/////wHCpbPM//////8B/v+D1v7/////AcD2p9r//////wGApJdwwLKYgQTByo7U//////8BwdS26vv/////AQ== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Tennis') UPDATE STATISTICS FOR COLUMN `firstname` SET('avgColLen'='5.875','maxColLen'='10','numNulls'='0','numDVs'='16' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Tennis' FOR COLUMN `firstname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQgp+Lswf/6/Gw+/////8BgM2YkAPB0+LF/f////8B/8ulC8DngZ0CgJ+o9v//////AcDR9AWBwp7N+v////8BwZT4/QH/qIfj//////8BgP+t//3/////Ab+a5YgDxoiV3QK8+K/t/v////8Bgdruk/7/////AQ== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Tennis') UPDATE STATISTICS FOR COLUMN `country` SET('avgColLen'='9.5625','maxColLen'='27','numNulls'='0','numDVs'='16' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Tennis' FOR COLUMN `country` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQweClxwTAto6P/P////8Bg5qv8gO9nZOuAYWnjJT8/////wH7y9SkAoD13gGBtJOV/v////8BgqrYtf7/////Af3z25EEgbjb3wK/2Jud//////8BgZbmgvr/////Af+XtN4Dga/pvP//////AcH6mY8B +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Tennis') UPDATE STATISTICS FOR COLUMN `city` SET('avgColLen'='9.5625','maxColLen'='18','numNulls'='0','numDVs'='16' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Tennis' FOR COLUMN `city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBAQgfSH0gPAg7CJAoCl9ljC0YLK//////8B/picz/7/////AcDlwtP7/////wHAl84twIbP+v//////AYCoomaApeWJBoG1lab6/////wH/sNr5AoDjqJr8/////wHBz+68BcCH7fb9/////wH/+7xK +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Basketball') UPDATE STATISTICS FOR COLUMN `lastname` SET('avgColLen'='6.2','maxColLen'='8','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Basketball' FOR COLUMN `lastname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgZ/tvgSA18jgAoGZwtb+/////wH/zaeIAcDWpIP//////wGC/43R+/////8B/5/ogwGDhYerAfz89K4DgL6l8v7/////AYDwwZj//////wHAlpaV/f////8BgLqSK8GJg9D9/////wHCiZiWAw== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Basketball') UPDATE STATISTICS FOR COLUMN `id` SET('lowValue'='1001','highValue'='1097','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Basketball' FOR COLUMN `id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgYCNxQOA2pic/f////8BxuPc2AP8m7G4A7+Ix+z7/////wG/xNKo/f////8BgLCJswSAxK+U/f////8BgPmo2gHE6pNr/pSPjQK+4MTY+v////8BgIiW4ASEgZT0+/////8B/OCGvQE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Basketball') UPDATE STATISTICS FOR COLUMN `firstname` SET('avgColLen'='6.466666666666667','maxColLen'='9','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Basketball' FOR COLUMN `firstname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgbilpAeC7+ssvtnsxfj/////AcCW6PAFwLLw9f7/////AcHwo4v9/////wH/nYz3AoGH1J/9/////wH/7Jv7/f////8BxNrM2wK85uvN/v////8BgJnvHMCdv9b//////wGAw7/oBcLAloD5/////wE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Basketball') UPDATE STATISTICS FOR COLUMN `country` SET('avgColLen'='8.666666666666666','maxColLen'='16','numNulls'='0','numDVs'='14' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Basketball' FOR COLUMN `country` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4OhdzCtAL8jYW5A8Ce5MX6/////wHAicoqgNKYqAGA+cYigKy/jwGB3qTH/f////8BgNG/hwLDkchH/YOnjf//////Af/knsj9/////wHA+9/oBICtjo7//////wE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Basketball') UPDATE STATISTICS FOR COLUMN `city` SET('avgColLen'='9.733333333333333','maxColLen'='26','numNulls'='0','numDVs'='15' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Basketball' FOR COLUMN `city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA8PgZjVoAbAiv9ng+yLbv3bzYP8/////wGFitRK/MrIowP/zYSG+/////8BwvzCl/7/////Af+jp8UBv+6Ctf//////AYGf/9wDv6/ZgQLCvb65+f////8BvsGr/QKAtqUo +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Football') UPDATE STATISTICS FOR COLUMN `lastname` SET('avgColLen'='5.6521739130434785','maxColLen'='8','numNulls'='0','numDVs'='23' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Football' FOR COLUMN `lastname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXgqW6lwS/7cK2AsGpjOz7/////wGCm5fW//////8BvbT96wTA5rbf/P////8Bgoq+hwO/6qXp/P////8B/8GzqP7/////AcCqq+L//////wGA5PWu/v////8Bwa37xwa/u+SQ/P////8Bwcrsrf3/////Ab+2k+ADgNahiAHAyNff/v////8BwNftnP7/////AYDDkyfB/tKCA4DswM8Bgoeop/3/////Af3g8/oB +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Football') UPDATE STATISTICS FOR COLUMN `id` SET('lowValue'='1006','highValue'='1100','numNulls'='0','numDVs'='23' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Football' FOR COLUMN `id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXgaCXhwXA1P+S/f////8Bg93J8wG91quf/P////8Bg8OW9gG9v/KA//////8Bhbns+AW7mZtJwM2xGcCm2JD7/////wHAzv2WAsDMyob9/////wHCpeL4/v////8Bv4iY4wW/56Sb+f////8BgLbX1QHB2tTlBP+Sqp36/////wGA9fLJAcGhkpcBgIXC0QPBxYbz//////8B/uGZ+vv/////AQ== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Football') UPDATE STATISTICS FOR COLUMN `firstname` SET('avgColLen'='5.782608695652174','maxColLen'='9','numNulls'='0','numDVs'='23' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Football' FOR COLUMN `firstname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXwbS80wPA08+OBMDEmZT6/////wHA7fi1BMLU84kBvs2zyfr/////AcDRiGfCgsLrA4DDgsb9/////wG+ipqK/v////8BwMjJowWC5Jyw//////8Bv5j19Pn/////AYDBosL//////wHA9t7fAb/QhPkDgJCxk/v/////AYKK0Qb+ra2ZAoOA5Pj//////wH9vuKJAcH15av8/////wGA1v/hAg== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Football') UPDATE STATISTICS FOR COLUMN `country` SET('avgColLen'='10.826086956521738','maxColLen'='33','numNulls'='0','numDVs'='22' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Football' FOR COLUMN `country` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBYWw66T/gX/gadGwYr4n/3/////AYG57zXAkP/9+/////8BvbWmigfB/KHo+v////8BwYzq0gS/zsX3/f////8B//i05Pz/////AcG9m/4D//vz0fz/////AcCqpx3AxKqpAcCF+PACgIGGwPv/////AYSQlM0B/fqH6v//////Af+1/M38/////wGCp8si/pO8vQKC4MSEAQ== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Football') UPDATE STATISTICS FOR COLUMN `city` SET('avgColLen'='8.782608695652174','maxColLen'='20','numNulls'='0','numDVs'='23' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Football' FOR COLUMN `city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBcXw7/xvgW/vaexAoD6oqj6/////wHBqdfw//////8BvuPJ1P7/////AYCN2LgDwaXjsv3/////Af+IxJcGwbyujvv/////Af+ttFnBqYmkA7+Yrqb7/////wHAgr+LBIGHpJP9/////wH/iY3oAYDAj7/8/////wGAie6wAYD20A3Cw5oi/ozl0QKAl+LVAcHRpASBoNS9//////8B +-- ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `lastname` SET('avgColLen'='6.333333333333333','maxColLen'='8','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `lastname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGgbePuQfB9sjg+P////8Bw93qogW83dm9/P////8BwYbtqQK/yYTa/v////8B +-- ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `id` SET('lowValue'='1005','highValue'='1098','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGgeXMkgKAqeDgAYDOgWfAjve+/P////8Bga6jv///////Af/B69cD +-- ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `firstname` SET('avgColLen'='7.0','maxColLen'='9','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `firstname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwe3GFsGq05QBv6XU8P7/////AYHCq/AGwJ7v2Pr/////AcG15yw= +-- ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `country` SET('avgColLen'='9.333333333333334','maxColLen'='15','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `country` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwaXO9gaA4928+f////8BgoXl0AP/u9nv/v////8BwP3t9gP/wbzN/P////8B +-- ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='__HIVE_DEFAULT_PARTITION__') UPDATE STATISTICS FOR COLUMN `city` SET('avgColLen'='10.5','maxColLen'='17','numNulls'='0','numDVs'='6' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='__HIVE_DEFAULT_PARTITION__' FOR COLUMN `city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAYGwaGlyQKAyt+3AYCs6oQBwt6jv/7/////Af/P9r4C//fig/v/////AQ== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Cricket') UPDATE STATISTICS FOR COLUMN `lastname` SET('avgColLen'='6.642857142857143','maxColLen'='10','numNulls'='0','numDVs'='14' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Cricket' FOR COLUMN `lastname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4Oh/eB3gH6kIqEBoDm49H7/////wGBzfvyAcDCzlTC2eaU/P////8B/q3H8v3/////Af+bxdoFgJL6jPv/////AYG84W//1I1igqC6pwH+kKX+/v////8Bwr362gQ= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Cricket') UPDATE STATISTICS FOR COLUMN `id` SET('lowValue'='1002','highValue'='1099','numNulls'='0','numDVs'='14' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Cricket' FOR COLUMN `id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4Owp+5vgXAgIGC+/////8B/6frowTArIixAcCvrknA3Jmv//////8BgIHmBoGA38n6/////wHAhO5lgs76yAG+9dT6Ar+cjO36/////wGAx7GYBML+3Mj8/////wE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Cricket') UPDATE STATISTICS FOR COLUMN `firstname` SET('avgColLen'='5.285714285714286','maxColLen'='9','numNulls'='0','numDVs'='14' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Cricket' FOR COLUMN `firstname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4OwbS80wOB6fyi/f////8B/4eU6QKB+P2a/f////8Bw7z4qQT8+8KdAsKaipb9/////wH/pLfZ//////8Bg/367/3/////Af72zOj//////wG+8a95gvbxuf3/////Ab7LzcwFwNTv8Pz/////AQ== +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Cricket') UPDATE STATISTICS FOR COLUMN `country` SET('avgColLen'='9.714285714285714','maxColLen'='18','numNulls'='0','numDVs'='14' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Cricket' FOR COLUMN `country` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4OwanPogfEsvOR+/////8B/M6b8AKCibP3/f////8Bvprl2gLB+fLA//////8B/7zUlv7/////AcXwxLz+/////wG7+JuG/v////8BgPWXlAHB1KGNAv/45vkDwvKSh/n/////AYSysbYC +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Cricket') UPDATE STATISTICS FOR COLUMN `city` SET('avgColLen'='9.5','maxColLen'='18','numNulls'='0','numDVs'='14' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Cricket' FOR COLUMN `city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoA4Owv2KsweAxJ3Y+f////8Bv7aO5QaAo7XI+v////8BwM6p9/3/////AcCGkuoBwYfi6f//////Ab/3+6ICxPu9zwP90YX2//////8BgpXXv/j/////Af3c9qcEgZnkk/z/////Af/BmTE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Badminton') UPDATE STATISTICS FOR COLUMN `lastname` SET('avgColLen'='6.269230769230769','maxColLen'='9','numNulls'='0','numDVs'='26' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Badminton' FOR COLUMN `lastname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoaw5bxWsDKgKgE/pbs5vz/////AYDw18AEguubl/3/////AcHBj579/////wH9pu6ABYKuotL+/////wH+x+S0//////8BgMj+2QLAjorG+f////8BwJm1pwLBgbeMBL/BgaD8/////wHAuJDKA4CXh93//////wHBuPXl+v////8B//SC/wWAweCB//////8BwMTBhv7/////AYC+l/cBwI3g1fr/////AYTay4cBvtbOsAL//sj+A8Hd+ZH4/////wE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Badminton') UPDATE STATISTICS FOR COLUMN `id` SET('lowValue'='1004','highValue'='1094','numNulls'='0','numDVs'='26' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Badminton' FOR COLUMN `id` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoahd3Cdf32590G/6SLj/3/////AYCykT7Av92e/f////8BwYfGqwP/9oKWAYD4z638/////wGBwrxzv+i89wHA0Niv/f////8BgLL2rgGAstnL+/////8BguuQrAH+1vPaBMG99/D5/////wH//4P6BcDSr2WAnKXH/P////8BgKfB3P3/////AYCTgJEEgqb8+Pv/////AcCCvmq+0O7IAoDBhvb+/////wHA0dlC +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Badminton') UPDATE STATISTICS FOR COLUMN `firstname` SET('avgColLen'='5.615384615384615','maxColLen'='8','numNulls'='0','numDVs'='25' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Badminton' FOR COLUMN `firstname` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBkZyJCAxAK5lvTQBMDlqKP5/////wHAl7CAA8CbmRzA8faZ/f////8Bgf/q/f//////Ab/LlfIGgNTwqvz/////AcDcuH6Ay4aWAoDC4QXA6L+q/f////8BwPv+XoDLzxPAzvF9waDs/Pz/////AcCQvbL+/////wHAuJKC//////8Bv8CUgAXAzNZPw+WL8vz/////AYD4gCH9tourAcPksWU= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Badminton') UPDATE STATISTICS FOR COLUMN `country` SET('avgColLen'='10.038461538461538','maxColLen'='44','numNulls'='0','numDVs'='26' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Badminton' FOR COLUMN `country` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoahbLBzAP85PKJ/f////8BwLe62gLD4vSXAYGN3q0B/6Xhi/r/////Ab2osqUFwqq+oQH+tdPa+v////8BgPenYoCwhkKAndeXA4Sn15/+/////wH+le6UAr/epnP/7bOp+v////8BgKHJV4DwkcACg4u68wGB85+S/f////8B/JyLowOAs7kWgYmk8fr/////Af/OrLoDgO721P//////AYGQ3sD7/////wE= +ALTER TABLE `db_bdpbase`.`default_partition_test` PARTITION (`sports`='Badminton') UPDATE STATISTICS FOR COLUMN `city` SET('avgColLen'='7.961538461538462','maxColLen'='15','numNulls'='0','numDVs'='26' ); +-- BIT VECTORS PRESENT FOR `db_bdpbase`.`default_partition_test` PARTITION `sports`='Badminton' FOR COLUMN `city` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoBoagcnkoQHDmv3pA/6E55gChOm3qP//////AbzE6Yj7/////wH/jeLHAYC7qMgEgO/pvP//////AYDl7q35/////wHAnoCTBsDeqIT+/////wGArM7x//////8BwdCEL//10sL+/////wGB/Yxrv/G7V8HV1cT7/////wGA/62HA4C7sfsBwJT/rPr/////AYCFgrIBgbbW6P//////Ab707dv//////wGEkeKsBYLSvzy6mZuX+v////8B @@ -1004,3 +1004,121 @@ POSTHOOK: Input: database:db_bdpbase POSTHOOK: Output: database:db_bdpbase POSTHOOK: Output: db_bdpbase@default_partition_test POSTHOOK: Output: db_bdpbase@emp_sports +PREHOOK: query: create table test_partitions ( + a int, + b string +) partitioned by ( + c string, + d date, + e int +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_partitions +POSTHOOK: query: create table test_partitions ( + a int, + b string +) partitioned by ( + c string, + d date, + e int +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_partitions +PREHOOK: query: insert into test_partitions partition (c="str1", d="1970-01-01", e=1) values ('a_1', 'b_1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@test_partitions@c=str1/d=1970-01-01/e=1 +POSTHOOK: query: insert into test_partitions partition (c="str1", d="1970-01-01", e=1) values ('a_1', 'b_1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@test_partitions@c=str1/d=1970-01-01/e=1 +POSTHOOK: Lineage: test_partitions PARTITION(c=str1,d=1970-01-01,e=1).a SCRIPT [] +POSTHOOK: Lineage: test_partitions PARTITION(c=str1,d=1970-01-01,e=1).b SCRIPT [] +PREHOOK: query: insert into test_partitions partition (c="str2", d="1970-02-02", e=2) values ('a_2', 'b_2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@test_partitions@c=str2/d=1970-02-02/e=2 +POSTHOOK: query: insert into test_partitions partition (c="str2", d="1970-02-02", e=2) values ('a_2', 'b_2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@test_partitions@c=str2/d=1970-02-02/e=2 +POSTHOOK: Lineage: test_partitions PARTITION(c=str2,d=1970-02-02,e=2).a SCRIPT [] +POSTHOOK: Lineage: test_partitions PARTITION(c=str2,d=1970-02-02,e=2).b SCRIPT [] +PREHOOK: query: explain ddl select * from test_partitions +PREHOOK: type: QUERY +PREHOOK: Input: default@test_partitions +PREHOOK: Input: default@test_partitions@c=str1/d=1970-01-01/e=1 +PREHOOK: Input: default@test_partitions@c=str2/d=1970-02-02/e=2 +#### A masked pattern was here #### +POSTHOOK: query: explain ddl select * from test_partitions +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_partitions +POSTHOOK: Input: default@test_partitions@c=str1/d=1970-01-01/e=1 +POSTHOOK: Input: default@test_partitions@c=str2/d=1970-02-02/e=2 +#### A masked pattern was here #### + +CREATE TABLE `default`.`test_partitions`( + `a` int, + `b` string) +PARTITIONED BY ( + `c` string, + `d` date, + `e` int) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION +#### A masked pattern was here #### +TBLPROPERTIES ( + 'bucketing_version'='2', +#### A masked pattern was here #### +ALTER TABLE `default`.`test_partitions` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `default`.`test_partitions` ADD IF NOT EXISTS PARTITION (`c`='str1',`d`='1970-01-01',`e`=1); +ALTER TABLE `default`.`test_partitions` PARTITION (`c`='str1',`d`='1970-01-01',`e`=1) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); +ALTER TABLE `default`.`test_partitions` ADD IF NOT EXISTS PARTITION (`c`='str2',`d`='1970-02-02',`e`=2); +ALTER TABLE `default`.`test_partitions` PARTITION (`c`='str2',`d`='1970-02-02',`e`=2) UPDATE STATISTICS SET('numRows'='1','rawDataSize'='6' ); +ALTER TABLE `default`.`test_partitions` PARTITION (`c`='str1',`d`='1970-01-01',`e`=1) UPDATE STATISTICS FOR COLUMN `b` SET('avgColLen'='3.0','maxColLen'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test_partitions` PARTITION `c`='str1',`d`='1970-01-01',`e`=1 FOR COLUMN `b` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBw8Gy1wE= +ALTER TABLE `default`.`test_partitions` PARTITION (`c`='str1',`d`='1970-01-01',`e`=1) UPDATE STATISTICS FOR COLUMN `a` SET('lowValue'='0','highValue'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test_partitions` PARTITION `c`='str1',`d`='1970-01-01',`e`=1 FOR COLUMN `a` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA +ALTER TABLE `default`.`test_partitions` PARTITION (`c`='str2',`d`='1970-02-02',`e`=2) UPDATE STATISTICS FOR COLUMN `b` SET('avgColLen'='3.0','maxColLen'='3','numNulls'='0','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test_partitions` PARTITION `c`='str2',`d`='1970-02-02',`e`=2 FOR COLUMN `b` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEBgYOGywQ= +ALTER TABLE `default`.`test_partitions` PARTITION (`c`='str2',`d`='1970-02-02',`e`=2) UPDATE STATISTICS FOR COLUMN `a` SET('lowValue'='0','highValue'='0','numNulls'='1','numDVs'='1' ); +-- BIT VECTORS PRESENT FOR `default`.`test_partitions` PARTITION `c`='str2',`d`='1970-02-02',`e`=2 FOR COLUMN `a` BUT THEY ARE NOT SUPPORTED YET. THE BASE64 VALUE FOR THE BITVECTOR IS SExMoAEA + + + +EXPLAIN select * from test_partitions; + + +EXPLAIN CBO select * from test_partitions; + + +EXPLAIN VECTORIZED select * from test_partitions; +CBO PLAN:HiveProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) + HiveTableScan(table=[[default, test_partitions]], table:alias=[test_partitions]) + +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: test_partitions + Select Operator + expressions: a (type: int), b (type: string), c (type: string), d (type: date), e (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + ListSink + diff --git a/ql/src/test/results/clientpositive/llap/quotedid_basic.q.out b/ql/src/test/results/clientpositive/llap/quotedid_basic.q.out index e21ee969d1ee..9377a1a532a9 100644 --- a/ql/src/test/results/clientpositive/llap/quotedid_basic.q.out +++ b/ql/src/test/results/clientpositive/llap/quotedid_basic.q.out @@ -277,8 +277,8 @@ PREHOOK: Input: default@"%&'()*+,-/:;<=>?[]_|{}$^!~#@` POSTHOOK: query: show create table ` "%&'()*+,-/:;<=>?[]_|{}$^!~#@``` POSTHOOK: type: SHOW_CREATETABLE POSTHOOK: Input: default@"%&'()*+,-/:;<=>?[]_|{}$^!~#@` -CREATE TABLE `"%&'()*+,-/:;<=>?[]_|{}$^!~#@``( - ` "%&'()*+,-/;<=>?[]_|{}$^!~#@`` string) +CREATE TABLE `"%&'()*+,-/:;<=>?[]_|{}$^!~#@```( + ` "%&'()*+,-/;<=>?[]_|{}$^!~#@``` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT diff --git a/ql/src/test/results/clientpositive/llap/quotedid_basic_standard.q.out b/ql/src/test/results/clientpositive/llap/quotedid_basic_standard.q.out index b43ecbbb1611..d0e7395259dd 100644 --- a/ql/src/test/results/clientpositive/llap/quotedid_basic_standard.q.out +++ b/ql/src/test/results/clientpositive/llap/quotedid_basic_standard.q.out @@ -294,8 +294,8 @@ PREHOOK: Input: default@"%&'()*+,-/:;<=>?[]_|{}$^!~#@` POSTHOOK: query: show create table " ""%&'()*+,-/:;<=>?[]_|{}$^!~#@`" POSTHOOK: type: SHOW_CREATETABLE POSTHOOK: Input: default@"%&'()*+,-/:;<=>?[]_|{}$^!~#@` -CREATE TABLE `"%&'()*+,-/:;<=>?[]_|{}$^!~#@``( - ` "%&'()*+,-/;<=>?[]_|{}$^!~#@`` string) +CREATE TABLE `"%&'()*+,-/:;<=>?[]_|{}$^!~#@```( + ` "%&'()*+,-/;<=>?[]_|{}$^!~#@``` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT @@ -346,8 +346,8 @@ PREHOOK: Input: default@"%&'()*+,-/:;<=>?[]_|{}$^!~#@` POSTHOOK: query: show create table ` "%&'()*+,-/:;<=>?[]_|{}$^!~#@``` POSTHOOK: type: SHOW_CREATETABLE POSTHOOK: Input: default@"%&'()*+,-/:;<=>?[]_|{}$^!~#@` -CREATE TABLE `"%&'()*+,-/:;<=>?[]_|{}$^!~#@``( - ` "%&'()*+,-/;<=>?[]_|{}$^!~#@`` string) +CREATE TABLE `"%&'()*+,-/:;<=>?[]_|{}$^!~#@```( + ` "%&'()*+,-/;<=>?[]_|{}$^!~#@``` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT diff --git a/ql/src/test/results/clientpositive/llap/show_create_table.q.out b/ql/src/test/results/clientpositive/llap/show_create_table.q.out index 304e5ad7ffb7..3d92e504989c 100644 --- a/ql/src/test/results/clientpositive/llap/show_create_table.q.out +++ b/ql/src/test/results/clientpositive/llap/show_create_table.q.out @@ -1,11 +1,11 @@ PREHOOK: query: CREATE TABLE TEST( col1 varchar(100) NOT NULL COMMENT "comment for column 1", col2 timestamp DEFAULT CURRENT_TIMESTAMP() COMMENT "comment for column 2", - col3 decimal CHECK (col3 + col4 > 1) enable novalidate rely, + `col 3` decimal CHECK (`col 3` + col4 > 1) enable novalidate rely, col4 decimal NOT NULL, col5 varchar(100), primary key(col1, col2) disable novalidate rely, - constraint c3_c4_check CHECK((col3 + col4)/(col3 - col4) > 3) enable novalidate norely, + constraint c3_c4_check CHECK((`col 3` + col4)/(`col 3` - col4) > 3) enable novalidate norely, constraint c4_unique UNIQUE(col4) disable novalidate rely) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' @@ -19,11 +19,11 @@ PREHOOK: Output: default@TEST POSTHOOK: query: CREATE TABLE TEST( col1 varchar(100) NOT NULL COMMENT "comment for column 1", col2 timestamp DEFAULT CURRENT_TIMESTAMP() COMMENT "comment for column 2", - col3 decimal CHECK (col3 + col4 > 1) enable novalidate rely, + `col 3` decimal CHECK (`col 3` + col4 > 1) enable novalidate rely, col4 decimal NOT NULL, col5 varchar(100), primary key(col1, col2) disable novalidate rely, - constraint c3_c4_check CHECK((col3 + col4)/(col3 - col4) > 3) enable novalidate norely, + constraint c3_c4_check CHECK((`col 3` + col4)/(`col 3` - col4) > 3) enable novalidate norely, constraint c4_unique UNIQUE(col4) disable novalidate rely) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' @@ -97,7 +97,7 @@ POSTHOOK: Input: default@test CREATE TABLE `test`( `col1` varchar(100) COMMENT 'comment for column 1', `col2` timestamp COMMENT 'comment for column 2', - `col3` decimal(10,0), + `col 3` decimal(10,0), `col4` decimal(10,0), `col5` varchar(100)) ROW FORMAT SERDE @@ -111,13 +111,13 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.test ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (col1,col2) DISABLE NOVALIDATE RELY; -ALTER TABLE default.test ADD CONSTRAINT c4_unique UNIQUE (col4) DISABLE NOVALIDATE RELY; -ALTER TABLE test.test CHANGE COLUMN col2 col2 timestamp CONSTRAINT DEFAULT CURRENT_TIMESTAMP() ENABLE NOVALIDATE RELY; -ALTER TABLE default.test ADD CONSTRAINT c3_c4_check CHECK ((`col3` + `col4`)/(`col3` - `col4`) > 3) ENABLE NOVALIDATE NORELY; -ALTER TABLE default.test ADD CONSTRAINT #### A masked pattern was here #### CHECK (`col3` + `col4` > 1) ENABLE NOVALIDATE RELY; -ALTER TABLE default.test CHANGE COLUMN col1 col1 varchar(100) CONSTRAINT #### A masked pattern was here #### NOT NULL ENABLE NOVALIDATE RELY; -ALTER TABLE default.test CHANGE COLUMN col4 col4 decimal(10,0) CONSTRAINT #### A masked pattern was here #### NOT NULL ENABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`col1`,`col2`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test` ADD CONSTRAINT `c4_unique` UNIQUE (`col4`) DISABLE NOVALIDATE RELY; +ALTER TABLE `test`.`test` CHANGE COLUMN `col2` `col2` timestamp CONSTRAINT DEFAULT CURRENT_TIMESTAMP() ENABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test` ADD CONSTRAINT `c3_c4_check` CHECK ((`col 3` + `col4`)/(`col 3` - `col4`) > 3) ENABLE NOVALIDATE NORELY; +ALTER TABLE `default`.`test` ADD CONSTRAINT `#### A masked pattern was here ####` CHECK (`col 3` + `col4` > 1) ENABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test` CHANGE COLUMN `col1` `col1` varchar(100) CONSTRAINT `#### A masked pattern was here ####` NOT NULL ENABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test` CHANGE COLUMN `col4` `col4` decimal(10,0) CONSTRAINT `#### A masked pattern was here ####` NOT NULL ENABLE NOVALIDATE RELY; PREHOOK: query: SHOW CREATE TABLE TEST2 PREHOOK: type: SHOW_CREATETABLE PREHOOK: Input: default@test2 @@ -137,7 +137,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.test2 ADD CONSTRAINT #### A masked pattern was here #### PRIMARY KEY (col) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test2` ADD CONSTRAINT `#### A masked pattern was here ####` PRIMARY KEY (`col`) DISABLE NOVALIDATE RELY; PREHOOK: query: SHOW CREATE TABLE TEST3 PREHOOK: type: SHOW_CREATETABLE PREHOOK: Input: default@test3 @@ -159,8 +159,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE default.test3 ADD CONSTRAINT #### A masked pattern was here #### FOREIGN KEY (col1) REFERENCES default.test(col1) DISABLE NOVALIDATE RELY; -ALTER TABLE default.test3 ADD CONSTRAINT #### A masked pattern was here #### FOREIGN KEY (col2) REFERENCES default.test(col2) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test3` ADD CONSTRAINT `#### A masked pattern was here ####` FOREIGN KEY (`col1`) REFERENCES `default`.`test`(`col1`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test3` ADD CONSTRAINT `#### A masked pattern was here ####` FOREIGN KEY (`col2`) REFERENCES `default`.`test`(`col2`) DISABLE NOVALIDATE RELY; PREHOOK: query: CREATE TABLE TEST_RESERVED ( `member_nr` varchar(8), `plan_nr` varchar(11), @@ -198,10 +198,10 @@ CREATE TABLE `test_reserved`( `plan_nr` varchar(11), `timestamp` timestamp, `shared_ind` varchar(1)) -CLUSTERED BY ( - `member_nr`, - `plan_nr`, - `timestamp`) +CLUSTERED BY ( + `member_nr`, + `plan_nr`, + `timestamp`) INTO 4 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' diff --git a/ql/src/test/results/clientpositive/llap/show_create_table_alter.q.out b/ql/src/test/results/clientpositive/llap/show_create_table_alter.q.out index 5c3d756eea57..97b31882aeda 100644 --- a/ql/src/test/results/clientpositive/llap/show_create_table_alter.q.out +++ b/ql/src/test/results/clientpositive/llap/show_create_table_alter.q.out @@ -17,10 +17,10 @@ POSTHOOK: Input: default@tmp_showcrt1_n1 CREATE EXTERNAL TABLE `tmp_showcrt1_n1`( `key` smallint, `value` float) -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - value DESC) + `value` DESC) INTO 5 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -51,10 +51,10 @@ CREATE TABLE `tmp_showcrt1_n1`( `key` smallint, `value` float) COMMENT 'temporary table' -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - value DESC) + `value` DESC) INTO 5 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -85,10 +85,10 @@ CREATE EXTERNAL TABLE `tmp_showcrt1_n1`( `key` smallint, `value` float) COMMENT 'changed comment' -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - value DESC) + `value` DESC) INTO 5 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -119,10 +119,10 @@ CREATE EXTERNAL TABLE `tmp_showcrt1_n1`( `key` smallint, `value` float) COMMENT 'changed comment' -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - value DESC) + `value` DESC) INTO 5 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -153,10 +153,10 @@ CREATE EXTERNAL TABLE `tmp_showcrt1_n1`( `key` smallint, `value` float) COMMENT 'changed comment' -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - value DESC) + `value` DESC) INTO 5 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' diff --git a/ql/src/test/results/clientpositive/llap/show_create_table_db_table.q.out b/ql/src/test/results/clientpositive/llap/show_create_table_db_table.q.out index 18d44b70ffac..3e0a13fe92d1 100644 --- a/ql/src/test/results/clientpositive/llap/show_create_table_db_table.q.out +++ b/ql/src/test/results/clientpositive/llap/show_create_table_db_table.q.out @@ -77,7 +77,7 @@ POSTHOOK: Input: tmp_feng@tmp_showcrt2 CREATE TABLE `tmp_feng`.`tmp_showcrt2`( `key` string, `value` int) -SKEWED BY (key) +SKEWED BY (`key`) ON (('1'),('2')) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -99,7 +99,7 @@ POSTHOOK: Input: tmp_feng@tmp_showcrt3 CREATE TABLE `tmp_feng`.`tmp_showcrt3`( `key` string, `value` int) -SKEWED BY (key) +SKEWED BY (`key`) ON (('1'),('2')) STORED AS DIRECTORIES ROW FORMAT SERDE diff --git a/ql/src/test/results/clientpositive/llap/table_storage.q.out b/ql/src/test/results/clientpositive/llap/table_storage.q.out index b370beac23a2..bd47ee822df6 100644 --- a/ql/src/test/results/clientpositive/llap/table_storage.q.out +++ b/ql/src/test/results/clientpositive/llap/table_storage.q.out @@ -62,10 +62,10 @@ POSTHOOK: Input: default@t CREATE TABLE `t`( `key` string, `val` string) -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - key ASC) + `key` ASC) INTO 2 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -109,10 +109,10 @@ POSTHOOK: Input: default@t CREATE TABLE `t`( `key` string, `val` string) -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) SORTED BY ( - key ASC) + `key` ASC) INTO 3 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -158,8 +158,8 @@ POSTHOOK: Input: default@t CREATE TABLE `t`( `key` string, `val` string) -CLUSTERED BY ( - `key`) +CLUSTERED BY ( + `key`) INTO 3 BUCKETS ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' @@ -252,7 +252,7 @@ POSTHOOK: Input: default@t CREATE TABLE `t`( `key` string, `val` string) -SKEWED BY (key) +SKEWED BY (`key`) ON (('a'),('b')) STORED AS DIRECTORIES ROW FORMAT SERDE @@ -301,7 +301,7 @@ POSTHOOK: Input: default@t CREATE TABLE `t`( `key` string, `val` string) -SKEWED BY (key) +SKEWED BY (`key`) ON (('a'),('b')) STORED AS DIRECTORIES ROW FORMAT SERDE diff --git a/ql/src/test/results/clientpositive/llap/views_explain_ddl.q.out b/ql/src/test/results/clientpositive/llap/views_explain_ddl.q.out index 3bc773f62978..f93a1a2f4e5c 100644 --- a/ql/src/test/results/clientpositive/llap/views_explain_ddl.q.out +++ b/ql/src/test/results/clientpositive/llap/views_explain_ddl.q.out @@ -152,7 +152,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@v1_n17 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table1_n19`( `key` string, `value` string) @@ -167,7 +167,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v1_n17` AS SELECT `table1_n19`.`key`, `table1_n19`.`value` FROM `db1`.`table1_n19`; @@ -213,7 +213,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@v2_n10 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table1_n19`( `key` string, `value` string) @@ -228,7 +228,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v2_n10` AS SELECT `t1`.`key`, `t1`.`value` FROM `db1`.`table1_n19` `t1`; @@ -273,7 +273,7 @@ POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@table2_n13 POSTHOOK: Input: db1@v3_n3 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table2_n13`( `key` string, `value` string) @@ -302,8 +302,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table2_n13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table2_n13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v3_n3` AS SELECT `t1`.`key`, `t1`.`value`, `t2`.`key` `k` FROM `db1`.`table1_n19` `t1` JOIN `db1`.`table2_n13` `t2` ON `t1`.`key` = `t2`.`key`; @@ -439,7 +439,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@v4_n3 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table1_n19`( `key` string, `value` string) @@ -454,7 +454,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v4_n3` AS SELECT `table1_n19`.`key`, `table1_n19`.`value` FROM `db1`.`table1_n19`; @@ -497,7 +497,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@v5_n1 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table1_n19`( `key` string, `value` string) @@ -512,7 +512,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v5_n1` AS SELECT `t1`.`key`, `t1`.`value` FROM `db1`.`table1_n19` `t1`; @@ -557,7 +557,7 @@ POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@table2_n13 POSTHOOK: Input: db1@v6 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table2_n13`( `key` string, `value` string) @@ -586,8 +586,8 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table2_n13 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table2_n13` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v6` AS SELECT `t1`.`key`, `t1`.`value`, `t2`.`key` `k` FROM `db1`.`table1_n19` `t1` JOIN `db1`.`table2_n13` `t2` ON `t1`.`key` = `t2`.`key`; @@ -723,7 +723,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@v7 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table1_n19`( `key` string, `value` string) @@ -738,7 +738,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v7` AS SELECT `table1_n19`.`key` from `db1`.`table1_n19`; @@ -781,7 +781,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: db1@table1_n19 POSTHOOK: Input: db1@v8 #### A masked pattern was here #### -CREATE DATABASE IF NOT EXISTS db1; +CREATE DATABASE IF NOT EXISTS `db1`; CREATE TABLE `db1`.`table1_n19`( `key` string, `value` string) @@ -796,7 +796,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE db1.table1_n19 UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); +ALTER TABLE `db1`.`table1_n19` UPDATE STATISTICS SET('numRows'='0','rawDataSize'='0' ); CREATE VIEW `db1`.`v8` AS SELECT `table1_n19`.`key` from `db1`.`table1_n19`; From a4aef8173269d213eb13666baf1751bf110579c8 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Wed, 8 Nov 2023 17:15:35 +0800 Subject: [PATCH 043/179] HIVE-27593: Iceberg: Keep Iceberg properties in sync with HMS (Butao Zhang, reviewed by Denys Kuzmenko) Closes #4573 --- .../iceberg/mr/hive/HiveIcebergMetaHook.java | 36 +++++++++++--- .../TestHiveIcebergStorageHandlerNoScan.java | 47 +++++++++++++++++++ 2 files changed, 76 insertions(+), 7 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index f174bd7b2fe6..540a3b8638c0 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -128,6 +128,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.iceberg.TableProperties.DELETE_MODE; +import static org.apache.iceberg.TableProperties.MERGE_MODE; +import static org.apache.iceberg.TableProperties.UPDATE_MODE; + public class HiveIcebergMetaHook implements HiveMetaHook { private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergMetaHook.class); public static final Map COMMON_HMS_PROPERTIES = ImmutableMap.of( @@ -428,7 +432,7 @@ private void doPreAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable assertNotMigratedTable(hmsTable.getParameters(), "CHANGE COLUMN"); handleChangeColumn(hmsTable); } else { - setDeleteModeOnTableProperties(icebergTable, hmsTable.getParameters()); + setDeleteModeOnTableProperties(icebergTable, hmsTable.getParameters(), context); assertNotCrossTableMetadataLocationChange(hmsTable.getParameters(), context); } @@ -741,7 +745,7 @@ private void setCommonHmsTablePropertiesForIceberg(org.apache.hadoop.hive.metast // Remove creation related properties PARAMETERS_TO_REMOVE.forEach(hmsParams::remove); - setDeleteModeOnTableProperties(null, hmsParams); + setDeleteModeOnTableProperties(null, hmsParams, null); } /** @@ -981,14 +985,32 @@ private boolean isOrcOnlyFiles(org.apache.hadoop.hive.metastore.api.Table hmsTab } // TODO: remove this if copy-on-write mode gets implemented in Hive - private static void setDeleteModeOnTableProperties(Table icebergTable, Map newProps) { + private void setDeleteModeOnTableProperties(Table icebergTbl, Map newProps, + EnvironmentContext context) { // Hive only supports merge-on-read delete mode, it will actually throw an error if DML operations are attempted on // tables that don't have this (the default is copy-on-write). We set this at table creation and v1->v2 conversion. - if ((icebergTable == null || ((BaseTable) icebergTable).operations().current().formatVersion() == 1) && + if ((icebergTbl == null || ((BaseTable) icebergTbl).operations().current().formatVersion() == 1) && IcebergTableUtil.isV2Table(newProps)) { - newProps.put(TableProperties.DELETE_MODE, HiveIcebergStorageHandler.MERGE_ON_READ); - newProps.put(TableProperties.UPDATE_MODE, HiveIcebergStorageHandler.MERGE_ON_READ); - newProps.put(TableProperties.MERGE_MODE, HiveIcebergStorageHandler.MERGE_ON_READ); + List writeModeList = Arrays.asList(DELETE_MODE, UPDATE_MODE, MERGE_MODE); + writeModeList.stream() + .filter(writeMode -> catalogProperties.get(writeMode) == null) + .forEach(writeMode -> { + catalogProperties.put(writeMode, HiveIcebergStorageHandler.MERGE_ON_READ); + newProps.put(writeMode, HiveIcebergStorageHandler.MERGE_ON_READ); + }); + + if (context != null) { + Splitter splitter = Splitter.on(PROPERTIES_SEPARATOR); + Map contextProperties = context.getProperties(); + if (contextProperties.containsKey(SET_PROPERTIES)) { + String propValue = context.getProperties().get(SET_PROPERTIES); + String writeModeStr = writeModeList.stream().filter(writeMode -> + !splitter.splitToList(propValue).contains(writeMode)).collect(Collectors.joining("'")); + if (!writeModeStr.isEmpty()) { + contextProperties.put(SET_PROPERTIES, propValue + "'" + writeModeStr); + } + } + } } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index dc17e472b9c3..0baf210340f8 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -1928,6 +1928,53 @@ public void checkIcebergTableLocation() throws TException, InterruptedException, Assert.assertTrue(hmsTblLocation.getFileSystem(shell.getHiveConf()).exists(hmsTblLocation)); } + @Test + public void testSnycProperties() throws TException, InterruptedException { + Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTables.TestTableType.HIVE_CATALOG); + + // Test create v2 iceberg table and check iceberg properties & hms properties + TableIdentifier identifier = TableIdentifier.of("default", "customers_v2"); + shell.executeStatement("CREATE TABLE customers_v2 (id int, name string) Stored by Iceberg stored as ORC " + + "TBLPROPERTIES ('format-version'='2','write.delete.mode'='copy-on-write')"); + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + org.apache.hadoop.hive.metastore.api.Table hmsTable = shell.metastore().getTable("default", "customers_v2"); + Map icePros = icebergTable.properties(); + Map hmsProps = hmsTable.getParameters(); + Assert.assertEquals(icePros.get(TableProperties.DELETE_MODE), HiveIcebergStorageHandler.COPY_ON_WRITE); + Assert.assertEquals(icePros.get(TableProperties.UPDATE_MODE), HiveIcebergStorageHandler.MERGE_ON_READ); + Assert.assertEquals(icePros.get(TableProperties.MERGE_MODE), HiveIcebergStorageHandler.MERGE_ON_READ); + Assert.assertEquals(icePros.get(TableProperties.DELETE_MODE), hmsProps.get(TableProperties.DELETE_MODE)); + Assert.assertEquals(icePros.get(TableProperties.UPDATE_MODE), hmsProps.get(TableProperties.UPDATE_MODE)); + Assert.assertEquals(icePros.get(TableProperties.MERGE_MODE), hmsProps.get(TableProperties.MERGE_MODE)); + + // Test create v1 iceberg table and check its properties before and after it upgrades to v2 + identifier = TableIdentifier.of("default", "customers_v1"); + shell.executeStatement("CREATE TABLE customers_v1 (id int, name string) Stored by Iceberg stored as ORC"); + icebergTable = testTables.loadTable(identifier); + hmsTable = shell.metastore().getTable("default", "customers_v1"); + icePros = icebergTable.properties(); + hmsProps = hmsTable.getParameters(); + // check v1 iceberg table properties + Assert.assertEquals(icePros.get(TableProperties.DELETE_MODE), null); + Assert.assertEquals(icePros.get(TableProperties.UPDATE_MODE), null); + Assert.assertEquals(icePros.get(TableProperties.MERGE_MODE), null); + Assert.assertEquals(icePros.get(TableProperties.DELETE_MODE), hmsProps.get(TableProperties.DELETE_MODE)); + Assert.assertEquals(icePros.get(TableProperties.UPDATE_MODE), hmsProps.get(TableProperties.UPDATE_MODE)); + Assert.assertEquals(icePros.get(TableProperties.MERGE_MODE), hmsProps.get(TableProperties.MERGE_MODE)); + // check table properties after upgrading to v2 + shell.executeStatement("ALTER TABLE customers_v1 SET TBLPROPERTIES ('format-version'='2')"); + icebergTable = testTables.loadTable(identifier); + hmsTable = shell.metastore().getTable("default", "customers_v1"); + icePros = icebergTable.properties(); + hmsProps = hmsTable.getParameters(); + Assert.assertEquals(icePros.get(TableProperties.DELETE_MODE), HiveIcebergStorageHandler.MERGE_ON_READ); + Assert.assertEquals(icePros.get(TableProperties.UPDATE_MODE), HiveIcebergStorageHandler.MERGE_ON_READ); + Assert.assertEquals(icePros.get(TableProperties.MERGE_MODE), HiveIcebergStorageHandler.MERGE_ON_READ); + Assert.assertEquals(icePros.get(TableProperties.DELETE_MODE), hmsProps.get(TableProperties.DELETE_MODE)); + Assert.assertEquals(icePros.get(TableProperties.UPDATE_MODE), hmsProps.get(TableProperties.UPDATE_MODE)); + Assert.assertEquals(icePros.get(TableProperties.MERGE_MODE), hmsProps.get(TableProperties.MERGE_MODE)); + } + private String getCurrentSnapshotForHiveCatalogTable(org.apache.iceberg.Table icebergTable) { return ((BaseMetastoreTableOperations) ((BaseTable) icebergTable).operations()).currentMetadataLocation(); } From 2eef89b5b6f350c2c9a72499c3253760df20a328 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 9 Nov 2023 00:10:04 +0800 Subject: [PATCH 044/179] HIVE-27817: Disable ssl hostname verification for 127.0.0.1 (#4823) --- .../org/apache/hadoop/hive/common/auth/HiveAuthUtils.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java b/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java index a37d527c1ea5..68163afa42ba 100644 --- a/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java +++ b/common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java @@ -142,7 +142,11 @@ private static TSocket getSSLSocketWithHttps(TSocket tSSLSocket, int maxMessageS throws TTransportException { SSLSocket sslSocket = (SSLSocket) tSSLSocket.getSocket(); SSLParameters sslParams = sslSocket.getSSLParameters(); - sslParams.setEndpointIdentificationAlgorithm("HTTPS"); + if (sslSocket.getLocalAddress().getHostAddress().equals("127.0.0.1")) { + sslParams.setEndpointIdentificationAlgorithm(null); + } else { + sslParams.setEndpointIdentificationAlgorithm("HTTPS"); + } sslSocket.setSSLParameters(sslParams); TSocket tSocket = new TSocket(sslSocket); return configureThriftMaxMessageSize(tSocket, maxMessageSize); From 30b3a233bec2bbdcb984ef7ff152bfda24c55940 Mon Sep 17 00:00:00 2001 From: Vinod Anandan Date: Fri, 10 Nov 2023 06:37:08 +0200 Subject: [PATCH 045/179] HIVE-27864: Update plugin for SBOM generation to 2.7.10 (#4833). (Vinod Anandan reviewed Ayush Saxena, Attila Turoczy) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c163ce1d3a29..a2c4078318c0 100644 --- a/pom.xml +++ b/pom.xml @@ -99,7 +99,7 @@ 2.16.0 3.5.0 3.0.0-M4 - 2.7.9 + 2.7.10 2.1.0 1.10.1 From 102b89e6816a7756cb50f2cc3889351dc0e270b2 Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Sat, 11 Nov 2023 03:01:32 +0800 Subject: [PATCH 046/179] HIVE-27762: Don't fall back to jdo query in ObjectStore if direct sql throws unrecoverable exception (Wechar Yu, reviewed by Sai Hemanth Gantasala, Denys Kuzmenko) Closes #4767 --- .../hive/metastore/DatabaseProduct.java | 12 ++++++ .../hadoop/hive/metastore/ObjectStore.java | 9 ++--- .../hive/metastore/RetryingHMSHandler.java | 11 +---- .../hive/metastore/TestObjectStore.java | 40 +++++++++++++++++++ 4 files changed, 56 insertions(+), 16 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java index 3f3d361b9a0a..afee07974204 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.metastore; import java.sql.SQLException; +import java.sql.SQLIntegrityConstraintViolationException; import java.sql.SQLTransactionRollbackException; import java.sql.Timestamp; import java.util.ArrayList; @@ -26,7 +27,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Stream; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -47,6 +50,10 @@ * */ public class DatabaseProduct implements Configurable { static final private Logger LOG = LoggerFactory.getLogger(DatabaseProduct.class.getName()); + private static final Class[] unrecoverableSqlExceptions = new Class[]{ + // TODO: collect more unrecoverable SQLExceptions + SQLIntegrityConstraintViolationException.class + }; public enum DbType {DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER, CUSTOM, UNDEFINED}; public DbType dbType; @@ -154,6 +161,11 @@ private static DbType getDbType(String productName) { return dbt; } + public static boolean isRecoverableException(Throwable t) { + return Stream.of(unrecoverableSqlExceptions) + .allMatch(ex -> ExceptionUtils.indexOfType(t, ex) < 0); + } + public final boolean isDERBY() { return dbType == DbType.DERBY; } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index fa132629934b..98f7f6f85be2 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -4431,6 +4431,9 @@ private void start(boolean initTable) throws MetaException, NoSuchObjectExceptio } private void handleDirectSqlError(Exception ex, String savePoint) throws MetaException, NoSuchObjectException { + if (!allowJdo || !DatabaseProduct.isRecoverableException(ex)) { + throw ExceptionHandler.newMetaException(ex); + } String message = null; try { message = generateShorterMessage(ex); @@ -4439,12 +4442,6 @@ private void handleDirectSqlError(Exception ex, String savePoint) throws MetaExc } LOG.warn(message); // Don't log the exception, people just get confused. LOG.debug("Full DirectSQL callstack for debugging (not an error)", ex); - if (!allowJdo) { - if (ex instanceof MetaException) { - throw (MetaException)ex; - } - throw new MetaException(ex.getMessage()); - } if (!isInTxn) { JDOException rollbackEx = null; try { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java index ce7656fdb0c9..0d52107ad095 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java @@ -21,10 +21,7 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.UndeclaredThrowableException; -import java.sql.SQLException; -import java.sql.SQLIntegrityConstraintViolationException; import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; import javax.jdo.JDOException; @@ -44,10 +41,6 @@ @InterfaceStability.Evolving public class RetryingHMSHandler extends AbstractHMSHandlerProxy { private static final Logger LOG = LoggerFactory.getLogger(RetryingHMSHandler.class); - private static final Class[] unrecoverableSqlExceptions = new Class[]{ - // TODO: collect more unrecoverable SQLExceptions - SQLIntegrityConstraintViolationException.class - }; private final long retryInterval; private final int retryLimit; @@ -185,8 +178,6 @@ private boolean isRecoverableException(Throwable t) { if (!(t instanceof JDOException || t instanceof NucleusException)) { return false; } - - return Stream.of(unrecoverableSqlExceptions) - .allMatch(ex -> ExceptionUtils.indexOfType(t, ex) < 0); + return DatabaseProduct.isRecoverableException(t); } } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java index 5746a9526499..940b18d1db46 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java @@ -97,6 +97,7 @@ import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.SQLIntegrityConstraintViolationException; import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; @@ -116,6 +117,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import static java.util.concurrent.Executors.newFixedThreadPool; @@ -1631,6 +1633,44 @@ protected Object getJdoResult(ObjectStore.GetHelper ctx) throws MetaExce Assert.assertEquals(0, objectStore.getPartitionCount()); } + @Test + public void testNoJdoForUnrecoverableException() throws Exception { + objectStore.openTransaction(); + AtomicBoolean runDirectSql = new AtomicBoolean(false); + AtomicBoolean runJdo = new AtomicBoolean(false); + try { + objectStore.new GetHelper(DEFAULT_CATALOG_NAME, DB1, TABLE1, true, true) { + @Override + protected String describeResult() { + return "test not run jdo for unrecoverable exception"; + } + + @Override + protected Object getSqlResult(ObjectStore.GetHelper ctx) throws MetaException { + runDirectSql.set(true); + SQLIntegrityConstraintViolationException ex = new SQLIntegrityConstraintViolationException("Unrecoverable ex"); + MetaException me = new MetaException("Throwing unrecoverable exception to test not run jdo."); + me.initCause(ex); + throw me; + } + + @Override + protected Object getJdoResult(ObjectStore.GetHelper ctx) throws MetaException, NoSuchObjectException { + runJdo.set(true); + SQLIntegrityConstraintViolationException ex = new SQLIntegrityConstraintViolationException("Unrecoverable ex"); + MetaException me = new MetaException("Throwing unrecoverable exception to test not run jdo."); + me.initCause(ex); + throw me; + } + }.run(false); + } catch (MetaException ex) { + // expected + } + objectStore.commitTransaction(); + Assert.assertEquals(true, runDirectSql.get()); + Assert.assertEquals(false, runJdo.get()); + } + /** * Helper method to check whether the Java system properties were set correctly in {@link ObjectStore#configureSSL(Configuration)} * @param useSSL whether or not SSL is enabled From 3be30dc0177cb56d74c5c08e9001eb7a6fb7c5c5 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sat, 11 Nov 2023 03:24:40 +0800 Subject: [PATCH 047/179] HIVE-27819: Iceberg: Upgrade iceberg version to 1.4.2 (Butao Zhang, reviewed by Ayush Saxena, Attila Turoczy, Krisztian Kasa, Denys Kuzmenko) Closes #4824 --- .../apache/hadoop/hive/conf/Constants.java | 2 +- iceberg/iceberg-catalog/pom.xml | 5 + .../iceberg/hive/HiveTableOperations.java | 1 + .../apache/iceberg/hive/MetastoreLock.java | 2 +- .../hive/HiveCreateReplaceTableTest.java | 152 ++++---- .../iceberg/hive/HiveMetastoreTest.java | 8 +- .../iceberg/hive/HiveTableBaseTest.java | 8 +- .../apache/iceberg/hive/HiveTableTest.java | 221 ++++++----- .../iceberg/hive/TestCachedClientPool.java | 72 ++-- .../apache/iceberg/hive/TestHiveCatalog.java | 351 ++++++++++-------- .../iceberg/hive/TestHiveClientPool.java | 37 +- .../iceberg/hive/TestHiveCommitLocks.java | 86 +++-- .../apache/iceberg/hive/TestHiveCommits.java | 142 ++++--- .../iceberg/hive/TestHiveMetastore.java | 4 +- .../iceberg/hive/TestHiveSchemaUtil.java | 28 +- .../hive/TestHiveTableConcurrency.java | 11 +- .../iceberg/hive/TestLoadHiveCatalog.java | 19 +- .../org/apache/iceberg/mr/TestCatalogs.java | 6 +- .../TestHiveIcebergStorageHandlerNoScan.java | 23 +- .../positive/dynamic_partition_writes.q | 2 +- ...rg_insert_overwrite_partition_transforms.q | 2 +- ...ry_iceberg_metadata_of_partitioned_table.q | 14 +- .../queries/positive/show_partitions_test.q | 8 +- .../positive/vectorized_iceberg_merge_mixed.q | 2 +- .../positive/vectorized_iceberg_read_mixed.q | 6 +- .../positive/vectorized_iceberg_read_orc.q | 8 +- .../vectorized_iceberg_read_parquet.q | 8 +- .../negative/merge_split_update_off.q.out | 2 +- ...e_partition_spec_with_part_evolution.q.out | 2 +- .../negative/update_split_update_off.q.out | 2 +- .../alter_multi_part_table_to_iceberg.q.out | 6 + .../alter_part_table_to_iceberg.q.out | 6 + .../positive/alter_table_to_iceberg.q.out | 6 + .../src/test/results/positive/col_stats.q.out | 2 + .../positive/create_iceberg_table.q.out | 2 + ...e_iceberg_table_stored_as_fileformat.q.out | 10 + ...eate_iceberg_table_stored_by_iceberg.q.out | 2 + ...ored_by_iceberg_with_serdeproperties.q.out | 2 + .../ctas_iceberg_partitioned_orc.q.out | 1 + .../test/results/positive/ctlt_iceberg.q.out | 12 +- .../results/positive/delete_all_iceberg.q.out | 1 + .../describe_iceberg_metadata_tables.q.out | 9 + .../positive/describe_iceberg_table.q.out | 8 + .../positive/dynamic_partition_writes.q.out | 30 +- .../iceberg_insert_into_partition.q.out | 12 + ...erg_insert_into_partition_transforms.q.out | 10 + ...insert_into_partition_with_evolution.q.out | 2 + .../iceberg_insert_overwrite_partition.q.out | 12 + ...nsert_overwrite_partition_transforms.q.out | 16 +- .../results/positive/iceberg_v2_deletes.q.out | 3 + .../llap/vectorized_iceberg_read_mixed.q.out | 12 +- .../llap/vectorized_iceberg_read_orc.q.out | 16 +- .../vectorized_iceberg_read_parquet.q.out | 16 +- .../positive/mv_iceberg_partitioned_orc.q.out | 2 + .../mv_iceberg_partitioned_orc2.q.out | 2 + ...ceberg_metadata_of_partitioned_table.q.out | 74 ++-- ...berg_metadata_of_unpartitioned_table.q.out | Bin 39970 -> 39970 bytes .../src/test/results/positive/row_count.q.out | 2 + .../positive/show_create_iceberg_table.q.out | 12 +- .../show_iceberg_materialized_views.q.out | 4 + .../positive/show_partitions_test.q.out | 55 +-- .../truncate_force_iceberg_table.q.out | 4 + .../positive/truncate_iceberg_table.q.out | 10 + .../truncate_partitioned_iceberg_table.q.out | 4 + .../use_basic_stats_from_iceberg.q.out | 2 + .../vectorized_iceberg_merge_mixed.q.out | 2 +- .../vectorized_iceberg_read_mixed.q.out | 12 +- .../vectorized_iceberg_read_orc.q.out | 16 +- .../vectorized_iceberg_read_parquet.q.out | 16 +- iceberg/patched-iceberg-core/pom.xml | 7 + .../org/apache/iceberg/PartitionsTable.java | 345 +++++++++++++++++ iceberg/pom.xml | 16 +- 72 files changed, 1361 insertions(+), 654 deletions(-) create mode 100644 iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java diff --git a/common/src/java/org/apache/hadoop/hive/conf/Constants.java b/common/src/java/org/apache/hadoop/hive/conf/Constants.java index 919e40bec0fc..5d2550cbbd6a 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/Constants.java +++ b/common/src/java/org/apache/hadoop/hive/conf/Constants.java @@ -110,6 +110,6 @@ public class Constants { public static final String ICEBERG = "iceberg"; public static final String ICEBERG_PARTITION_TABLE_SCHEMA = "partition,spec_id,record_count,file_count," + "position_delete_record_count,position_delete_file_count,equality_delete_record_count," + - "equality_delete_file_count"; + "equality_delete_file_count,last_updated_at,total_data_file_size_in_bytes,last_updated_snapshot_id"; public static final String DELIMITED_JSON_SERDE = "org.apache.hadoop.hive.serde2.DelimitedJSONSerDe"; } diff --git a/iceberg/iceberg-catalog/pom.xml b/iceberg/iceberg-catalog/pom.xml index 3280eb0945e0..3587125bcc67 100644 --- a/iceberg/iceberg-catalog/pom.xml +++ b/iceberg/iceberg-catalog/pom.xml @@ -74,6 +74,11 @@ junit-jupiter-api test + + org.junit.jupiter + junit-jupiter-params + test + org.junit.vintage junit-vintage-engine diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index c5adae435fb8..7d49069b3fab 100644 --- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -234,6 +234,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { commitStatus = CommitStatus.SUCCESS; } catch (LockException le) { + commitStatus = CommitStatus.UNKNOWN; throw new CommitStateUnknownException( "Failed to heartbeat for hive lock while " + "committing changes. This can lead to a concurrent commit attempt be able to overwrite this commit. " + diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java index 454a3a5f5e1a..f25a35ab7830 100644 --- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java +++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/MetastoreLock.java @@ -308,7 +308,7 @@ private LockInfo createLock() throws LockException { try { // If we can not check for lock, or we do not find it, then rethrow the exception // Otherwise we are happy as the findLock sets the lockId and the state correctly - if (!HiveVersion.min(HiveVersion.HIVE_2)) { + if (HiveVersion.min(HiveVersion.HIVE_2)) { LockInfo lockFound = findLock(); if (lockFound != null) { lockInfo.lockId = lockFound.lockId; diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java index 4165346a673a..aa1790fd215a 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java @@ -19,7 +19,7 @@ package org.apache.iceberg.hive; -import java.io.IOException; +import java.nio.file.Path; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -27,23 +27,25 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import static org.apache.iceberg.PartitionSpec.builderFor; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class HiveCreateReplaceTableTest extends HiveMetastoreTest { @@ -57,24 +59,24 @@ public class HiveCreateReplaceTableTest extends HiveMetastoreTest { .identity("id") .build(); - @Rule - public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + private Path temp; private String tableLocation; - @Before - public void createTableLocation() throws IOException { - tableLocation = temp.newFolder("hive-").getPath(); + @BeforeEach + public void createTableLocation() { + tableLocation = temp.resolve("hive-").toString(); } - @After + @AfterEach public void cleanup() { catalog.dropTable(TABLE_IDENTIFIER); } @Test public void testCreateTableTxn() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); @@ -83,33 +85,33 @@ public void testCreateTableTxn() { .commit(); // verify the table is still not visible before the transaction is committed - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateTableTxnTableCreatedConcurrently() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); // create the table concurrently catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: hivedb.tbl"); } @Test public void testCreateTableTxnAndAppend() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); @@ -126,18 +128,20 @@ public void testCreateTableTxnAndAppend() { Table table = catalog.loadTable(TABLE_IDENTIFIER); Snapshot snapshot = table.currentSnapshot(); - Assert.assertTrue("Table should have one manifest file", snapshot.allManifests(table.io()).size() == 1); + assertThat(snapshot.allManifests(table.io())) + .as("Table should have one manifest file") + .hasSize(1); } @Test public void testCreateTableTxnTableAlreadyExists() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); // create a table before starting a transaction catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.newCreateTableTransaction( TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap())) @@ -145,26 +149,35 @@ public void testCreateTableTxnTableAlreadyExists() { .hasMessage("Table already exists: hivedb.tbl"); } - @Test - public void testReplaceTableTxn() { - catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void testReplaceTableTxn(int formatVersion) { + catalog.createTable( + TABLE_IDENTIFIER, + SCHEMA, + SPEC, + tableLocation, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, false); txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()) - .alwaysNull("id", "id") - .withSpecId(1) - .build(); - Assert.assertEquals("Table should have a spec with one void field", - v1Expected, table.spec()); + if (formatVersion == 1) { + PartitionSpec v1Expected = + PartitionSpec.builderFor(table.schema()).alwaysNull("id", "id").withSpecId(1).build(); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("Table spec must be unpartitioned").isTrue(); + } } @Test public void testReplaceTableTxnTableNotExists() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, false)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: hivedb.tbl"); @@ -173,7 +186,7 @@ public void testReplaceTableTxnTableNotExists() { @Test public void testReplaceTableTxnTableDeletedConcurrently() { catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, false); @@ -183,7 +196,7 @@ public void testReplaceTableTxnTableDeletedConcurrently() { .set("prop", "value") .commit(); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(NoSuchTableException.class) .hasMessage("No such table: hivedb.tbl"); } @@ -191,7 +204,7 @@ public void testReplaceTableTxnTableDeletedConcurrently() { @Test public void testReplaceTableTxnTableModifiedConcurrently() { Table table = catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, false); @@ -207,13 +220,15 @@ public void testReplaceTableTxnTableModifiedConcurrently() { // the replace should still succeed table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertNull("Table props should be updated", table.properties().get("another-prop")); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()) + .as("Table props should be updated") + .doesNotContainKey("another-prop") + .containsEntry("prop", "value"); } @Test public void testCreateOrReplaceTableTxnTableNotExists() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, SPEC, true); txn.updateProperties() @@ -222,32 +237,41 @@ public void testCreateOrReplaceTableTxnTableNotExists() { txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } - @Test - public void testCreateOrReplaceTableTxnTableExists() { - catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void testCreateOrReplaceTableTxnTableExists(int formatVersion) { + catalog.createTable( + TABLE_IDENTIFIER, + SCHEMA, + SPEC, + tableLocation, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); Transaction txn = catalog.newReplaceTableTransaction(TABLE_IDENTIFIER, SCHEMA, true); txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()) - .alwaysNull("id", "id") - .withSpecId(1) - .build(); - Assert.assertEquals("Table should have a spec with one void field", - v1Expected, table.spec()); + if (formatVersion == 1) { + PartitionSpec v1Expected = + PartitionSpec.builderFor(table.schema()).alwaysNull("id", "id").withSpecId(1).build(); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("Table spec must be unpartitioned").isTrue(); + } } @Test public void testCreateOrReplaceTableTxnTableDeletedConcurrently() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); Transaction txn = catalog.newReplaceTableTransaction( TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned(), tableLocation, Maps.newHashMap(), true); @@ -262,12 +286,12 @@ public void testCreateOrReplaceTableTxnTableDeletedConcurrently() { txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateOrReplaceTableTxnTableCreatedConcurrently() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newReplaceTableTransaction( TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned(), tableLocation, Maps.newHashMap(), true); @@ -277,19 +301,21 @@ public void testCreateOrReplaceTableTxnTableCreatedConcurrently() { // create the table concurrently catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC); - Assert.assertTrue("Table should be created", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should be created").isTrue(); // expect the transaction to succeed anyway txn.commitTransaction(); Table table = catalog.loadTable(TABLE_IDENTIFIER); - Assert.assertEquals("Partition spec should match", PartitionSpec.unpartitioned(), table.spec()); - Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + assertThat(table.spec()) + .as("Partition spec should match") + .isEqualTo(PartitionSpec.unpartitioned()); + assertThat(table.properties()).as("Table props should match").containsEntry("prop", "value"); } @Test public void testCreateTableTxnWithGlobalTableLocation() { - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); Transaction txn = catalog.newCreateTableTransaction( TABLE_IDENTIFIER, SCHEMA, SPEC, "file:///" + tableLocation, Maps.newHashMap()); @@ -307,6 +333,6 @@ public void testCreateTableTxnWithGlobalTableLocation() { .appendFile(dataFile) .commit(); - Assert.assertEquals("Write should succeed", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Write should succeed").hasSize(1); } } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java index 7f39fb1505ad..27fbb41f7a01 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveMetastoreTest.java @@ -29,8 +29,8 @@ import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; public abstract class HiveMetastoreTest { @@ -42,7 +42,7 @@ public abstract class HiveMetastoreTest { protected static HiveConf hiveConf; protected static TestHiveMetastore metastore; - @BeforeClass + @BeforeAll public static void startMetastore() throws Exception { startMetastore(Collections.emptyMap()); } @@ -73,7 +73,7 @@ public static void startMetastore(Map hiveConfOverride) throws E hiveConfWithOverrides); } - @AfterClass + @AfterAll public static void stopMetastore() throws Exception { HiveMetastoreTest.catalog = null; diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java index b584b9e73c14..ad6d78cdc91c 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java @@ -30,8 +30,8 @@ import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import static org.apache.iceberg.PartitionSpec.builderFor; import static org.apache.iceberg.TableMetadataParser.getFileExtension; @@ -55,12 +55,12 @@ public class HiveTableBaseTest extends HiveMetastoreTest { private Path tableLocation; - @Before + @BeforeEach public void createTestTable() { this.tableLocation = new Path(catalog.createTable(TABLE_IDENTIFIER, schema, partitionSpec).location()); } - @After + @AfterEach public void dropTestTable() throws Exception { // drop the table data tableLocation.getFileSystem(hiveConf).delete(tableLocation, true); diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 64c6bc3d330b..a6207f3159a0 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -64,11 +65,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import static java.nio.file.Files.createTempDirectory; import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; @@ -80,12 +78,14 @@ import static org.apache.iceberg.TableMetadataParser.getFileExtension; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class HiveTableTest extends HiveTableBaseTest { static final String NON_DEFAULT_DATABASE = "nondefault"; - @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir + private Path tempFolder; @Test public void testCreate() throws TException { @@ -97,23 +97,23 @@ public void testCreate() throws TException { // check parameters are in expected state Map parameters = table.getParameters(); - Assert.assertNotNull(parameters); - Assert.assertTrue(ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(parameters.get(TABLE_TYPE_PROP))); - Assert.assertTrue("EXTERNAL_TABLE".equalsIgnoreCase(table.getTableType())); + assertThat(parameters).isNotNull(); + assertThat(parameters.get(TABLE_TYPE_PROP)).isEqualToIgnoringCase(ICEBERG_TABLE_TYPE_VALUE); + assertThat(table.getTableType()).isEqualToIgnoringCase("EXTERNAL_TABLE"); // Ensure the table is pointing to empty location - Assert.assertEquals(getTableLocation(tableName), table.getSd().getLocation()); + assertThat(table.getSd().getLocation()).isEqualTo(getTableLocation(tableName)); // Ensure it is stored as unpartitioned table in hive. - Assert.assertEquals(0, table.getPartitionKeysSize()); + assertThat(table.getPartitionKeysSize()).isEqualTo(0); // Only 1 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(1, metadataVersionFiles(tableName).size()); - Assert.assertEquals(0, manifestFiles(tableName).size()); + assertThat(metadataVersionFiles(tableName)).hasSize(1); + assertThat(manifestFiles(tableName)).hasSize(0); final Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Iceberg schema should match the loaded table - Assert.assertEquals(schema.asStruct(), icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(schema.asStruct()); } @Test @@ -123,24 +123,26 @@ public void testRename() { Table original = catalog.loadTable(TABLE_IDENTIFIER); catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); - Assert.assertTrue(catalog.tableExists(renameTableIdentifier)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + assertThat(catalog.tableExists(renameTableIdentifier)).isTrue(); Table renamed = catalog.loadTable(renameTableIdentifier); - Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct()); - Assert.assertEquals(original.spec(), renamed.spec()); - Assert.assertEquals(original.location(), renamed.location()); - Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot()); + assertThat(renamed.schema().asStruct()).isEqualTo(original.schema().asStruct()); + assertThat(renamed.spec()).isEqualTo(original.spec()); + assertThat(renamed.location()).isEqualTo(original.location()); + assertThat(renamed.currentSnapshot()).isEqualTo(original.currentSnapshot()); - Assert.assertTrue(catalog.dropTable(renameTableIdentifier)); + assertThat(catalog.dropTable(renameTableIdentifier)).isTrue(); } @Test public void testDrop() { - Assert.assertTrue("Table should exist", catalog.tableExists(TABLE_IDENTIFIER)); - Assert.assertTrue("Drop should return true and drop the table", catalog.dropTable(TABLE_IDENTIFIER)); - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should exist").isTrue(); + assertThat(catalog.dropTable(TABLE_IDENTIFIER)) + .as("Drop should return true and drop the table") + .isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); } @Test @@ -174,14 +176,13 @@ public void testDropWithoutPurgeLeavesTableData() throws IOException { String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); - Assert.assertTrue("Drop should return true and drop the table", - catalog.dropTable(TABLE_IDENTIFIER, false /* do not delete underlying files */)); - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.dropTable(TABLE_IDENTIFIER, false /* do not delete underlying files */)) + .as("Drop should return true and drop the table") + .isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); - Assert.assertTrue("Table data files should exist", - new File(fileLocation).exists()); - Assert.assertTrue("Table metadata files should exist", - new File(manifestListLocation).exists()); + assertThat(new File(fileLocation)).as("Table data files should exist").exists(); + assertThat(new File(manifestListLocation)).as("Table metadata files should exist").exists(); } @Test @@ -237,23 +238,25 @@ public void testDropTable() throws IOException { List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertTrue("Drop (table and data) should return true and drop the table", - catalog.dropTable(TABLE_IDENTIFIER)); - Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.dropTable(TABLE_IDENTIFIER)) + .as("Drop (table and data) should return true and drop the table") + .isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).as("Table should not exist").isFalse(); - Assert.assertFalse("Table data files should not exist", - new File(location1).exists()); - Assert.assertFalse("Table data files should not exist", - new File(location2).exists()); - Assert.assertFalse("Table manifest list files should not exist", - new File(manifestListLocation).exists()); + assertThat(new File(location1)).as("Table data files should not exist").doesNotExist(); + assertThat(new File(location2)).as("Table data files should not exist").doesNotExist(); + assertThat(new File(manifestListLocation)) + .as("Table manifest list files should not exist") + .doesNotExist(); for (ManifestFile manifest : manifests) { - Assert.assertFalse("Table manifest files should not exist", - new File(manifest.path().replace("file:", "")).exists()); + assertThat(new File(manifest.path().replace("file:", ""))) + .as("Table manifest files should not exist") + .doesNotExist(); } - Assert.assertFalse("Table metadata file should not exist", - new File(((HasTableOperations) table).operations().current() - .metadataFileLocation().replace("file:", "")).exists()); + assertThat(new File(((HasTableOperations) table).operations().current().metadataFileLocation() + .replace("file:", ""))) + .as("Table metadata file should not exist") + .doesNotExist(); } @Test @@ -265,9 +268,9 @@ public void testExistingTableUpdate() throws TException { icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Only 2 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size()); - Assert.assertEquals(0, manifestFiles(TABLE_NAME).size()); - Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct()); + assertThat(metadataVersionFiles(TABLE_NAME)).hasSize(2); + assertThat(manifestFiles(TABLE_NAME)).hasSize(0); + assertThat(icebergTable.schema().asStruct()).isEqualTo(altered.asStruct()); final org.apache.hadoop.hive.metastore.api.Table table = metastoreClient.getTable(DB_NAME, TABLE_NAME); final List hiveColumns = table.getSd().getCols().stream() @@ -276,7 +279,7 @@ public void testExistingTableUpdate() throws TException { final List icebergColumns = altered.columns().stream() .map(Types.NestedField::name) .collect(Collectors.toList()); - Assert.assertEquals(icebergColumns, hiveColumns); + assertThat(hiveColumns).isEqualTo(icebergColumns); } @Test @@ -297,7 +300,9 @@ public void testColumnTypeChangeInMetastore() throws TException { .addColumn("int", Types.IntegerType.get()) .commit(); - Assert.assertEquals("Schema should match expected", expectedSchema.asStruct(), icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema.asStruct()); expectedSchema = new Schema(Types.StructType.of( required(1, "id", Types.LongType.get()), @@ -305,19 +310,23 @@ public void testColumnTypeChangeInMetastore() throws TException { optional(4, "int", Types.IntegerType.get())).fields()); icebergTable.updateSchema().deleteColumn("string").commit(); - Assert.assertEquals("Schema should match expected", expectedSchema.asStruct(), icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema.asStruct()); } - @Test(expected = CommitFailedException.class) + @Test public void testFailure() throws TException { Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); org.apache.hadoop.hive.metastore.api.Table table = metastoreClient.getTable(DB_NAME, TABLE_NAME); String dummyLocation = "dummylocation"; table.getParameters().put(METADATA_LOCATION_PROP, dummyLocation); metastoreClient.alter_table(DB_NAME, TABLE_NAME, table); - icebergTable.updateSchema() + assertThatThrownBy(() -> icebergTable.updateSchema() .addColumn("data", Types.LongType.get()) - .commit(); + .commit()) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("is not same as the current table metadata location 'dummylocation'"); } @Test @@ -327,8 +336,8 @@ public void testListTables() throws TException, IOException { .filter(t -> t.namespace().level(0).equals(DB_NAME) && t.name().equals(TABLE_NAME)) .collect(Collectors.toList()); - Assert.assertEquals(1, expectedIdents.size()); - Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(expectedIdents).hasSize(1); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); // create a hive table String hiveTableName = "test_hive_table"; @@ -337,13 +346,13 @@ public void testListTables() throws TException, IOException { catalog.setListAllTables(false); List tableIdents1 = catalog.listTables(TABLE_IDENTIFIER.namespace()); - Assert.assertEquals("should only 1 iceberg table .", 1, tableIdents1.size()); + assertThat(tableIdents1).as("should only 1 iceberg table .").hasSize(1); catalog.setListAllTables(true); List tableIdents2 = catalog.listTables(TABLE_IDENTIFIER.namespace()); - Assert.assertEquals("should be 2 tables in namespace .", 2, tableIdents2.size()); + assertThat(tableIdents2).as("should be 2 tables in namespace .").hasSize(2); - Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); metastoreClient.dropTable(DB_NAME, hiveTableName); } @@ -355,7 +364,7 @@ private org.apache.hadoop.hive.metastore.api.Table createHiveTable(String hiveTa SerDeInfo serDeInfo = new SerDeInfo(null, "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer", parameters); // StorageDescriptor has an empty list of fields - SerDe will report them. - StorageDescriptor sd = new StorageDescriptor(Lists.newArrayList(), tempFolder.newFolder().getAbsolutePath(), + StorageDescriptor sd = new StorageDescriptor(Lists.newArrayList(), tempFolder.toAbsolutePath().toString(), "org.apache.hadoop.mapred.TextInputFormat", "org.apache.hadoop.mapred.TextOutputFormat", false, -1, serDeInfo, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap()); @@ -375,14 +384,14 @@ public void testNonDefaultDatabaseLocation() throws IOException, TException { catalog.createNamespace(namespace, Collections.singletonMap("location", nonDefaultLocation.getPath())); Map namespaceMeta = catalog.loadNamespaceMetadata(namespace); // Make sure that we are testing a namespace with a non default location :) - Assert.assertEquals(namespaceMeta.get("location"), "file:" + nonDefaultLocation.getPath()); + assertThat("file:" + nonDefaultLocation.getPath()).isEqualTo(namespaceMeta.get("location")); TableIdentifier tableIdentifier = TableIdentifier.of(namespace, TABLE_NAME); catalog.createTable(tableIdentifier, schema); // Let's check the location loaded through the catalog Table table = catalog.loadTable(tableIdentifier); - Assert.assertEquals(namespaceMeta.get("location") + "/" + TABLE_NAME, table.location()); + assertThat(table.location()).isEqualTo(namespaceMeta.get("location") + "/" + TABLE_NAME); // Drop the database and purge the files metastoreClient.dropDatabase(NON_DEFAULT_DATABASE, true, true, true); @@ -393,31 +402,32 @@ public void testRegisterTable() throws TException { org.apache.hadoop.hive.metastore.api.Table originalTable = metastoreClient.getTable(DB_NAME, TABLE_NAME); Map originalParams = originalTable.getParameters(); - Assert.assertNotNull(originalParams); - Assert.assertTrue(ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(originalParams.get(TABLE_TYPE_PROP))); - Assert.assertTrue("EXTERNAL_TABLE".equalsIgnoreCase(originalTable.getTableType())); + assertThat(originalParams).isNotNull(); + assertThat(originalParams.get(TABLE_TYPE_PROP)).isEqualToIgnoringCase(ICEBERG_TABLE_TYPE_VALUE); + assertThat(originalTable.getTableType()).isEqualToIgnoringCase("EXTERNAL_TABLE"); catalog.dropTable(TABLE_IDENTIFIER, false); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); List metadataVersionFiles = metadataVersionFiles(TABLE_NAME); - Assert.assertEquals(1, metadataVersionFiles.size()); + assertThat(metadataVersionFiles).hasSize(1); catalog.registerTable(TABLE_IDENTIFIER, "file:" + metadataVersionFiles.get(0)); org.apache.hadoop.hive.metastore.api.Table newTable = metastoreClient.getTable(DB_NAME, TABLE_NAME); Map newTableParameters = newTable.getParameters(); - Assert.assertNull(newTableParameters.get(PREVIOUS_METADATA_LOCATION_PROP)); - Assert.assertEquals(originalParams.get(TABLE_TYPE_PROP), newTableParameters.get(TABLE_TYPE_PROP)); - Assert.assertEquals(originalParams.get(METADATA_LOCATION_PROP), newTableParameters.get(METADATA_LOCATION_PROP)); - Assert.assertEquals(originalTable.getSd(), newTable.getSd()); + assertThat(newTableParameters) + .doesNotContainKey(PREVIOUS_METADATA_LOCATION_PROP) + .containsEntry(TABLE_TYPE_PROP, originalParams.get(TABLE_TYPE_PROP)) + .containsEntry(METADATA_LOCATION_PROP, originalParams.get(METADATA_LOCATION_PROP)); + assertThat(newTable.getSd()).isEqualTo(originalTable.getSd()); } @Test public void testRegisterHadoopTableToHiveCatalog() throws IOException, TException { // create a hadoop catalog - String tableLocation = tempFolder.newFolder().toString(); + String tableLocation = tempFolder.toString(); HadoopCatalog hadoopCatalog = new HadoopCatalog(new Configuration(), tableLocation); // create table using hadoop catalog TableIdentifier identifier = TableIdentifier.of(DB_NAME, "table1"); @@ -427,8 +437,8 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio // insert some data String file1Location = appendData(table, "file1"); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 1 file", 1, tasks.size()); - Assert.assertEquals(tasks.get(0).file().path(), file1Location); + assertThat(tasks).as("Should scan 1 file").hasSize(1); + assertThat(file1Location).isEqualTo(tasks.get(0).file().path()); // collect metadata file List metadataFiles = @@ -436,31 +446,31 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio .map(File::getAbsolutePath) .filter(f -> f.endsWith(getFileExtension(TableMetadataParser.Codec.NONE))) .collect(Collectors.toList()); - Assert.assertEquals(2, metadataFiles.size()); + assertThat(metadataFiles).hasSize(2); - Assertions.assertThatThrownBy(() -> metastoreClient.getTable(DB_NAME, "table1")) + assertThatThrownBy(() -> metastoreClient.getTable(DB_NAME, "table1")) .isInstanceOf(NoSuchObjectException.class) .hasMessage("hive.hivedb.table1 table not found"); - Assertions.assertThatThrownBy(() -> catalog.loadTable(identifier)) + assertThatThrownBy(() -> catalog.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: hivedb.table1"); // register the table to hive catalog using the latest metadata file String latestMetadataFile = ((BaseTable) table).operations().current().metadataFileLocation(); catalog.registerTable(identifier, "file:" + latestMetadataFile); - Assert.assertNotNull(metastoreClient.getTable(DB_NAME, "table1")); + assertThat(metastoreClient.getTable(DB_NAME, "table1")).isNotNull(); // load the table in hive catalog table = catalog.loadTable(identifier); - Assert.assertNotNull(table); + assertThat(table).isNotNull(); // insert some data String file2Location = appendData(table, "file2"); tasks = Lists.newArrayList(table.newScan().planFiles()); - Assert.assertEquals("Should scan 2 files", 2, tasks.size()); + assertThat(tasks).as("Should scan 2 files").hasSize(2); Set files = tasks.stream().map(task -> task.file().path().toString()).collect(Collectors.toSet()); - Assert.assertTrue(files.contains(file1Location) && files.contains(file2Location)); + assertThat(files).contains(file1Location, file2Location); } private String appendData(Table table, String fileName) throws IOException { @@ -497,15 +507,15 @@ public void testRegisterExistingTable() throws TException { org.apache.hadoop.hive.metastore.api.Table originalTable = metastoreClient.getTable(DB_NAME, TABLE_NAME); Map originalParams = originalTable.getParameters(); - Assert.assertNotNull(originalParams); - Assert.assertTrue(ICEBERG_TABLE_TYPE_VALUE.equalsIgnoreCase(originalParams.get(TABLE_TYPE_PROP))); - Assert.assertTrue("EXTERNAL_TABLE".equalsIgnoreCase(originalTable.getTableType())); + assertThat(originalParams).isNotNull(); + assertThat(originalParams.get(TABLE_TYPE_PROP)).isEqualToIgnoringCase(ICEBERG_TABLE_TYPE_VALUE); + assertThat(originalTable.getTableType()).isEqualToIgnoringCase("EXTERNAL_TABLE"); List metadataVersionFiles = metadataVersionFiles(TABLE_NAME); - Assert.assertEquals(1, metadataVersionFiles.size()); + assertThat(metadataVersionFiles).hasSize(1); // Try to register an existing table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable(TABLE_IDENTIFIER, "file:" + metadataVersionFiles.get(0))) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: hivedb.tbl"); @@ -582,29 +592,34 @@ public void testMissingMetadataWontCauseHang() { File realLocation = new File(metadataLocation(TABLE_NAME)); File fakeLocation = new File(metadataLocation(TABLE_NAME) + "_dummy"); - Assert.assertTrue(realLocation.renameTo(fakeLocation)); - Assertions.assertThatThrownBy(() -> catalog.loadTable(TABLE_IDENTIFIER)) + assertThat(realLocation.renameTo(fakeLocation)).isTrue(); + assertThatThrownBy(() -> catalog.loadTable(TABLE_IDENTIFIER)) .isInstanceOf(NotFoundException.class) .hasMessageStartingWith("Failed to open input stream for file"); - Assert.assertTrue(fakeLocation.renameTo(realLocation)); + assertThat(fakeLocation.renameTo(realLocation)).isTrue(); } private void assertHiveEnabled(org.apache.hadoop.hive.metastore.api.Table hmsTable, boolean expected) { if (expected) { - Assert.assertEquals("org.apache.iceberg.mr.hive.HiveIcebergStorageHandler", - hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals("org.apache.iceberg.mr.hive.HiveIcebergSerDe", - hmsTable.getSd().getSerdeInfo().getSerializationLib()); - Assert.assertEquals("org.apache.iceberg.mr.hive.HiveIcebergInputFormat", - hmsTable.getSd().getInputFormat()); - Assert.assertEquals("org.apache.iceberg.mr.hive.HiveIcebergOutputFormat", - hmsTable.getSd().getOutputFormat()); + assertThat(hmsTable.getParameters()) + .containsEntry( + hive_metastoreConstants.META_TABLE_STORAGE, + "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"); + assertThat(hmsTable.getSd().getSerdeInfo().getSerializationLib()) + .isEqualTo("org.apache.iceberg.mr.hive.HiveIcebergSerDe"); + assertThat(hmsTable.getSd().getInputFormat()) + .isEqualTo("org.apache.iceberg.mr.hive.HiveIcebergInputFormat"); + assertThat(hmsTable.getSd().getOutputFormat()) + .isEqualTo("org.apache.iceberg.mr.hive.HiveIcebergOutputFormat"); } else { - Assert.assertNull(hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", - hmsTable.getSd().getSerdeInfo().getSerializationLib()); - Assert.assertEquals("org.apache.hadoop.mapred.FileInputFormat", hmsTable.getSd().getInputFormat()); - Assert.assertEquals("org.apache.hadoop.mapred.FileOutputFormat", hmsTable.getSd().getOutputFormat()); + assertThat(hmsTable.getParameters()) + .doesNotContainKey(hive_metastoreConstants.META_TABLE_STORAGE); + assertThat(hmsTable.getSd().getSerdeInfo().getSerializationLib()) + .isEqualTo("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"); + assertThat(hmsTable.getSd().getInputFormat()) + .isEqualTo("org.apache.hadoop.mapred.FileInputFormat"); + assertThat(hmsTable.getSd().getOutputFormat()) + .isEqualTo("org.apache.hadoop.mapred.FileOutputFormat"); } } } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java index fd9e5f569e4f..8348c8f3293c 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestCachedClientPool.java @@ -29,12 +29,12 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hive.CachedClientPool.Key; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestCachedClientPool extends HiveMetastoreTest { @@ -42,21 +42,22 @@ public class TestCachedClientPool extends HiveMetastoreTest { public void testClientPoolCleaner() throws InterruptedException { CachedClientPool clientPool = new CachedClientPool(hiveConf, Collections.emptyMap()); HiveClientPool clientPool1 = clientPool.clientPool(); - Assertions.assertThat( + assertThat(clientPool1) + .isSameAs( CachedClientPool.clientPoolCache() - .getIfPresent(CachedClientPool.extractKey(null, hiveConf))) - .isSameAs(clientPool1); + .getIfPresent(CachedClientPool.extractKey(null, hiveConf))); TimeUnit.MILLISECONDS.sleep(EVICTION_INTERVAL - TimeUnit.SECONDS.toMillis(2)); HiveClientPool clientPool2 = clientPool.clientPool(); - Assert.assertSame(clientPool1, clientPool2); + assertThat(clientPool2).isSameAs(clientPool1); TimeUnit.MILLISECONDS.sleep(EVICTION_INTERVAL + TimeUnit.SECONDS.toMillis(5)); - Assert.assertNull( + assertThat( CachedClientPool.clientPoolCache() - .getIfPresent(CachedClientPool.extractKey(null, hiveConf))); + .getIfPresent(CachedClientPool.extractKey(null, hiveConf))) + .isNull(); // The client has been really closed. - Assert.assertTrue(clientPool1.isClosed()); - Assert.assertTrue(clientPool2.isClosed()); + assertThat(clientPool1.isClosed()).isTrue(); + assertThat(clientPool2.isClosed()).isTrue(); } @Test @@ -74,19 +75,22 @@ public void testCacheKey() throws Exception { foo2.doAs( (PrivilegedAction) () -> CachedClientPool.extractKey("conf:key1,user_name", hiveConf)); - Assert.assertEquals("Key elements order shouldn't matter", key1, key2); + assertThat(key2).as("Key elements order shouldn't matter").isEqualTo(key1); key1 = foo1.doAs((PrivilegedAction) () -> CachedClientPool.extractKey("ugi", hiveConf)); key2 = bar.doAs((PrivilegedAction) () -> CachedClientPool.extractKey("ugi", hiveConf)); - Assert.assertNotEquals("Different users are not supposed to be equivalent", key1, key2); + assertThat(key2).as("Different users are not supposed to be equivalent").isNotEqualTo(key1); key2 = foo2.doAs((PrivilegedAction) () -> CachedClientPool.extractKey("ugi", hiveConf)); - Assert.assertNotEquals("Different UGI instances are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Different UGI instances are not supposed to be equivalent") + .isNotEqualTo(key1); key1 = CachedClientPool.extractKey("ugi", hiveConf); key2 = CachedClientPool.extractKey("ugi,conf:key1", hiveConf); - Assert.assertNotEquals( - "Keys with different number of elements are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Keys with different number of elements are not supposed to be equivalent") + .isNotEqualTo(key1); Configuration conf1 = new Configuration(hiveConf); Configuration conf2 = new Configuration(hiveConf); @@ -94,31 +98,33 @@ public void testCacheKey() throws Exception { conf1.set("key1", "val"); key1 = CachedClientPool.extractKey("conf:key1", conf1); key2 = CachedClientPool.extractKey("conf:key1", conf2); - Assert.assertNotEquals( - "Config with different values are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Config with different values are not supposed to be equivalent") + .isNotEqualTo(key1); conf2.set("key1", "val"); conf2.set("key2", "val"); key2 = CachedClientPool.extractKey("conf:key2", conf2); - Assert.assertNotEquals( - "Config with different keys are not supposed to be equivalent", key1, key2); + assertThat(key2) + .as("Config with different keys are not supposed to be equivalent") + .isNotEqualTo(key1); key1 = CachedClientPool.extractKey("conf:key1,ugi", conf1); key2 = CachedClientPool.extractKey("ugi,conf:key1", conf2); - Assert.assertEquals("Config with same key/value should be equivalent", key1, key2); + assertThat(key2).as("Config with same key/value should be equivalent").isEqualTo(key1); conf1.set("key2", "val"); key1 = CachedClientPool.extractKey("conf:key2 ,conf:key1", conf1); key2 = CachedClientPool.extractKey("conf:key2,conf:key1", conf2); - Assert.assertEquals("Config with same key/value should be equivalent", key1, key2); + assertThat(key2).as("Config with same key/value should be equivalent").isEqualTo(key1); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CachedClientPool.extractKey("ugi,ugi", hiveConf), "Duplicate key elements should result in an error") .isInstanceOf(ValidationException.class) .hasMessageContaining("UGI key element already specified"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CachedClientPool.extractKey("conf:k1,conf:k2,CONF:k1", hiveConf), "Duplicate conf key elements should result in an error") .isInstanceOf(ValidationException.class) @@ -154,16 +160,16 @@ public void testHmsCatalog() { HiveClientPool pool3 = ((CachedClientPool) catalog3.clientPool()).clientPool(); HiveClientPool pool4 = ((CachedClientPool) catalog4.clientPool()).clientPool(); - Assert.assertSame(pool1, pool2); - Assert.assertNotSame(pool3, pool1); - Assert.assertNotSame(pool3, pool2); - Assert.assertNotSame(pool3, pool4); - Assert.assertNotSame(pool4, pool1); - Assert.assertNotSame(pool4, pool2); + assertThat(pool2).isSameAs(pool1); + assertThat(pool1).isNotSameAs(pool3); + assertThat(pool2).isNotSameAs(pool3); + assertThat(pool4).isNotSameAs(pool3); + assertThat(pool1).isNotSameAs(pool4); + assertThat(pool2).isNotSameAs(pool4); - Assert.assertEquals("foo", pool1.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)); - Assert.assertEquals("bar", pool3.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)); - Assert.assertNull(pool4.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)); + assertThat(pool1.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)).isEqualTo("foo"); + assertThat(pool3.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)).isEqualTo("bar"); + assertThat(pool4.hiveConf().get(HiveCatalog.HIVE_CONF_CATALOG)).isNull(); pool1.close(); pool3.close(); diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index 03a1bcf74103..d65d38085a1d 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -20,6 +20,7 @@ package org.apache.iceberg.hive; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.Set; @@ -65,11 +66,10 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; import org.apache.thrift.TException; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.jupiter.api.Assertions; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import static org.apache.iceberg.NullOrder.NULLS_FIRST; import static org.apache.iceberg.SortDirection.ASC; @@ -79,9 +79,11 @@ import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_TIMESTAMP; import static org.apache.iceberg.TableProperties.DEFAULT_PARTITION_SPEC; import static org.apache.iceberg.TableProperties.DEFAULT_SORT_ORDER; +import static org.apache.iceberg.TableProperties.SNAPSHOT_COUNT; import static org.apache.iceberg.expressions.Expressions.bucket; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -92,8 +94,8 @@ public class TestHiveCatalog extends HiveMetastoreTest { "group", "iceberg", "comment", "iceberg hiveCatalog test"); - @Rule - public TemporaryFolder temp = new TemporaryFolder(); + @TempDir + private Path temp; private Schema getTestSchema() { return new Schema( @@ -106,7 +108,7 @@ public void testCreateTableBuilder() throws Exception { Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { Table table = catalog.buildTable(tableIdent, schema) @@ -116,11 +118,16 @@ public void testCreateTableBuilder() throws Exception { .withProperty("key2", "value2") .create(); - Assert.assertEquals(location, table.location()); - Assert.assertEquals(2, table.schema().columns().size()); - Assert.assertEquals(1, table.spec().fields().size()); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.location()).isEqualTo(location); + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1"); + assertThat(table.properties()).containsEntry("key2", "value2"); + // default Parquet compression is explicitly set for new tables + assertThat(table.properties()) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0); } finally { catalog.dropTable(tableIdent); } @@ -131,18 +138,23 @@ public void testCreateTableWithCaching() throws Exception { Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); ImmutableMap properties = ImmutableMap.of("key1", "value1", "key2", "value2"); Catalog cachingCatalog = CachingCatalog.wrap(catalog); try { Table table = cachingCatalog.createTable(tableIdent, schema, spec, location, properties); - Assert.assertEquals(location, table.location()); - Assert.assertEquals(2, table.schema().columns().size()); - Assert.assertEquals(1, table.spec().fields().size()); - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.location()).isEqualTo(location); + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1"); + assertThat(table.properties()).containsEntry("key2", "value2"); + // default Parquet compression is explicitly set for new tables + assertThat(table.properties()) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0); } finally { cachingCatalog.dropTable(tableIdent); } @@ -150,18 +162,22 @@ public void testCreateTableWithCaching() throws Exception { @Test public void testInitialize() { - Assertions.assertDoesNotThrow(() -> { - HiveCatalog catalog = new HiveCatalog(); - catalog.initialize("hive", Maps.newHashMap()); - }); + assertThatNoException() + .isThrownBy( + () -> { + HiveCatalog catalog = new HiveCatalog(); + catalog.initialize("hive", Maps.newHashMap()); + }); } @Test public void testToStringWithoutSetConf() { - Assertions.assertDoesNotThrow(() -> { - HiveCatalog catalog = new HiveCatalog(); - catalog.toString(); - }); + assertThatNoException() + .isThrownBy( + () -> { + HiveCatalog catalog = new HiveCatalog(); + catalog.toString(); + }); } @Test @@ -172,15 +188,16 @@ public void testInitializeCatalogWithProperties() { HiveCatalog catalog = new HiveCatalog(); catalog.initialize("hive", properties); - Assert.assertEquals(catalog.getConf().get("hive.metastore.uris"), "thrift://examplehost:9083"); - Assert.assertEquals(catalog.getConf().get("hive.metastore.warehouse.dir"), "/user/hive/testwarehouse"); + assertThat(catalog.getConf().get("hive.metastore.uris")).isEqualTo("thrift://examplehost:9083"); + assertThat(catalog.getConf().get("hive.metastore.warehouse.dir")) + .isEqualTo("/user/hive/testwarehouse"); } @Test public void testCreateTableTxnBuilder() throws Exception { Schema schema = getTestSchema(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { Transaction txn = catalog.buildTable(tableIdent, schema) @@ -189,33 +206,35 @@ public void testCreateTableTxnBuilder() throws Exception { txn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals(location, table.location()); - Assert.assertEquals(2, table.schema().columns().size()); - Assert.assertTrue(table.spec().isUnpartitioned()); + assertThat(table.location()).isEqualTo(location); + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.spec().isUnpartitioned()).isTrue(); } finally { catalog.dropTable(tableIdent); } } - @Test - public void testReplaceTxnBuilder() throws Exception { + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void testReplaceTxnBuilder(int formatVersion) { Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { Transaction createTxn = catalog.buildTable(tableIdent, schema) .withPartitionSpec(spec) .withLocation(location) .withProperty("key1", "value1") + .withProperty(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)) .createOrReplaceTransaction(); createTxn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals(1, table.spec().fields().size()); + assertThat(table.spec().fields()).hasSize(1); - String newLocation = temp.newFolder("tbl-2").toString(); + String newLocation = temp.resolve("tbl-2").toString(); Transaction replaceTxn = catalog.buildTable(tableIdent, schema) .withProperty("key2", "value2") @@ -224,17 +243,23 @@ public void testReplaceTxnBuilder() throws Exception { replaceTxn.commitTransaction(); table = catalog.loadTable(tableIdent); - Assert.assertEquals(newLocation, table.location()); - Assert.assertNull(table.currentSnapshot()); - PartitionSpec v1Expected = PartitionSpec.builderFor(table.schema()) - .alwaysNull("data", "data_bucket") - .withSpecId(1) - .build(); - Assert.assertEquals("Table should have a spec with one void field", - v1Expected, table.spec()); + assertThat(table.location()).isEqualTo(newLocation); + assertThat(table.currentSnapshot()).isNull(); + if (formatVersion == 1) { + PartitionSpec v1Expected = + PartitionSpec.builderFor(table.schema()) + .alwaysNull("data", "data_bucket") + .withSpecId(1) + .build(); + assertThat(table.spec()) + .as("Table should have a spec with one void field") + .isEqualTo(v1Expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("Table spec must be unpartitioned").isTrue(); + } - Assert.assertEquals("value1", table.properties().get("key1")); - Assert.assertEquals("value2", table.properties().get("key2")); + assertThat(table.properties()).containsEntry("key1", "value1"); + assertThat(table.properties()).containsEntry("key2", "value2"); } finally { catalog.dropTable(tableIdent); } @@ -260,13 +285,13 @@ private void createTableAndVerifyOwner( Schema schema = getTestSchema(); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 16).build(); TableIdentifier tableIdent = TableIdentifier.of(db, tbl); - String location = temp.newFolder(tbl).toString(); + String location = temp.resolve(tbl).toString(); try { Table table = catalog.createTable(tableIdent, schema, spec, location, properties); org.apache.hadoop.hive.metastore.api.Table hmsTable = metastoreClient.getTable(db, tbl); - Assert.assertEquals(owner, hmsTable.getOwner()); + assertThat(hmsTable.getOwner()).isEqualTo(owner); Map hmsTableParams = hmsTable.getParameters(); - Assert.assertFalse(hmsTableParams.containsKey(HiveCatalog.HMS_TABLE_OWNER)); + assertThat(hmsTableParams).doesNotContainKey(HiveCatalog.HMS_TABLE_OWNER); } finally { catalog.dropTable(tableIdent); } @@ -280,11 +305,12 @@ public void testCreateTableDefaultSortOrder() throws Exception { try { Table table = catalog.createTable(tableIdent, schema, spec); - Assert.assertEquals("Order ID must match", 0, table.sortOrder().orderId()); - Assert.assertTrue("Order must unsorted", table.sortOrder().isUnsorted()); + assertThat(table.sortOrder().orderId()).as("Order ID must match").isEqualTo(0); + assertThat(table.sortOrder().isUnsorted()).as("Order must unsorted").isTrue(); - Assert.assertFalse("Must not have default sort order in catalog", - hmsTableParameters().containsKey(DEFAULT_SORT_ORDER)); + assertThat(hmsTableParameters()) + .as("Must not have default sort order in catalog") + .doesNotContainKey(DEFAULT_SORT_ORDER); } finally { catalog.dropTable(tableIdent); } @@ -303,14 +329,19 @@ public void testCreateTableCustomSortOrder() throws Exception { .withSortOrder(order) .create(); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); - Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); - Assert.assertEquals("Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + assertThat(sortOrder.fields()).as("Order must have 1 field").hasSize(1); + assertThat(sortOrder.fields().get(0).direction()).as("Direction must match ").isEqualTo(ASC); + assertThat(sortOrder.fields().get(0).nullOrder()) + .as("Null order must match ") + .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(Types.IntegerType.get()); - Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + assertThat(sortOrder.fields().get(0).transform()) + .as("Transform must match") + .isEqualTo(transform); - Assert.assertEquals(SortOrderParser.toJson(table.sortOrder()), hmsTableParameters().get(DEFAULT_SORT_ORDER)); + assertThat(hmsTableParameters()) + .containsEntry(DEFAULT_SORT_ORDER, SortOrderParser.toJson(table.sortOrder())); } finally { catalog.dropTable(tableIdent); } @@ -322,16 +353,17 @@ public void testCreateNamespace() throws Exception { catalog.createNamespace(namespace1, meta); Database database1 = metastoreClient.getDatabase(namespace1.toString()); - Assert.assertTrue(database1.getParameters().get("owner").equals("apache")); - Assert.assertTrue(database1.getParameters().get("group").equals("iceberg")); + assertThat(database1.getParameters()).containsEntry("owner", "apache"); + assertThat(database1.getParameters()).containsEntry("group", "iceberg"); - Assert.assertEquals("There no same location for db and namespace", - database1.getLocationUri(), defaultUri(namespace1)); + assertThat(defaultUri(namespace1)) + .as("There no same location for db and namespace") + .isEqualTo(database1.getLocationUri()); assertThatThrownBy(() -> catalog.createNamespace(namespace1)) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Namespace '" + namespace1 + "' already exists!"); - String hiveLocalDir = temp.newFolder().toURI().toString(); + String hiveLocalDir = temp.toFile().toURI().toString(); // remove the trailing slash of the URI hiveLocalDir = hiveLocalDir.substring(0, hiveLocalDir.length() - 1); ImmutableMap newMeta = ImmutableMap.builder() @@ -342,8 +374,9 @@ public void testCreateNamespace() throws Exception { catalog.createNamespace(namespace2, newMeta); Database database2 = metastoreClient.getDatabase(namespace2.toString()); - Assert.assertEquals("There no same location for db and namespace", - database2.getLocationUri(), hiveLocalDir); + assertThat(hiveLocalDir) + .as("There no same location for db and namespace") + .isEqualTo(database2.getLocationUri()); } @Test @@ -422,8 +455,8 @@ private void createNamespaceAndVerifyOwnership( catalog.createNamespace(namespace, prop); Database db = metastoreClient.getDatabase(namespace.toString()); - Assert.assertEquals(expectedOwner, db.getOwnerName()); - Assert.assertEquals(expectedOwnerType, db.getOwnerType()); + assertThat(db.getOwnerName()).isEqualTo(expectedOwner); + assertThat(db.getOwnerType()).isEqualTo(expectedOwnerType); } @Test @@ -432,13 +465,13 @@ public void testListNamespace() throws TException { Namespace namespace1 = Namespace.of("dbname1"); catalog.createNamespace(namespace1, meta); namespaces = catalog.listNamespaces(namespace1); - Assert.assertTrue("Hive db not hive the namespace 'dbname1'", namespaces.isEmpty()); + assertThat(namespaces).as("Hive db not hive the namespace 'dbname1'").isEmpty(); Namespace namespace2 = Namespace.of("dbname2"); catalog.createNamespace(namespace2, meta); namespaces = catalog.listNamespaces(); - Assert.assertTrue("Hive db not hive the namespace 'dbname2'", namespaces.contains(namespace2)); + assertThat(namespaces).as("Hive db not hive the namespace 'dbname2'").contains(namespace2); } @Test @@ -448,10 +481,11 @@ public void testLoadNamespaceMeta() throws TException { catalog.createNamespace(namespace, meta); Map nameMata = catalog.loadNamespaceMetadata(namespace); - Assert.assertTrue(nameMata.get("owner").equals("apache")); - Assert.assertTrue(nameMata.get("group").equals("iceberg")); - Assert.assertEquals("There no same location for db and namespace", - nameMata.get("location"), catalog.convertToDatabase(namespace, meta).getLocationUri()); + assertThat(nameMata).containsEntry("owner", "apache"); + assertThat(nameMata).containsEntry("group", "iceberg"); + assertThat(catalog.convertToDatabase(namespace, meta).getLocationUri()) + .as("There no same location for db and namespace") + .isEqualTo(nameMata.get("location")); } @Test @@ -460,10 +494,10 @@ public void testNamespaceExists() throws TException { catalog.createNamespace(namespace, meta); - Assert.assertTrue("Should true to namespace exist", - catalog.namespaceExists(namespace)); - Assert.assertTrue("Should false to namespace doesn't exist", - !catalog.namespaceExists(Namespace.of("db2", "db2", "ns2"))); + assertThat(catalog.namespaceExists(namespace)).as("Should true to namespace exist").isTrue(); + assertThat(catalog.namespaceExists(Namespace.of("db2", "db2", "ns2"))) + .as("Should false to namespace doesn't exist") + .isFalse(); } @Test @@ -480,9 +514,9 @@ public void testSetNamespaceProperties() throws TException { ); Database database = metastoreClient.getDatabase(namespace.level(0)); - Assert.assertEquals(database.getParameters().get("owner"), "alter_apache"); - Assert.assertEquals(database.getParameters().get("test"), "test"); - Assert.assertEquals(database.getParameters().get("group"), "iceberg"); + assertThat(database.getParameters()).containsEntry("owner", "alter_apache"); + assertThat(database.getParameters()).containsEntry("test", "test"); + assertThat(database.getParameters()).containsEntry("group", "iceberg"); assertThatThrownBy( () -> catalog.setProperties(Namespace.of("db2", "db2", "ns2"), ImmutableMap.of())) .isInstanceOf(NoSuchNamespaceException.class) @@ -666,8 +700,8 @@ private void setNamespaceOwnershipAndVerify( catalog.setProperties(Namespace.of(name), propToSet); Database database = metastoreClient.getDatabase(name); - Assert.assertEquals(expectedOwnerPostSet, database.getOwnerName()); - Assert.assertEquals(expectedOwnerTypePostSet, database.getOwnerType()); + assertThat(database.getOwnerName()).isEqualTo(expectedOwnerPostSet); + assertThat(database.getOwnerType()).isEqualTo(expectedOwnerTypePostSet); } @Test @@ -680,8 +714,8 @@ public void testRemoveNamespaceProperties() throws TException { Database database = metastoreClient.getDatabase(namespace.level(0)); - Assert.assertEquals(database.getParameters().get("owner"), null); - Assert.assertEquals(database.getParameters().get("group"), "iceberg"); + assertThat(database.getParameters()).doesNotContainKey("owner"); + assertThat(database.getParameters()).containsEntry("group", "iceberg"); assertThatThrownBy( () -> @@ -812,8 +846,8 @@ private void removeNamespaceOwnershipAndVerify( Database database = metastoreClient.getDatabase(name); - Assert.assertEquals(expectedOwnerPostRemove, database.getOwnerName()); - Assert.assertEquals(expectedOwnerTypePostRemove, database.getOwnerType()); + assertThat(database.getOwnerName()).isEqualTo(expectedOwnerPostRemove); + assertThat(database.getOwnerType()).isEqualTo(expectedOwnerTypePostRemove); } @Test @@ -825,17 +859,19 @@ public void testDropNamespace() throws TException { catalog.createNamespace(namespace, meta); catalog.createTable(identifier, schema); Map nameMata = catalog.loadNamespaceMetadata(namespace); - Assert.assertTrue(nameMata.get("owner").equals("apache")); - Assert.assertTrue(nameMata.get("group").equals("iceberg")); + assertThat(nameMata).containsEntry("owner", "apache"); + assertThat(nameMata).containsEntry("group", "iceberg"); assertThatThrownBy(() -> catalog.dropNamespace(namespace)) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace dbname_drop is not empty. One or more tables exist."); - Assert.assertTrue(catalog.dropTable(identifier, true)); - Assert.assertTrue("Should fail to drop namespace if it is not empty", - catalog.dropNamespace(namespace)); - Assert.assertFalse("Should fail to drop when namespace doesn't exist", - catalog.dropNamespace(Namespace.of("db.ns1"))); + assertThat(catalog.dropTable(identifier, true)).isTrue(); + assertThat(catalog.dropNamespace(namespace)) + .as("Should fail to drop namespace if it is not empty") + .isTrue(); + assertThat(catalog.dropNamespace(Namespace.of("db.ns1"))) + .as("Should fail to drop when namespace doesn't exist") + .isFalse(); assertThatThrownBy(() -> catalog.loadNamespaceMetadata(namespace)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: dbname_drop"); @@ -849,7 +885,7 @@ public void testDropTableWithoutMetadataFile() { String metadataFileLocation = catalog.newTableOps(identifier).current().metadataFileLocation(); TableOperations ops = catalog.newTableOps(identifier); ops.io().deleteFile(metadataFileLocation); - Assert.assertTrue(catalog.dropTable(identifier)); + assertThat(catalog.dropTable(identifier)).isTrue(); org.assertj.core.api.Assertions.assertThatThrownBy(() -> catalog.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) @@ -868,11 +904,13 @@ public void testTableName() { .create(); Table table = catalog.loadTable(tableIdent); - Assert.assertEquals("Name must match", "hive.hivedb.tbl", table.name()); + assertThat(table.name()).as("Name must match").isEqualTo("hive.hivedb.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of(DB_NAME, "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assert.assertEquals("Name must match", "hive.hivedb.tbl.snapshots", snapshotsTable.name()); + assertThat(snapshotsTable.name()) + .as("Name must match") + .isEqualTo("hive.hivedb.tbl.snapshots"); } finally { catalog.dropTable(tableIdent); } @@ -887,14 +925,14 @@ private String defaultUri(Namespace namespace) throws TException { public void testUUIDinTableProperties() throws Exception { Schema schema = getTestSchema(); TableIdentifier tableIdentifier = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { catalog.buildTable(tableIdentifier, schema) .withLocation(location) .create(); - Assert.assertNotNull(hmsTableParameters().get(TableProperties.UUID)); + assertThat(hmsTableParameters()).containsKey(TableProperties.UUID); } finally { catalog.dropTable(tableIdentifier); } @@ -904,7 +942,7 @@ public void testUUIDinTableProperties() throws Exception { public void testSnapshotStatsTableProperties() throws Exception { Schema schema = getTestSchema(); TableIdentifier tableIdentifier = TableIdentifier.of(DB_NAME, "tbl"); - String location = temp.newFolder("tbl").toString(); + String location = temp.resolve("tbl").toString(); try { catalog.buildTable(tableIdentifier, schema) @@ -917,10 +955,11 @@ public void testSnapshotStatsTableProperties() throws Exception { // check whether parameters are in expected state Map parameters = hmsTableParameters(); - Assert.assertEquals("0", parameters.get(TableProperties.SNAPSHOT_COUNT)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_SUMMARY)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_ID)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_TIMESTAMP)); + assertThat(parameters).containsEntry(SNAPSHOT_COUNT, "0"); + assertThat(parameters) + .doesNotContainKey(CURRENT_SNAPSHOT_SUMMARY) + .doesNotContainKey(CURRENT_SNAPSHOT_ID) + .doesNotContainKey(CURRENT_SNAPSHOT_TIMESTAMP); // create a snapshot Table icebergTable = catalog.loadTable(tableIdentifier); @@ -934,13 +973,15 @@ public void testSnapshotStatsTableProperties() throws Exception { // check whether parameters are in expected state parameters = hmsTableParameters(); - Assert.assertEquals("1", parameters.get(TableProperties.SNAPSHOT_COUNT)); + assertThat(parameters).containsEntry(SNAPSHOT_COUNT, "1"); String summary = JsonUtil.mapper().writeValueAsString(icebergTable.currentSnapshot().summary()); - Assert.assertEquals(summary, parameters.get(CURRENT_SNAPSHOT_SUMMARY)); + assertThat(parameters).containsEntry(CURRENT_SNAPSHOT_SUMMARY, summary); long snapshotId = icebergTable.currentSnapshot().snapshotId(); - Assert.assertEquals(String.valueOf(snapshotId), parameters.get(CURRENT_SNAPSHOT_ID)); - Assert.assertEquals(String.valueOf(icebergTable.currentSnapshot().timestampMillis()), - parameters.get(CURRENT_SNAPSHOT_TIMESTAMP)); + assertThat(parameters).containsEntry(CURRENT_SNAPSHOT_ID, String.valueOf(snapshotId)); + assertThat(parameters) + .containsEntry( + CURRENT_SNAPSHOT_TIMESTAMP, + String.valueOf(icebergTable.currentSnapshot().timestampMillis())); } finally { catalog.dropTable(tableIdentifier); } @@ -959,10 +1000,10 @@ public void testSetSnapshotSummary() throws Exception { for (int i = 0; i < 100; i++) { summary.put(String.valueOf(i), "value"); } - Assert.assertTrue(JsonUtil.mapper().writeValueAsString(summary).length() < 4000); + assertThat(JsonUtil.mapper().writeValueAsString(summary).length()).isLessThan(4000); Map parameters = Maps.newHashMap(); ops.setSnapshotSummary(parameters, snapshot); - Assert.assertEquals("The snapshot summary must be in parameters", 1, parameters.size()); + assertThat(parameters).as("The snapshot summary must be in parameters").hasSize(1); // create a snapshot summary whose json string size exceeds the limit for (int i = 0; i < 1000; i++) { @@ -970,10 +1011,12 @@ public void testSetSnapshotSummary() throws Exception { } long summarySize = JsonUtil.mapper().writeValueAsString(summary).length(); // the limit has been updated to 4000 instead of the default value(32672) - Assert.assertTrue(summarySize > 4000 && summarySize < 32672); + assertThat(summarySize).isGreaterThan(4000).isLessThan(32672); parameters.remove(CURRENT_SNAPSHOT_SUMMARY); ops.setSnapshotSummary(parameters, snapshot); - Assert.assertEquals("The snapshot summary must not be in parameters due to the size limit", 0, parameters.size()); + assertThat(parameters) + .as("The snapshot summary must not be in parameters due to the size limit") + .isEmpty(); } @Test @@ -991,18 +1034,19 @@ public void testNotExposeTableProperties() { parameters.put(DEFAULT_SORT_ORDER, "sortOrder"); ops.setSnapshotStats(metadata, parameters); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_SUMMARY)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_ID)); - Assert.assertNull(parameters.get(CURRENT_SNAPSHOT_TIMESTAMP)); + assertThat(parameters) + .doesNotContainKey(CURRENT_SNAPSHOT_SUMMARY) + .doesNotContainKey(CURRENT_SNAPSHOT_ID) + .doesNotContainKey(CURRENT_SNAPSHOT_TIMESTAMP); ops.setSchema(metadata, parameters); - Assert.assertNull(parameters.get(CURRENT_SCHEMA)); + assertThat(parameters).doesNotContainKey(CURRENT_SCHEMA); ops.setPartitionSpec(metadata, parameters); - Assert.assertNull(parameters.get(DEFAULT_PARTITION_SPEC)); + assertThat(parameters).doesNotContainKey(DEFAULT_PARTITION_SPEC); ops.setSortOrder(metadata, parameters); - Assert.assertNull(parameters.get(DEFAULT_SORT_ORDER)); + assertThat(parameters).doesNotContainKey(DEFAULT_SORT_ORDER); } @Test @@ -1012,12 +1056,14 @@ public void testSetDefaultPartitionSpec() throws Exception { try { Table table = catalog.buildTable(tableIdent, schema).create(); - Assert.assertFalse("Must not have default partition spec", - hmsTableParameters().containsKey(TableProperties.DEFAULT_PARTITION_SPEC)); + assertThat(hmsTableParameters()) + .as("Must not have default partition spec") + .doesNotContainKey(TableProperties.DEFAULT_PARTITION_SPEC); table.updateSpec().addField(bucket("data", 16)).commit(); - Assert.assertEquals(PartitionSpecParser.toJson(table.spec()), - hmsTableParameters().get(TableProperties.DEFAULT_PARTITION_SPEC)); + assertThat(hmsTableParameters()) + .containsEntry( + TableProperties.DEFAULT_PARTITION_SPEC, PartitionSpecParser.toJson(table.spec())); } finally { catalog.dropTable(tableIdent); } @@ -1031,7 +1077,8 @@ public void testSetCurrentSchema() throws Exception { try { Table table = catalog.buildTable(tableIdent, schema).create(); - Assert.assertEquals(SchemaParser.toJson(table.schema()), hmsTableParameters().get(CURRENT_SCHEMA)); + assertThat(hmsTableParameters()) + .containsEntry(CURRENT_SCHEMA, SchemaParser.toJson(table.schema())); // add many new fields to make the schema json string exceed the limit UpdateSchema updateSchema = table.updateSchema(); @@ -1040,8 +1087,8 @@ public void testSetCurrentSchema() throws Exception { } updateSchema.commit(); - Assert.assertTrue(SchemaParser.toJson(table.schema()).length() > 32672); - Assert.assertNull(hmsTableParameters().get(CURRENT_SCHEMA)); + assertThat(SchemaParser.toJson(table.schema()).length()).isGreaterThan(32672); + assertThat(hmsTableParameters()).doesNotContainKey(CURRENT_SCHEMA); } finally { catalog.dropTable(tableIdent); } @@ -1059,10 +1106,9 @@ public void testConstructorWarehousePathWithEndSlash() { catalogWithSlash.initialize( "hive_catalog", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, wareHousePath + "/")); - Assert.assertEquals( - "Should have trailing slash stripped", - wareHousePath, - catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + assertThat(catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) + .as("Should have trailing slash stripped") + .isEqualTo(wareHousePath); } @Test @@ -1093,28 +1139,23 @@ public void testTablePropsDefinedAtCatalogLevel() { .withProperty("key5", "table-key5") .create(); - Assert.assertEquals( - "Table defaults set for the catalog must be added to the table properties.", - "catalog-default-key1", - table.properties().get("key1")); - Assert.assertEquals( - "Table property must override table default properties set at catalog level.", - "table-key2", - table.properties().get("key2")); - Assert.assertEquals( - "Table property override set at catalog level must override table default" + - " properties set at catalog level and table property specified.", - "catalog-override-key3", - table.properties().get("key3")); - Assert.assertEquals( - "Table override not in table props or defaults should be added to table properties", - "catalog-override-key4", - table.properties().get("key4")); - Assert.assertEquals( - "Table properties without any catalog level default or override should be added to table" + - " properties.", - "table-key5", - table.properties().get("key5")); + assertThat(table.properties()) + .as("Table defaults set for the catalog must be added to the table properties.") + .containsEntry("key1", "catalog-default-key1"); + assertThat(table.properties()) + .as("Table property must override table default properties set at catalog level.") + .containsEntry("key2", "table-key2"); + assertThat(table.properties()) + .as("Table property override set at catalog level must override table default" + + " properties set at catalog level and table property specified.") + .containsEntry("key3", "catalog-override-key3"); + assertThat(table.properties()) + .as("Table override not in table props or defaults should be added to table properties") + .containsEntry("key4", "catalog-override-key4"); + assertThat(table.properties()) + .as("Table properties without any catalog level default or override should be added to table" + + " properties.") + .containsEntry("key5", "table-key5"); } finally { hiveCatalog.dropTable(tableIdent); } @@ -1132,7 +1173,7 @@ public void testDatabaseLocationWithSlashInWarehouseDir() { Database database = catalog.convertToDatabase(Namespace.of("database"), ImmutableMap.of()); - Assert.assertEquals("s3://bucket/database.db", database.getLocationUri()); + assertThat(database.getLocationUri()).isEqualTo("s3://bucket/database.db"); } @Test diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java index 208e95e0e432..3a1b92a12c7f 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java @@ -34,13 +34,14 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.thrift.transport.TTransportException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + public class TestHiveClientPool { private static final String HIVE_SITE_CONTENT = "\n" + @@ -54,13 +55,13 @@ public class TestHiveClientPool { HiveClientPool clients; - @Before + @BeforeEach public void before() { HiveClientPool clientPool = new HiveClientPool(2, new Configuration()); clients = Mockito.spy(clientPool); } - @After + @AfterEach public void after() { clients.close(); clients = null; @@ -74,14 +75,14 @@ public void testConf() { HiveClientPool clientPool = new HiveClientPool(10, conf); HiveConf clientConf = clientPool.hiveConf(); - Assert.assertEquals(conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname), - clientConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - Assert.assertEquals(10, clientPool.poolSize()); + assertThat(clientConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) + .isEqualTo(conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + assertThat(clientPool.poolSize()).isEqualTo(10); // 'hive.metastore.sasl.enabled' should be 'true' as defined in xml - Assert.assertEquals(conf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname), - clientConf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)); - Assert.assertTrue(clientConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL)); + assertThat(clientConf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)) + .isEqualTo(conf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)); + assertThat(clientConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL)).isTrue(); } private HiveConf createHiveConf() { @@ -97,7 +98,7 @@ private HiveConf createHiveConf() { @Test public void testNewClientFailure() { Mockito.doThrow(new RuntimeException("Connection exception")).when(clients).newClient(); - Assertions.assertThatThrownBy(() -> clients.run(Object::toString)) + assertThatThrownBy(() -> clients.run(Object::toString)) .isInstanceOf(RuntimeException.class) .hasMessage("Connection exception"); } @@ -109,7 +110,7 @@ public void testGetTablesFailsForNonReconnectableException() throws Exception { Mockito.doThrow(new MetaException("Another meta exception")) .when(hmsClient) .getTables(Mockito.anyString(), Mockito.anyString()); - Assertions.assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) + assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) .isInstanceOf(MetaException.class) .hasMessage("Another meta exception"); } @@ -129,7 +130,8 @@ public void testConnectionFailureRestoreForMetaException() throws Exception { Mockito.doReturn(databases).when(newClient).getAllDatabases(); // The return is OK when the reconnect method is called. - Assert.assertEquals(databases, clients.run(client -> client.getAllDatabases(), true)); + assertThat((List) clients.run(client -> client.getAllDatabases(), true)) + .isEqualTo(databases); // Verify that the method is called. Mockito.verify(clients).reconnect(hmsClient); @@ -149,7 +151,8 @@ public void testConnectionFailureRestoreForTTransportException() throws Exceptio new Function("concat", "db1", "classname", "root", PrincipalType.USER, 100, FunctionType.JAVA, null)); Mockito.doReturn(response).when(newClient).getAllFunctions(); - Assert.assertEquals(response, clients.run(client -> client.getAllFunctions(), true)); + assertThat((GetAllFunctionsResponse) clients.run(client -> client.getAllFunctions(), true)) + .isEqualTo(response); Mockito.verify(clients).reconnect(hmsClient); Mockito.verify(clients, Mockito.never()).reconnect(newClient); diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java index f4f9a861034b..a9095a89ce42 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java @@ -45,16 +45,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.AdditionalAnswers; import org.mockito.ArgumentCaptor; +import org.mockito.MockedStatic; import org.mockito.invocation.InvocationOnMock; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; @@ -62,6 +63,8 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -86,7 +89,7 @@ public class TestHiveCommitLocks extends HiveTableBaseTest { LockResponse notAcquiredLockResponse = new LockResponse(dummyLockId, LockState.NOT_ACQUIRED); ShowLocksResponse emptyLocks = new ShowLocksResponse(Lists.newArrayList()); - @BeforeClass + @BeforeAll public static void startMetastore() throws Exception { HiveMetastoreTest.startMetastore( ImmutableMap.of(HiveConf.ConfVars.HIVE_TXN_TIMEOUT.varname, "1s")); @@ -113,12 +116,12 @@ public static void startMetastore() throws Exception { spyCachedClientPool = spy(new CachedClientPool(hiveConf, Collections.emptyMap())); when(spyCachedClientPool.clientPool()).thenAnswer(invocation -> spyClientPool); - Assert.assertNotNull(spyClientRef.get()); + assertThat(spyClientRef.get()).isNotNull(); spyClient = spyClientRef.get(); } - @Before + @BeforeEach public void before() throws Exception { Table table = catalog.loadTable(TABLE_IDENTIFIER); ops = (HiveTableOperations) ((HasTableOperations) table).operations(); @@ -135,7 +138,7 @@ public void before() throws Exception { metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); spyOps = spy( @@ -149,7 +152,7 @@ public void before() throws Exception { reset(spyClient); } - @AfterClass + @AfterAll public static void cleanup() { try { spyClientPool.close(); @@ -166,7 +169,7 @@ public void testLockAcquisitionAtFirstTime() throws TException, InterruptedExcep spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -185,7 +188,7 @@ public void testLockAcquisitionAfterRetries() throws TException, InterruptedExce spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -204,7 +207,8 @@ public void testLockAcquisitionAfterFailedNotFoundLock() throws TException, Inte spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + verify(spyClient, times(1)).showLocks(any()); // Make sure HiveLock's findLock method is called + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -229,7 +233,8 @@ public void testLockAcquisitionAfterFailedAndFoundLock() throws TException, Inte spyOps.doCommit(metadataV2, metadataV1); - Assert.assertEquals(1, spyOps.current().schema().columns().size()); // should be 1 again + verify(spyClient, times(1)).showLocks(any()); // Make sure HiveLock's findLock method is called + assertThat(spyOps.current().schema().columns()).hasSize(1); // should be 1 again } @Test @@ -283,7 +288,7 @@ public void testUnLockAfterInterruptedLock() throws TException { doNothing().when(spyClient).unlock(eq(dummyLockId)); doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " + @@ -307,7 +312,7 @@ public void testUnLockAfterInterruptedLockCheck() throws TException { doNothing().when(spyClient).unlock(eq(dummyLockId)); doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " + @@ -331,7 +336,7 @@ public void testUnLockAfterInterruptedGetTable() throws TException { doNothing().when(spyClient).unlock(eq(dummyLockId)); doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage("Interrupted during commit"); @@ -366,7 +371,7 @@ public long getLockid() { public void testLockFailureAtFirstTime() throws TException { doReturn(notAcquiredLockResponse).when(spyClient).lock(any()); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " + @@ -384,7 +389,7 @@ public void testLockFailureAfterRetries() throws TException { .when(spyClient) .checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " + @@ -396,20 +401,46 @@ public void testLockTimeoutAfterRetries() throws TException { doReturn(waitLockResponse).when(spyClient).lock(any()); doReturn(waitLockResponse).when(spyClient).checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith("org.apache.iceberg.hive.LockException") .hasMessageContaining("Timed out after") .hasMessageEndingWith("waiting for lock on hivedb.tbl"); } + @Test + public void testPassThroughThriftExceptionsForHiveVersion_1() + throws TException, InterruptedException { + try (MockedStatic ignore = mockStatic(HiveVersion.class)) { + // default order is 0, meets the requirements of this test + HiveVersion version = mock(HiveVersion.class); + when(HiveVersion.current()).thenReturn(version); + + doReturn(emptyLocks).when(spyClient).showLocks(any()); + doThrow(new TException("Failed to connect to HMS")) + .doReturn(waitLockResponse) + .when(spyClient) + .lock(any()); + doReturn(waitLockResponse) + .doReturn(acquiredLockResponse) + .when(spyClient) + .checkLock(eq(dummyLockId)); + doNothing().when(spyClient).heartbeat(eq(0L), eq(dummyLockId)); + + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + .isInstanceOf(CommitFailedException.class) + .hasMessage( + "org.apache.iceberg.hive.LockException: Failed to find lock for table hivedb.tbl"); + } + } + @Test public void testPassThroughThriftExceptions() throws TException { doReturn(waitLockResponse).when(spyClient).lock(any()); doReturn(waitLockResponse).doThrow(new TException("Test Thrift Exception")) .when(spyClient).checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(RuntimeException.class) .hasMessage( "org.apache.iceberg.hive.LockException: Metastore operation failed for hivedb.tbl"); @@ -424,7 +455,7 @@ public void testPassThroughInterruptions() throws TException { return waitLockResponse; }).when(spyClient).checkLock(eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " + @@ -481,7 +512,7 @@ public void testLockHeartbeatFailureDuringCommit() throws TException, Interrupte .when(spyClient) .heartbeat(eq(0L), eq(dummyLockId)); - Assertions.assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.doCommit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage( "org.apache.iceberg.hive.LockException: " + @@ -520,9 +551,8 @@ public void testNoLockCallsWithNoLock() throws TException { // Make sure that the expected parameter context values are set Map context = contextCaptor.getValue().getProperties(); - Assert.assertEquals(3, context.size()); - Assert.assertEquals( - context.get("expected_parameter_key"), HiveTableOperations.METADATA_LOCATION_PROP); - Assert.assertEquals(context.get("expected_parameter_value"), metadataV2.metadataFileLocation()); + assertThat(context).hasSize(3); + assertThat(HiveTableOperations.METADATA_LOCATION_PROP).isEqualTo(context.get("expected_parameter_key")); + assertThat(metadataV2.metadataFileLocation()).isEqualTo(context.get("expected_parameter_value")); } } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java index 60d4fa2f1bdf..97c1ed15e07e 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java @@ -32,10 +32,10 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Mockito.doAnswer; @@ -61,7 +61,7 @@ public void testSuppressUnlockExceptions() { TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -86,7 +86,7 @@ public void testSuppressUnlockExceptions() { ops.refresh(); // the commit must succeed - Assert.assertEquals(1, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(1); } /** @@ -108,21 +108,22 @@ public void testThriftExceptionUnknownStateIfNotInHistoryFailureOnCommit() throw TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); failCommitAndThrowException(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on fire"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("New metadata files should still exist, new location not in history but" + - " the commit may still succeed", 3, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("New metadata files should still exist, new location not in history but" + + " the commit may still succeed").isEqualTo(3); } /** @@ -143,7 +144,7 @@ public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedE TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -154,10 +155,13 @@ public void testThriftExceptionSuccessOnCommit() throws TException, InterruptedE spyOps.commit(metadataV2, metadataV1); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals("Commit should have been successful and new metadata file should be made", - 3, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Commit should have been successful and new metadata file should be made") + .isEqualTo(3); } /** @@ -179,23 +183,26 @@ public void testThriftExceptionUnknownFailedCommit() throws TException, Interrup TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); failCommitAndThrowException(spyOps); breakFallbackCatalogCommitCheck(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on fire"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals("Client could not determine outcome so new metadata file should also exist", - 3, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Client could not determine outcome so new metadata file should also exist") + .isEqualTo(3); } /** @@ -217,21 +224,23 @@ public void testThriftExceptionsUnknownSuccessCommit() throws TException, Interr TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); commitAndThrowException(ops, spyOps); breakFallbackCatalogCommitCheck(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on fire"); ops.refresh(); - Assert.assertFalse("Current metadata should have changed", ops.current().equals(metadataV2)); - Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current())); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); } /** @@ -267,7 +276,7 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -289,25 +298,29 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted spyOps.commit(metadataV2, metadataV1); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals("The column addition from the concurrent commit should have been successful", - 2, ops.current().schema().columns().size()); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(ops.current().schema().columns()) + .as("The column addition from the concurrent commit should have been successful") + .hasSize(2); } @Test public void testInvalidObjectException() { TableIdentifier badTi = TableIdentifier.of(DB_NAME, "£tbl"); - Assert.assertThrows(String.format("Invalid table name for %s.%s", DB_NAME, "`tbl`"), - ValidationException.class, - () -> catalog.createTable(badTi, schema, PartitionSpec.unpartitioned())); + assertThatThrownBy(() -> catalog.createTable(badTi, schema, PartitionSpec.unpartitioned())) + .isInstanceOf(ValidationException.class) + .hasMessage(String.format("Invalid Hive object for %s.%s", DB_NAME, "£tbl")); } @Test public void testAlreadyExistsException() { - Assert.assertThrows(String.format("Table already exists: %s.%s", DB_NAME, TABLE_NAME), - AlreadyExistsException.class, - () -> catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned())); + assertThatThrownBy( + () -> catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessage(String.format("Table already exists: %s.%s", DB_NAME, TABLE_NAME)); } /** Uses NoLock and pretends we throw an error because of a concurrent commit */ @@ -324,7 +337,7 @@ public void testNoLockThriftExceptionConcurrentCommit() throws TException, Inter TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, ops.current().schema().columns().size()); + assertThat(ops.current().schema().columns()).hasSize(2); HiveTableOperations spyOps = spy(ops); @@ -340,14 +353,59 @@ public void testNoLockThriftExceptionConcurrentCommit() throws TException, Inter .persistTable(any(), anyBoolean(), any()); // Should throw a CommitFailedException so the commit could be retried - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage("The table hivedb.tbl has been modified concurrently"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("New metadata files should not exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("New metadata files should not exist") + .isEqualTo(2); + } + + @Test + public void testLockExceptionUnknownSuccessCommit() throws TException, InterruptedException { + Table table = catalog.loadTable(TABLE_IDENTIFIER); + HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations(); + + TableMetadata metadataV1 = ops.current(); + + table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); + + ops.refresh(); + + TableMetadata metadataV2 = ops.current(); + + assertThat(ops.current().schema().columns()).hasSize(2); + + HiveTableOperations spyOps = spy(ops); + + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + ops.persistTable(tbl, true, location); + throw new LockException("Datacenter on fire"); + }) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .hasMessageContaining("Failed to heartbeat for hive lock while") + .isInstanceOf(CommitStateUnknownException.class); + + ops.refresh(); + + assertThat(ops.current().location()) + .as("Current metadata should have changed to metadata V1") + .isEqualTo(metadataV1.location()); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); } private void commitAndThrowException( diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java index 6af2e0507550..6fc54de9ec2a 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java @@ -48,11 +48,11 @@ import org.apache.thrift.server.TThreadPoolServer; import org.apache.thrift.transport.TServerSocket; import org.apache.thrift.transport.TTransportFactory; -import org.junit.Assert; import static java.nio.file.Files.createTempDirectory; import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; import static java.nio.file.attribute.PosixFilePermissions.fromString; +import static org.assertj.core.api.Assertions.assertThat; public class TestHiveMetastore { @@ -103,7 +103,7 @@ public class TestHiveMetastore { FileSystem fs = Util.getFs(localDirPath, new Configuration()); String errMsg = "Failed to delete " + localDirPath; try { - Assert.assertTrue(errMsg, fs.delete(localDirPath, true)); + assertThat(fs.delete(localDirPath, true)).as(errMsg).isTrue(); } catch (IOException e) { throw new RuntimeException(errMsg, e); } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java index 8c0ef6b5adbb..d839a0c65eb4 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java @@ -31,11 +31,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestHiveSchemaUtil { private static final Schema SIMPLE_ICEBERG_SCHEMA = new Schema( @@ -91,7 +91,7 @@ public class TestHiveSchemaUtil { @Test public void testSimpleSchemaConvertToIcebergSchema() { - Assert.assertEquals(SIMPLE_ICEBERG_SCHEMA.asStruct(), HiveSchemaUtil.convert(SIMPLE_HIVE_SCHEMA).asStruct()); + assertThat(HiveSchemaUtil.convert(SIMPLE_HIVE_SCHEMA).asStruct()).isEqualTo(SIMPLE_ICEBERG_SCHEMA.asStruct()); } @Test @@ -101,24 +101,24 @@ public void testSimpleSchemaConvertToIcebergSchemaFromNameAndTypeLists() { .map(field -> TypeInfoUtils.getTypeInfoFromTypeString(field.getType())) .collect(Collectors.toList()); List comments = SIMPLE_HIVE_SCHEMA.stream().map(FieldSchema::getComment).collect(Collectors.toList()); - Assert.assertEquals(SIMPLE_ICEBERG_SCHEMA.asStruct(), HiveSchemaUtil.convert(names, types, comments).asStruct()); + assertThat(HiveSchemaUtil.convert(names, types, comments).asStruct()).isEqualTo(SIMPLE_ICEBERG_SCHEMA.asStruct()); } @Test public void testComplexSchemaConvertToIcebergSchema() { - Assert.assertEquals(COMPLEX_ICEBERG_SCHEMA.asStruct(), HiveSchemaUtil.convert(COMPLEX_HIVE_SCHEMA).asStruct()); + assertThat(HiveSchemaUtil.convert(COMPLEX_HIVE_SCHEMA).asStruct()).isEqualTo(COMPLEX_ICEBERG_SCHEMA.asStruct()); } @Test public void testSchemaConvertToIcebergSchemaForEveryPrimitiveType() { Schema schemaWithEveryType = HiveSchemaUtil.convert(getSupportedFieldSchemas()); - Assert.assertEquals(getSchemaWithSupportedTypes().asStruct(), schemaWithEveryType.asStruct()); + assertThat(schemaWithEveryType.asStruct()).isEqualTo(getSchemaWithSupportedTypes().asStruct()); } @Test public void testNotSupportedTypes() { for (FieldSchema notSupportedField : getNotSupportedFieldSchemas()) { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> HiveSchemaUtil.convert(Lists.newArrayList(Arrays.asList(notSupportedField)))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Unsupported Hive type"); @@ -127,12 +127,12 @@ public void testNotSupportedTypes() { @Test public void testSimpleSchemaConvertToHiveSchema() { - Assert.assertEquals(SIMPLE_HIVE_SCHEMA, HiveSchemaUtil.convert(SIMPLE_ICEBERG_SCHEMA)); + assertThat(HiveSchemaUtil.convert(SIMPLE_ICEBERG_SCHEMA)).isEqualTo(SIMPLE_HIVE_SCHEMA); } @Test public void testComplexSchemaConvertToHiveSchema() { - Assert.assertEquals(COMPLEX_HIVE_SCHEMA, HiveSchemaUtil.convert(COMPLEX_ICEBERG_SCHEMA)); + assertThat(HiveSchemaUtil.convert(COMPLEX_ICEBERG_SCHEMA)).isEqualTo(COMPLEX_HIVE_SCHEMA); } @Test @@ -166,7 +166,7 @@ public void testConversionWithoutLastComment() { TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.STRING_TYPE_NAME)), Arrays.asList("customer comment")); - Assert.assertEquals(expected.asStruct(), schema.asStruct()); + assertThat(schema.asStruct()).isEqualTo(expected.asStruct()); } protected List getSupportedFieldSchemas() { @@ -216,7 +216,7 @@ protected Schema getSchemaWithSupportedTypes() { */ private void checkConvert(TypeInfo typeInfo, Type type) { // Convert to TypeInfo - Assert.assertEquals(typeInfo, HiveSchemaUtil.convert(type)); + assertThat(HiveSchemaUtil.convert(type)).isEqualTo(typeInfo); // Convert to Type assertEquals(type, HiveSchemaUtil.convert(typeInfo)); } @@ -228,13 +228,13 @@ private void checkConvert(TypeInfo typeInfo, Type type) { */ private void assertEquals(Type expected, Type actual) { if (actual.isPrimitiveType()) { - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } else { List expectedFields = ((Type.NestedType) expected).fields(); List actualFields = ((Type.NestedType) actual).fields(); for (int i = 0; i < expectedFields.size(); ++i) { assertEquals(expectedFields.get(i).type(), actualFields.get(i).type()); - Assert.assertEquals(expectedFields.get(i).name(), actualFields.get(i).name()); + assertThat(actualFields.get(i).name()).isEqualTo(expectedFields.get(i).name()); } } } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java index 3077dc51f6b4..06a10af4371d 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveTableConcurrency.java @@ -29,15 +29,14 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.util.Tasks; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.assertj.core.api.Assertions.assertThat; public class TestHiveTableConcurrency extends HiveTableBaseTest { @@ -75,7 +74,7 @@ public synchronized void testConcurrentFastAppends() { }); icebergTable.refresh(); - Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests(icebergTable.io()).size()); + assertThat(icebergTable.currentSnapshot().allManifests(icebergTable.io())).hasSize(20); } @Test @@ -103,7 +102,7 @@ public synchronized void testConcurrentConnections() throws InterruptedException } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(3, TimeUnit.MINUTES)); - Assert.assertEquals(7, Iterables.size(icebergTable.snapshots())); + assertThat(executorService.awaitTermination(3, TimeUnit.MINUTES)).as("Timeout").isTrue(); + assertThat(icebergTable.currentSnapshot().allManifests(icebergTable.io())).hasSize(20); } } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java index 7311432a54fc..52f12b24ee31 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestLoadHiveCatalog.java @@ -25,23 +25,24 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; public class TestLoadHiveCatalog { private static TestHiveMetastore metastore; - @BeforeClass + @BeforeAll public static void startMetastore() throws Exception { HiveConf hiveConf = new HiveConf(TestLoadHiveCatalog.class); metastore = new TestHiveMetastore(); metastore.start(hiveConf); } - @AfterClass + @AfterAll public static void stopMetastore() throws Exception { if (metastore != null) { metastore.stop(); @@ -68,7 +69,7 @@ public void testCustomCacheKeys() throws Exception { CachedClientPool clientPool1 = (CachedClientPool) hiveCatalog1.clientPool(); CachedClientPool clientPool2 = (CachedClientPool) hiveCatalog2.clientPool(); - Assert.assertSame(clientPool1.clientPool(), clientPool2.clientPool()); + assertThat(clientPool2.clientPool()).isSameAs(clientPool1.clientPool()); Configuration conf1 = new Configuration(metastore.hiveConf()); Configuration conf2 = new Configuration(metastore.hiveConf()); @@ -90,7 +91,7 @@ public void testCustomCacheKeys() throws Exception { conf2); clientPool1 = (CachedClientPool) hiveCatalog1.clientPool(); clientPool2 = (CachedClientPool) hiveCatalog2.clientPool(); - Assert.assertSame(clientPool1.clientPool(), clientPool2.clientPool()); + assertThat(clientPool2.clientPool()).isSameAs(clientPool1.clientPool()); conf2.set("any.key", "any.value2"); hiveCatalog2 = @@ -101,6 +102,6 @@ public void testCustomCacheKeys() throws Exception { ImmutableMap.of(CatalogProperties.CLIENT_POOL_CACHE_KEYS, "conf:any.key"), conf2); clientPool2 = (CachedClientPool) hiveCatalog2.clientPool(); - Assert.assertNotSame(clientPool1.clientPool(), clientPool2.clientPool()); + assertThat(clientPool2.clientPool()).isNotSameAs(clientPool1.clientPool()); } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java index 2bd1f70c9809..42a6b0c77c6d 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java @@ -20,7 +20,6 @@ package org.apache.iceberg.mr; import java.io.IOException; -import java.util.Collections; import java.util.Optional; import java.util.Properties; import org.apache.hadoop.conf.Configuration; @@ -46,6 +45,7 @@ import org.junit.rules.TemporaryFolder; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; public class TestCatalogs { @@ -127,7 +127,7 @@ public void testCreateDropTableToLocation() throws IOException { Assert.assertEquals(properties.getProperty("location"), table.location()); Assert.assertEquals(SchemaParser.toJson(SCHEMA), SchemaParser.toJson(table.schema())); Assert.assertEquals(PartitionSpecParser.toJson(SPEC), PartitionSpecParser.toJson(table.spec())); - Assert.assertEquals(Collections.singletonMap("dummy", "test"), table.properties()); + assertThat(table.properties()).containsEntry("dummy", "test"); Assertions.assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) .isInstanceOf(NullPointerException.class) @@ -179,7 +179,7 @@ public void testCreateDropTableToCatalog() throws IOException { Assert.assertEquals(SchemaParser.toJson(SCHEMA), SchemaParser.toJson(table.schema())); Assert.assertEquals(PartitionSpecParser.toJson(SPEC), PartitionSpecParser.toJson(table.spec())); - Assert.assertEquals(Collections.singletonMap("dummy", "test"), table.properties()); + assertThat(table.properties()).containsEntry("dummy", "test"); Assertions.assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) .isInstanceOf(NullPointerException.class) diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 0baf210340f8..696acd8903b7 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -194,7 +194,8 @@ public void testPartitionEvolution() { testTables.locationForCreateTableSQL(identifier) + " TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(schema) + "', " + - "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "')"); + "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "'" + + ", " + "'format-version'='1')"); shell.executeStatement("ALTER TABLE " + identifier + " SET PARTITION SPEC (month(ts))"); @@ -238,7 +239,8 @@ public void testSetPartitionTransform() { testTables.locationForCreateTableSQL(identifier) + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(schema) + "', " + - "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "')"); + "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "'" + + ", " + "'format-version'='1')"); PartitionSpec spec = PartitionSpec.builderFor(schema) .year("year_field") @@ -326,7 +328,8 @@ public void testSetPartitionTransformSameField() { testTables.locationForCreateTableSQL(identifier) + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(schema) + "', " + - "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "')"); + "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "'" + + ", " + "'format-version'='1')"); PartitionSpec spec = PartitionSpec.builderFor(schema) .truncate("truncate_field", 2) @@ -397,7 +400,8 @@ public void testSetPartitionTransformCaseSensitive() { testTables.locationForCreateTableSQL(identifier) + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(schema) + "', " + - "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "')"); + "'" + InputFormatConfig.CATALOG_NAME + "'='" + testTables.catalogName() + "'" + + ", " + "'format-version'='1')"); PartitionSpec spec = PartitionSpec.builderFor(schema) .truncate("truncate_field", 2) @@ -990,6 +994,9 @@ public void testIcebergAndHmsTableProperties() throws Exception { expectedIcebergProperties.put("EXTERNAL", "TRUE"); expectedIcebergProperties.put("storage_handler", HiveIcebergStorageHandler.class.getName()); expectedIcebergProperties.put(serdeConstants.SERIALIZATION_FORMAT, "1"); + expectedIcebergProperties.put( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0); // Check the HMS table parameters org.apache.hadoop.hive.metastore.api.Table hmsTable = shell.metastore().getTable("default", "customers"); @@ -1006,7 +1013,7 @@ public void testIcebergAndHmsTableProperties() throws Exception { Assert.assertEquals(expectedIcebergProperties, icebergTable.properties()); if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - Assert.assertEquals(12, hmsParams.size()); + Assert.assertEquals(13, hmsParams.size()); Assert.assertEquals("initial_val", hmsParams.get("custom_property")); Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); Assert.assertEquals(HiveIcebergStorageHandler.class.getName(), @@ -1042,7 +1049,8 @@ public void testIcebergAndHmsTableProperties() throws Exception { .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - Assert.assertEquals(15, hmsParams.size()); // 2 newly-added properties + previous_metadata_location prop + // 2 newly-added properties + previous_metadata_location prop + explicit Parquet compression + Assert.assertEquals(16, hmsParams.size()); Assert.assertEquals("true", hmsParams.get("new_prop_1")); Assert.assertEquals("false", hmsParams.get("new_prop_2")); Assert.assertEquals("new_val", hmsParams.get("custom_property")); @@ -1949,7 +1957,8 @@ public void testSnycProperties() throws TException, InterruptedException { // Test create v1 iceberg table and check its properties before and after it upgrades to v2 identifier = TableIdentifier.of("default", "customers_v1"); - shell.executeStatement("CREATE TABLE customers_v1 (id int, name string) Stored by Iceberg stored as ORC"); + shell.executeStatement("CREATE TABLE customers_v1 (id int, name string) Stored by Iceberg stored as ORC " + + "TBLPROPERTIES ('format-version'='1')"); icebergTable = testTables.loadTable(identifier); hmsTable = shell.metastore().getTable("default", "customers_v1"); icePros = icebergTable.properties(); diff --git a/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q b/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q index a70f19316156..85063e2b095a 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q @@ -30,7 +30,7 @@ create external table tbl_target_mixed (a int, ccy string, c bigint) partitioned explain insert into table tbl_target_mixed select * from tbl_src; insert into table tbl_target_mixed select * from tbl_src; select * from tbl_target_mixed order by a, ccy; -select * from default.tbl_target_mixed.partitions order by `partition`; +select `partition` from default.tbl_target_mixed.partitions order by `partition`; select * from default.tbl_target_mixed.files; --1 of 2 partition cols is folded with constant - should still sort diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_insert_overwrite_partition_transforms.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_insert_overwrite_partition_transforms.q index ef9f7af18e83..55f5046a28f4 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_insert_overwrite_partition_transforms.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_insert_overwrite_partition_transforms.q @@ -1,6 +1,6 @@ -- SORT_QUERY_RESULTS -- Mask the totalSize value as it can have slight variability, causing test flakiness ---! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/ +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#/ -- Mask random uuid --! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ -- Mask a random snapshot id diff --git a/iceberg/iceberg-handler/src/test/queries/positive/query_iceberg_metadata_of_partitioned_table.q b/iceberg/iceberg-handler/src/test/queries/positive/query_iceberg_metadata_of_partitioned_table.q index f0c738ef77b7..0dbf8dca36d9 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/query_iceberg_metadata_of_partitioned_table.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/query_iceberg_metadata_of_partitioned_table.q @@ -44,10 +44,10 @@ select summary from default.ice_meta_3.snapshots; select summary['changed-partition-count'] from default.ice_meta_2.snapshots; select partition_spec_id, partition_summaries from default.ice_meta_2.manifests; select partition_spec_id, partition_summaries[1].upper_bound from default.ice_meta_3.manifests; -select * from default.ice_meta_2.partitions; -select * from default.ice_meta_3.partitions; +select `partition` from default.ice_meta_2.partitions; +select `partition` from default.ice_meta_3.partitions; select `partition` from default.ice_meta_2.partitions where `partition`.b='four'; -select * from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday'; +select `partition` from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday'; select partition_summaries from default.ice_meta_3.manifests where partition_summaries[1].upper_bound='Wednesday'; select file_format, spec_id from default.ice_meta_2.data_files; select file_format, spec_id from default.ice_meta_3.data_files; @@ -75,10 +75,10 @@ select summary from default.ice_meta_3.snapshots; select summary['changed-partition-count'] from default.ice_meta_2.snapshots; select partition_spec_id, partition_summaries from default.ice_meta_2.manifests; select partition_spec_id, partition_summaries[1].upper_bound from default.ice_meta_3.manifests; -select * from default.ice_meta_2.partitions; -select * from default.ice_meta_3.partitions; +select `partition` from default.ice_meta_2.partitions; +select `partition` from default.ice_meta_3.partitions; select `partition` from default.ice_meta_2.partitions where `partition`.b='four'; -select * from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday'; +select `partition` from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday'; select partition_summaries from default.ice_meta_3.manifests where partition_summaries[1].upper_bound='Wednesday'; select file_format, spec_id from default.ice_meta_2.data_files; select file_format, spec_id from default.ice_meta_3.data_files; @@ -108,4 +108,4 @@ INSERT INTO partevv VALUES (2, '2022-04-29 16:32:02', '2022-04-29 16:32:02'); ALTER TABLE partevv SET PARTITION SPEC (day(ts)); INSERT INTO partevv VALUES (100, '2022-04-29 16:32:03', '2022-04-29 16:32:03'); -select * from default.partevv.partitions; \ No newline at end of file +select `partition` from default.partevv.partitions; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q b/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q index b424fac02806..ee030edf1a9c 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q @@ -15,7 +15,7 @@ insert into ice1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, --compare hive table with iceberg table show partitions hiveT1; describe default.ice1.partitions; -select * from default.ice1.partitions order by `partition`; +select `partition` from default.ice1.partitions order by `partition`; show partitions ice1 ; @@ -28,16 +28,16 @@ create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part in TBLPROPERTIES("format-version"='2') ; insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2,10, 5); -select * from default.ice2.partitions order by `partition`; +select `partition` from default.ice2.partitions order by `partition`; show partitions ice2; ALTER TABLE ice2 SET PARTITION SPEC (c) ; -select * from default.ice2.partitions order by `partition`; +select `partition` from default.ice2.partitions order by `partition`; show partitions ice2; insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5); -select * from default.ice2.partitions order by `partition`; +select `partition` from default.ice2.partitions order by `partition`; show partitions ice2; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q index 080857ef1380..1532b6191adb 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q @@ -3,7 +3,7 @@ -- Mask neededVirtualColumns due to non-strict order --! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ -- Mask width ---! qt:replace:/(width=58)\d+/$1###/ +--! qt:replace:/(width=55)\d+/$1###/ -- Mask total data size --! qt:replace:/(Data size: 11)\d+/$1####/ diff --git a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_mixed.q b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_mixed.q index 99069bc266ac..fd96b1161156 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_mixed.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_mixed.q @@ -2,7 +2,7 @@ set hive.vectorized.execution.enabled=true; drop table if exists tbl_ice_mixed; create external table tbl_ice_mixed(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into table tbl_ice_mixed values (1, 'one'), (2, 'two'), (3, 'three'), (4, 'four'), (5, 'five'), (111, 'one'), (22, 'two'), (11, 'one'), (44444, 'four'), (44, 'four'); explain vectorization only detail select b, max(a) from tbl_ice_mixed group by b; @@ -29,7 +29,7 @@ create external table tbl_ice_mixed_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into tbl_ice_mixed_all_types values (1.1, 1.2, false, 4, 567890123456789, '6', "col7", cast('2012-10-03 19:58:08' as timestamp), date('1234-09-09'), cast('10.01' as decimal(4,2))); explain vectorization only detail select max(t_float), t_double, t_boolean, t_int, t_bigint, t_binary, t_string, @@ -59,7 +59,7 @@ create external table tbl_ice_mixed_parted ( b string ) partitioned by (p1 string, p2 string) stored by iceberg stored as orc location 'file:/tmp/tbl_ice_mixed_parted' - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into tbl_ice_mixed_parted values (1, 'aa', 'Europe', 'Hungary'), diff --git a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_orc.q b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_orc.q index 50f9419eff61..e8baec320e9f 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_orc.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_orc.q @@ -2,7 +2,7 @@ set hive.vectorized.execution.enabled=true; drop table if exists tbl_ice_orc; create external table tbl_ice_orc(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into table tbl_ice_orc values (1, 'one'), (2, 'two'), (3, 'three'), (4, 'four'), (5, 'five'), (111, 'one'), (22, 'two'), (11, 'one'), (44444, 'four'), (44, 'four'); analyze table tbl_ice_orc compute statistics for columns; @@ -23,7 +23,7 @@ create external table tbl_ice_orc_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into tbl_ice_orc_all_types values (1.1, 1.2, false, 4, 567890123456789, '6', "col7", cast('2012-10-03 19:58:08' as timestamp), date('1234-09-09'), cast('10.01' as decimal(4,2))); explain select max(t_float), t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal from tbl_ice_orc_all_types @@ -39,7 +39,7 @@ create external table tbl_ice_orc_parted ( b string ) partitioned by (p1 string, p2 string) stored by iceberg stored as orc location 'file:/tmp/tbl_ice_orc_parted' - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into tbl_ice_orc_parted values (1, 'aa', 'Europe', 'Hungary'), @@ -89,7 +89,7 @@ create external table tbl_ice_orc_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); -- insert some test data insert into tbl_ice_orc_complex values ( diff --git a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_parquet.q b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_parquet.q index 8c29826f99c1..da2a69e05410 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_parquet.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_read_parquet.q @@ -2,7 +2,7 @@ set hive.vectorized.execution.enabled=true; drop table if exists tbl_ice_parquet; create external table tbl_ice_parquet(a int, b string) stored by iceberg stored as parquet -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into table tbl_ice_parquet values (1, 'one'), (2, 'two'), (3, 'three'), (4, 'four'), (5, 'five'), (111, 'one'), (22, 'two'), (11, 'one'), (44444, 'four'), (44, 'four'); analyze table tbl_ice_parquet compute statistics for columns; @@ -22,7 +22,7 @@ create external table tbl_ice_parquet_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into tbl_ice_parquet_all_types values (1.1, 1.2, false, 4, 567890123456789, '6', "col7", cast('2012-10-03 19:58:08' as timestamp), date('1234-09-09'), cast('10.01' as decimal(4,2))); @@ -39,7 +39,7 @@ create external table tbl_ice_parquet_parted ( b string ) partitioned by (p1 string, p2 string) stored by iceberg stored as parquet location 'file:/tmp/tbl_ice_parquet_parted' - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); insert into tbl_ice_parquet_parted values (1, 'aa', 'Europe', 'Hungary'), @@ -85,7 +85,7 @@ create external table tbl_ice_parquet_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true'); + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1'); -- insert some test data insert into tbl_ice_parquet_complex values ( diff --git a/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out b/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out index 27efe8cb9d46..d089db3abaff 100644 --- a/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out +++ b/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out @@ -26,4 +26,4 @@ POSTHOOK: query: create external table test_merge_source (a int, b string, c int POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@test_merge_source -FAILED: SemanticException [Error 10294]: Attempt to do update or delete using transaction manager that does not support these operations. +FAILED: UnsupportedOperationException Hive doesn't support copy-on-write mode as write.merge.mode. Please set 'write.merge.mode'='merge-on-read' on test_merge_target before running ACID operations on it. diff --git a/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out b/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out index a89f81c1e7b8..616297d09986 100644 --- a/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out @@ -29,4 +29,4 @@ POSTHOOK: query: alter table test_truncate_part_evolution set tblproperties('ext POSTHOOK: type: ALTERTABLE_PROPERTIES POSTHOOK: Input: default@test_truncate_part_evolution POSTHOOK: Output: default@test_truncate_part_evolution -FAILED: SemanticException Truncate conversion to delete is not possible since its not an Iceberg V2 table. Consider converting the table to Iceberg's V2 format specification. +FAILED: UnsupportedOperationException Hive doesn't support copy-on-write mode as write.merge.mode. Please set 'write.merge.mode'='merge-on-read' on test_truncate_part_evolution before running ACID operations on it. diff --git a/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out b/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out index eaa843205ed3..cbacc9179b6c 100644 --- a/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out +++ b/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out @@ -12,4 +12,4 @@ POSTHOOK: query: create external table test_update (id int, value string) stored POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@test_update -FAILED: SemanticException [Error 10294]: Attempt to do update or delete using transaction manager that does not support these operations. +FAILED: UnsupportedOperationException Hive doesn't support copy-on-write mode as write.merge.mode. Please set 'write.merge.mode'='merge-on-read' on test_update before running ACID operations on it. diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out index 9e57537aaa87..c937e4b7674d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_multi_part_table_to_iceberg.q.out @@ -200,11 +200,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 7 numRows 15 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, @@ -465,11 +467,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 7 numRows 15 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, @@ -730,11 +734,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"7\",\"added-records\":\"15\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"7\",\"total-records\":\"15\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"7\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000},{\"name\":\"c\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 7 numRows 15 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out index 7bcde7ebb973..1eae0333557d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_part_table_to_iceberg.q.out @@ -158,11 +158,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 4 numRows 9 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, @@ -372,11 +374,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 4 numRows 9 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, @@ -586,11 +590,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"4\",\"added-records\":\"9\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"4\",\"total-records\":\"9\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"4\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 4 numRows 9 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out index 1c9578068c49..1918abf18e8f 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out @@ -112,11 +112,13 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 1 numRows 5 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 455 schema.name-mapping.default [ { @@ -277,11 +279,13 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 1 numRows 5 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 116 schema.name-mapping.default [ { @@ -442,11 +446,13 @@ Table Parameters: current-snapshot-id #Masked# current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 1 numRows 5 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 schema.name-mapping.default [ { diff --git a/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out b/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out index dbf77956935e..82c2a6c69847 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out @@ -518,6 +518,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -561,6 +562,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out index fb2eeff31c3b..0b9a85d90a22 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table.q.out @@ -29,10 +29,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out index 9d53be2c0cb8..54481cb0cf23 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_as_fileformat.q.out @@ -35,10 +35,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -102,10 +104,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -169,10 +173,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -236,10 +242,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -298,10 +306,12 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} dummy dummy_value + format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out index fb2eeff31c3b..0b9a85d90a22 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg.q.out @@ -29,10 +29,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out index 46de7cdd213a..d7ececb5e6d0 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/create_iceberg_table_stored_by_iceberg_with_serdeproperties.q.out @@ -29,10 +29,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out index 607426a6890a..d0c2aef2bf19 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out @@ -305,6 +305,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 2 numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### serialization.format 1 snapshot-count 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out index d61952f476fb..d25fde849cdc 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out @@ -44,6 +44,7 @@ TBLPROPERTIES ( 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'snapshot-count'='0', 'table_type'='ICEBERG', #### A masked pattern was here #### @@ -125,8 +126,10 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'serialization.format'='1', 'snapshot-count'='0', 'table_type'='ICEBERG', @@ -163,12 +166,17 @@ TBLPROPERTIES ( 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'snapshot-count'='0', 'table_type'='ICEBERG', #### A masked pattern was here #### - 'uuid'='#Masked#') + 'uuid'='#Masked#', + 'write.delete.mode'='merge-on-read', + 'write.merge.mode'='merge-on-read', + 'write.update.mode'='merge-on-read') PREHOOK: query: create table emp (id int) partitioned by (company string) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default @@ -229,8 +237,10 @@ TBLPROPERTIES ( 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"company","transform":"identity","source-id":2,"field-id":1000}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'snapshot-count'='0', 'table_type'='ICEBERG', #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out index 40f2e0d4ddf0..c9ee05ffd2a8 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_all_iceberg.q.out @@ -121,6 +121,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_metadata_tables.q.out b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_metadata_tables.q.out index dd70d1a63e6b..7597296fdd32 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_metadata_tables.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_metadata_tables.q.out @@ -118,10 +118,13 @@ POSTHOOK: type: DESCTABLE POSTHOOK: Input: default@ice_meta_desc record_count bigint Count of records in data files file_count int Count of data files +total_data_file_size_in_bytes bigint Total size in bytes of data files position_delete_record_count bigint Count of records in position delete files position_delete_file_count int Count of position delete files equality_delete_record_count bigint Count of records in equality delete files equality_delete_file_count int Count of equality delete files +last_updated_at timestamp with local time zone Commit time of snapshot that last updated this partition +last_updated_snapshot_id bigint Id of snapshot that last updated this partition PREHOOK: query: describe default.ice_meta_desc.all_manifests PREHOOK: type: DESCTABLE PREHOOK: Input: default@ice_meta_desc @@ -374,10 +377,13 @@ POSTHOOK: Input: default@ice_meta_desc # col_name data_type comment record_count bigint Count of records in data files file_count int Count of data files +total_data_file_size_in_bytes bigint Total size in bytes of data files position_delete_record_count bigint Count of records in position delete files position_delete_file_count int Count of position delete files equality_delete_record_count bigint Count of records in equality delete files equality_delete_file_count int Count of equality delete files +last_updated_at timestamp with local time zone Commit time of snapshot that last updated this partition +last_updated_snapshot_id bigint Id of snapshot that last updated this partition PREHOOK: query: describe formatted default.ice_meta_desc.all_manifests PREHOOK: type: DESCTABLE PREHOOK: Input: default@ice_meta_desc @@ -633,10 +639,13 @@ POSTHOOK: type: DESCTABLE POSTHOOK: Input: default@ice_meta_desc record_count bigint Count of records in data files file_count int Count of data files +total_data_file_size_in_bytes bigint Total size in bytes of data files position_delete_record_count bigint Count of records in position delete files position_delete_file_count int Count of position delete files equality_delete_record_count bigint Count of records in equality delete files equality_delete_file_count int Count of equality delete files +last_updated_at timestamp with local time zone Commit time of snapshot that last updated this partition +last_updated_snapshot_id bigint Id of snapshot that last updated this partition PREHOOK: query: describe extended default.ice_meta_desc.all_manifests PREHOOK: type: DESCTABLE PREHOOK: Input: default@ice_meta_desc diff --git a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out index 84dc9677e2c7..e4d7e3941b34 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/describe_iceberg_table.q.out @@ -77,10 +77,12 @@ Table Parameters: EXTERNAL TRUE bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"i\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"s\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ts\",\"required\":false,\"type\":\"timestamp\"},{\"id\":4,\"name\":\"d\",\"required\":false,\"type\":\"date\"}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -133,10 +135,12 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"year_field\",\"required\":false,\"type\":\"date\"},{\"id\":2,\"name\":\"month_field\",\"required\":false,\"type\":\"date\"},{\"id\":3,\"name\":\"day_field\",\"required\":false,\"type\":\"date\"},{\"id\":4,\"name\":\"hour_field\",\"required\":false,\"type\":\"timestamp\"},{\"id\":5,\"name\":\"truncate_field\",\"required\":false,\"type\":\"string\"},{\"id\":6,\"name\":\"bucket_field\",\"required\":false,\"type\":\"int\"},{\"id\":7,\"name\":\"identity_field\",\"required\":false,\"type\":\"int\"}]} default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"year_field_year\",\"transform\":\"year\",\"source-id\":1,\"field-id\":1000},{\"name\":\"month_field_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1001},{\"name\":\"day_field_day\",\"transform\":\"day\",\"source-id\":3,\"field-id\":1002},{\"name\":\"hour_field_hour\",\"transform\":\"hour\",\"source-id\":4,\"field-id\":1003},{\"name\":\"truncate_field_trunc\",\"transform\":\"truncate[2]\",\"source-id\":5,\"field-id\":1004},{\"name\":\"bucket_field_bucket\",\"transform\":\"bucket[2]\",\"source-id\":6,\"field-id\":1005},{\"name\":\"identity_field\",\"transform\":\"identity\",\"source-id\":7,\"field-id\":1006}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -190,10 +194,12 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"id\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"year_field\",\"required\":false,\"type\":\"date\"},{\"id\":3,\"name\":\"month_field\",\"required\":false,\"type\":\"date\"},{\"id\":4,\"name\":\"day_field\",\"required\":false,\"type\":\"date\"},{\"id\":5,\"name\":\"hour_field\",\"required\":false,\"type\":\"timestamp\"},{\"id\":6,\"name\":\"truncate_field\",\"required\":false,\"type\":\"string\"},{\"id\":7,\"name\":\"bucket_field\",\"required\":false,\"type\":\"int\"},{\"id\":8,\"name\":\"identity_field\",\"required\":false,\"type\":\"int\"}]} default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"year_field_year\",\"transform\":\"year\",\"source-id\":2,\"field-id\":1000},{\"name\":\"month_field_month\",\"transform\":\"month\",\"source-id\":3,\"field-id\":1001},{\"name\":\"day_field_day\",\"transform\":\"day\",\"source-id\":4,\"field-id\":1002},{\"name\":\"hour_field_hour\",\"transform\":\"hour\",\"source-id\":5,\"field-id\":1003},{\"name\":\"truncate_field_trunc\",\"transform\":\"truncate[2]\",\"source-id\":6,\"field-id\":1004},{\"name\":\"bucket_field_bucket\",\"transform\":\"bucket[2]\",\"source-id\":7,\"field-id\":1005},{\"name\":\"identity_field\",\"transform\":\"identity\",\"source-id\":8,\"field-id\":1006}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -235,10 +241,12 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]} default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out index 60a78ea413dd..a05ebf9af733 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out @@ -334,27 +334,27 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 90 PLN 18 100 CZK 12 110 NULL NULL -PREHOOK: query: select * from default.tbl_target_mixed.partitions order by `partition` +PREHOOK: query: select `partition` from default.tbl_target_mixed.partitions order by `partition` PREHOOK: type: QUERY PREHOOK: Input: default@tbl_target_mixed PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.tbl_target_mixed.partitions order by `partition` +POSTHOOK: query: select `partition` from default.tbl_target_mixed.partitions order by `partition` POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_target_mixed POSTHOOK: Output: hdfs://### HDFS PATH ### -{"ccy":"CZK","c_bucket":1} 0 1 1 0 0 0 0 -{"ccy":"CZK","c_bucket":2} 0 1 1 0 0 0 0 -{"ccy":"EUR","c_bucket":0} 0 1 1 0 0 0 0 -{"ccy":"EUR","c_bucket":1} 0 2 1 0 0 0 0 -{"ccy":"EUR","c_bucket":2} 0 3 1 0 0 0 0 -{"ccy":"HUF","c_bucket":1} 0 2 1 0 0 0 0 -{"ccy":"PLN","c_bucket":0} 0 2 1 0 0 0 0 -{"ccy":"PLN","c_bucket":1} 0 1 1 0 0 0 0 -{"ccy":"PLN","c_bucket":2} 0 1 1 0 0 0 0 -{"ccy":"USD","c_bucket":0} 0 2 1 0 0 0 0 -{"ccy":"USD","c_bucket":1} 0 3 1 0 0 0 0 -{"ccy":"USD","c_bucket":2} 0 1 1 0 0 0 0 -{"ccy":null,"c_bucket":null} 0 2 1 0 0 0 0 +{"ccy":"CZK","c_bucket":1} +{"ccy":"CZK","c_bucket":2} +{"ccy":"EUR","c_bucket":0} +{"ccy":"EUR","c_bucket":1} +{"ccy":"EUR","c_bucket":2} +{"ccy":"HUF","c_bucket":1} +{"ccy":"PLN","c_bucket":0} +{"ccy":"PLN","c_bucket":1} +{"ccy":"PLN","c_bucket":2} +{"ccy":"USD","c_bucket":0} +{"ccy":"USD","c_bucket":1} +{"ccy":"USD","c_bucket":2} +{"ccy":null,"c_bucket":null} PREHOOK: query: select * from default.tbl_target_mixed.files PREHOOK: type: QUERY PREHOOK: Input: default@tbl_target_mixed diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out index de2ce3805c87..6ea485c288c2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out @@ -482,10 +482,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 8 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -1619,10 +1621,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"192\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"384\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"country\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000},{\"name\":\"state\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1001}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 384 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -2634,10 +2638,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"2\",\"added-records\":\"192\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"384\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"country\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000},{\"name\":\"state\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1001}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 384 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -3287,10 +3293,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -3940,10 +3948,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -4593,10 +4603,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out index 35d37f3367dd..5b3dbd62b159 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out @@ -601,10 +601,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_year\",\"transform\":\"year\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -1256,10 +1258,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -1911,10 +1915,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_day\",\"transform\":\"day\",\"source-id\":1,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -2337,10 +2343,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_trunc\",\"transform\":\"truncate[2]\",\"source-id\":1,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 8 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -2747,10 +2755,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_bucket\",\"transform\":\"bucket[16]\",\"source-id\":1,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 8 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out index 8f2e8ee412ca..9ed8cd184e1b 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out @@ -177,10 +177,12 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"1\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"7\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":1,\"fields\":[{\"name\":\"b_trunc_2\",\"transform\":\"truncate[2]\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 7 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out index 31e7b1b42b56..51676fe315c2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out @@ -276,10 +276,12 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 4 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -1201,10 +1203,12 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"2\",\"deleted-data-files\":\"2\",\"added-records\":\"20\",\"deleted-records\":\"10\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"30\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"country\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000},{\"name\":\"state\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1001}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 30 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -1652,10 +1656,12 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 10 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -2091,10 +2097,12 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 10 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -2530,10 +2538,12 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 10 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -2969,10 +2979,12 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"6\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 10 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out index 91ff6028e34f..9eede340d70e 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out @@ -597,17 +597,19 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_year\",\"transform\":\"year\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 snapshot-count 5 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG - totalSize #Masked# + totalSize #Masked# #### A masked pattern was here #### uuid #Masked# @@ -1226,17 +1228,19 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_month\",\"transform\":\"month\",\"source-id\":2,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 snapshot-count 5 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG - totalSize #Masked# + totalSize #Masked# #### A masked pattern was here #### uuid #Masked# @@ -1859,17 +1863,19 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"12\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_day\",\"transform\":\"day\",\"source-id\":1,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 24 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 snapshot-count 5 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG - totalSize #Masked# + totalSize #Masked# #### A masked pattern was here #### uuid #Masked# @@ -2285,17 +2291,19 @@ Table Parameters: current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"deleted-data-files\":\"1\",\"added-records\":\"2\",\"deleted-records\":\"2\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"2\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"#Masked#\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"pcol_trunc\",\"transform\":\"truncate[2]\",\"source-id\":1,\"field-id\":1000}]} + format-version 2 iceberg.orc.files.only false metadata_location hdfs://### HDFS PATH ### numFiles #Masked# numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 snapshot-count 3 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG - totalSize #Masked# + totalSize #Masked# #### A masked pattern was here #### uuid #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out index 18c9a4794a6b..738eeac4c11d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_v2_deletes.q.out @@ -30,6 +30,7 @@ TBLPROPERTIES ( 'iceberg.delete.skiprowdata'='false', 'iceberg.orc.files.only'='true', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'serialization.format'='1', 'snapshot-count'='0', 'table_type'='ICEBERG', @@ -141,6 +142,7 @@ TBLPROPERTIES ( 'iceberg.orc.files.only'='true', #### A masked pattern was here #### 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'previous_metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', 'snapshot-count'='4', @@ -287,6 +289,7 @@ TBLPROPERTIES ( 'iceberg.orc.files.only'='true', #### A masked pattern was here #### 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'previous_metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', 'snapshot-count'='4', diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out index 07b9aab6151a..4660af717bfb 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out @@ -5,12 +5,12 @@ POSTHOOK: query: drop table if exists tbl_ice_mixed POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_ice_mixed(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_mixed POSTHOOK: query: create external table tbl_ice_mixed(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_mixed @@ -320,7 +320,7 @@ PREHOOK: query: create external table tbl_ice_mixed_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_mixed_all_types @@ -336,7 +336,7 @@ POSTHOOK: query: create external table tbl_ice_mixed_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_mixed_all_types @@ -747,7 +747,7 @@ PREHOOK: query: create external table tbl_ice_mixed_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE #### A masked pattern was here #### PREHOOK: Output: database:default @@ -757,7 +757,7 @@ POSTHOOK: query: create external table tbl_ice_mixed_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE #### A masked pattern was here #### POSTHOOK: Output: database:default diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out index 599a3586a8b8..2e8c70a64bf0 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out @@ -5,12 +5,12 @@ POSTHOOK: query: drop table if exists tbl_ice_orc POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_ice_orc(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_orc POSTHOOK: query: create external table tbl_ice_orc(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_orc @@ -210,7 +210,7 @@ PREHOOK: query: create external table tbl_ice_orc_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_orc_all_types @@ -226,7 +226,7 @@ POSTHOOK: query: create external table tbl_ice_orc_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_orc_all_types @@ -419,7 +419,7 @@ PREHOOK: query: create external table tbl_ice_orc_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE #### A masked pattern was here #### PREHOOK: Output: database:default @@ -429,7 +429,7 @@ POSTHOOK: query: create external table tbl_ice_orc_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE #### A masked pattern was here #### POSTHOOK: Output: database:default @@ -828,7 +828,7 @@ PREHOOK: query: create external table tbl_ice_orc_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_orc_complex @@ -846,7 +846,7 @@ POSTHOOK: query: create external table tbl_ice_orc_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_orc_complex diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out index d1a0bd694318..76204ac92a2d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out @@ -5,12 +5,12 @@ POSTHOOK: query: drop table if exists tbl_ice_parquet POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_ice_parquet(a int, b string) stored by iceberg stored as parquet -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_parquet POSTHOOK: query: create external table tbl_ice_parquet(a int, b string) stored by iceberg stored as parquet -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_parquet @@ -210,7 +210,7 @@ PREHOOK: query: create external table tbl_ice_parquet_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_parquet_all_types @@ -226,7 +226,7 @@ POSTHOOK: query: create external table tbl_ice_parquet_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_parquet_all_types @@ -419,7 +419,7 @@ PREHOOK: query: create external table tbl_ice_parquet_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE #### A masked pattern was here #### PREHOOK: Output: database:default @@ -429,7 +429,7 @@ POSTHOOK: query: create external table tbl_ice_parquet_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE #### A masked pattern was here #### POSTHOOK: Output: database:default @@ -638,7 +638,7 @@ PREHOOK: query: create external table tbl_ice_parquet_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_parquet_complex @@ -656,7 +656,7 @@ POSTHOOK: query: create external table tbl_ice_parquet_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_parquet_complex diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out index 2a9eee7b0f07..de1b998e8b59 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out @@ -73,6 +73,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 2 numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler @@ -157,6 +158,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 2 numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out index a035a1cc962a..0c28420d5f16 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out @@ -74,6 +74,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 2 numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler @@ -159,6 +160,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 2 numRows 2 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler diff --git a/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out index fbf31438d8d4..136ca8abd5ce 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out @@ -292,31 +292,31 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_3 POSTHOOK: Output: hdfs://### HDFS PATH ### 0 Wednesday -PREHOOK: query: select * from default.ice_meta_2.partitions +PREHOOK: query: select `partition` from default.ice_meta_2.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice_meta_2.partitions +POSTHOOK: query: select `partition` from default.ice_meta_2.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_2 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"b":"four"} 0 1 1 0 0 0 0 -{"b":"three"} 0 3 1 0 0 0 0 -PREHOOK: query: select * from default.ice_meta_3.partitions +{"b":"four"} +{"b":"three"} +PREHOOK: query: select `partition` from default.ice_meta_3.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_3 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice_meta_3.partitions +POSTHOOK: query: select `partition` from default.ice_meta_3.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_3 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"b":"four","c":"Saturday"} 0 3 1 0 0 0 0 -{"b":"four","c":"Sunday"} 0 1 1 0 0 0 0 -{"b":"four","c":"Thursday"} 0 1 1 0 0 0 0 -{"b":"one","c":"Monday"} 0 3 1 0 0 0 0 -{"b":"three","c":"Wednesday"} 0 3 1 0 0 0 0 -{"b":"two","c":"Friday"} 0 2 1 0 0 0 0 -{"b":"two","c":"Tuesday"} 0 2 1 0 0 0 0 +{"b":"four","c":"Saturday"} +{"b":"four","c":"Sunday"} +{"b":"four","c":"Thursday"} +{"b":"one","c":"Monday"} +{"b":"three","c":"Wednesday"} +{"b":"two","c":"Friday"} +{"b":"two","c":"Tuesday"} PREHOOK: query: select `partition` from default.ice_meta_2.partitions where `partition`.b='four' PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_2 @@ -326,15 +326,15 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_2 POSTHOOK: Output: hdfs://### HDFS PATH ### {"b":"four"} -PREHOOK: query: select * from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' +PREHOOK: query: select `partition` from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_3 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' +POSTHOOK: query: select `partition` from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_3 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"b":"two","c":"Tuesday"} 0 2 1 0 0 0 0 +{"b":"two","c":"Tuesday"} PREHOOK: query: select partition_summaries from default.ice_meta_3.manifests where partition_summaries[1].upper_bound='Wednesday' PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_3 @@ -588,31 +588,31 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_3 POSTHOOK: Output: hdfs://### HDFS PATH ### 0 Wednesday -PREHOOK: query: select * from default.ice_meta_2.partitions +PREHOOK: query: select `partition` from default.ice_meta_2.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice_meta_2.partitions +POSTHOOK: query: select `partition` from default.ice_meta_2.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_2 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"b":"four"} 0 1 1 0 0 0 0 -{"b":"three"} 0 3 1 0 0 0 0 -PREHOOK: query: select * from default.ice_meta_3.partitions +{"b":"four"} +{"b":"three"} +PREHOOK: query: select `partition` from default.ice_meta_3.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_3 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice_meta_3.partitions +POSTHOOK: query: select `partition` from default.ice_meta_3.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_3 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"b":"four","c":"Saturday"} 0 3 1 0 0 0 0 -{"b":"four","c":"Sunday"} 0 1 1 0 0 0 0 -{"b":"four","c":"Thursday"} 0 1 1 0 0 0 0 -{"b":"one","c":"Monday"} 0 3 1 0 0 0 0 -{"b":"three","c":"Wednesday"} 0 3 1 0 0 0 0 -{"b":"two","c":"Friday"} 0 2 1 0 0 0 0 -{"b":"two","c":"Tuesday"} 0 2 1 0 0 0 0 +{"b":"four","c":"Saturday"} +{"b":"four","c":"Sunday"} +{"b":"four","c":"Thursday"} +{"b":"one","c":"Monday"} +{"b":"three","c":"Wednesday"} +{"b":"two","c":"Friday"} +{"b":"two","c":"Tuesday"} PREHOOK: query: select `partition` from default.ice_meta_2.partitions where `partition`.b='four' PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_2 @@ -622,15 +622,15 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_2 POSTHOOK: Output: hdfs://### HDFS PATH ### {"b":"four"} -PREHOOK: query: select * from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' +PREHOOK: query: select `partition` from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_3 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' +POSTHOOK: query: select `partition` from default.ice_meta_3.partitions where `partition`.b='two' and `partition`.c='Tuesday' POSTHOOK: type: QUERY POSTHOOK: Input: default@ice_meta_3 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"b":"two","c":"Tuesday"} 0 2 1 0 0 0 0 +{"b":"two","c":"Tuesday"} PREHOOK: query: select partition_summaries from default.ice_meta_3.manifests where partition_summaries[1].upper_bound='Wednesday' PREHOOK: type: QUERY PREHOOK: Input: default@ice_meta_3 @@ -833,14 +833,14 @@ POSTHOOK: query: INSERT INTO partevv VALUES (100, '2022-04-29 16:32:03', '2022-0 POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@partevv -PREHOOK: query: select * from default.partevv.partitions +PREHOOK: query: select `partition` from default.partevv.partitions PREHOOK: type: QUERY PREHOOK: Input: default@partevv PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.partevv.partitions +POSTHOOK: query: select `partition` from default.partevv.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@partevv POSTHOOK: Output: hdfs://### HDFS PATH ### -{"id":1,"ts_day":null} 1 1 1 0 0 0 0 -{"id":2,"ts_day":null} 1 1 1 0 0 0 0 -{"id":null,"ts_day":"2022-04-29"} 2 1 1 0 0 0 0 +{"id":1,"ts_day":null} +{"id":2,"ts_day":null} +{"id":null,"ts_day":"2022-04-29"} diff --git a/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_unpartitioned_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_unpartitioned_table.q.out index b43f204b55ee554f20b53c0f8e5e04edf1df8770..fe8f1ec128f31d79bc6d25fa729b088c15403579 100644 GIT binary patch delta 243 zcmZ3qgK5zYrVV=HjHa9Q#VZ*Z%_i%Mr)=hx6k}vG-)x^y#mH!}dA_ufAfq8f-gxqS y>1mT6#Ij62V}766kQ1-AT1I9gPLRfO5Cg7t^BMD6RXSVvO=hA1&*qo2EExgr6iD9y delta 243 zcmZ3qgK5zYrVV=Hj0T(a#VZ*Z4JPZ0r)=hx6az9h+b2|ksQJ=Hf(ZHf($gkCh-I04 t#{53BAtzpKwT#S0U=`(H#$>40&1cMORS^c##lmkg69srSzno>s2mr8bNM`^5 diff --git a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out index 481336ab61d0..5a93a8a13dd9 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out @@ -110,6 +110,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 10 numRows 21 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -194,6 +195,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 10 numRows 11 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out index 5770b87e7fce..3cc22bb42cf1 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_create_iceberg_table.q.out @@ -33,8 +33,10 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"i","required":false,"type":"int"},{"id":2,"name":"s","required":false,"type":"string"},{"id":3,"name":"ts","required":false,"type":"timestamp"},{"id":4,"name":"d","required":false,"type":"date"}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'serialization.format'='1', 'snapshot-count'='0', 'table_type'='ICEBERG', @@ -87,8 +89,10 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"year_field","required":false,"type":"date"},{"id":2,"name":"month_field","required":false,"type":"date"},{"id":3,"name":"day_field","required":false,"type":"date"},{"id":4,"name":"hour_field","required":false,"type":"timestamp"},{"id":5,"name":"truncate_field","required":false,"type":"string"},{"id":6,"name":"bucket_field","required":false,"type":"int"},{"id":7,"name":"identity_field","required":false,"type":"int"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"year_field_year","transform":"year","source-id":1,"field-id":1000},{"name":"month_field_month","transform":"month","source-id":2,"field-id":1001},{"name":"day_field_day","transform":"day","source-id":3,"field-id":1002},{"name":"hour_field_hour","transform":"hour","source-id":4,"field-id":1003},{"name":"truncate_field_trunc","transform":"truncate[2]","source-id":5,"field-id":1004},{"name":"bucket_field_bucket","transform":"bucket[2]","source-id":6,"field-id":1005},{"name":"identity_field","transform":"identity","source-id":7,"field-id":1006}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'serialization.format'='1', 'snapshot-count'='0', 'table_type'='ICEBERG', @@ -142,8 +146,10 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"year_field","required":false,"type":"date"},{"id":3,"name":"month_field","required":false,"type":"date"},{"id":4,"name":"day_field","required":false,"type":"date"},{"id":5,"name":"hour_field","required":false,"type":"timestamp"},{"id":6,"name":"truncate_field","required":false,"type":"string"},{"id":7,"name":"bucket_field","required":false,"type":"int"},{"id":8,"name":"identity_field","required":false,"type":"int"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"year_field_year","transform":"year","source-id":2,"field-id":1000},{"name":"month_field_month","transform":"month","source-id":3,"field-id":1001},{"name":"day_field_day","transform":"day","source-id":4,"field-id":1002},{"name":"hour_field_hour","transform":"hour","source-id":5,"field-id":1003},{"name":"truncate_field_trunc","transform":"truncate[2]","source-id":6,"field-id":1004},{"name":"bucket_field_bucket","transform":"bucket[2]","source-id":7,"field-id":1005},{"name":"identity_field","transform":"identity","source-id":8,"field-id":1006}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'serialization.format'='1', 'snapshot-count'='0', 'table_type'='ICEBERG', @@ -185,8 +191,10 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"a","required":false,"type":"int"},{"id":2,"name":"b","required":false,"type":"string"}]}', 'default-partition-spec'='{"spec-id":0,"fields":[{"name":"b","transform":"identity","source-id":2,"field-id":1000}]}', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'serialization.format'='1', 'snapshot-count'='0', 'table_type'='ICEBERG', @@ -234,10 +242,12 @@ TBLPROPERTIES ( 'bucketing_version'='2', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"i","required":false,"type":"int"},{"id":2,"name":"s","required":false,"type":"string"}]}', 'current-snapshot-id'='#SnapshotId#', - 'current-snapshot-summary'='{"added-data-files":"1","added-records":"3","added-files-size":"638","changed-partition-count":"1","total-records":"3","total-files-size":"638","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"}', + 'current-snapshot-summary'='{"added-data-files":"1","added-records":"3","added-files-size":"628","changed-partition-count":"1","total-records":"3","total-files-size":"628","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"}', 'current-snapshot-timestamp-ms'='#Masked#', + 'format-version'='2', 'iceberg.orc.files.only'='false', 'metadata_location'='hdfs://### HDFS PATH ###', + 'parquet.compression'='zstd', 'previous_metadata_location'='hdfs://### HDFS PATH ###', 'serialization.format'='1', 'snapshot-count'='1', diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out index cb796611a99d..9b784e9d54d9 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out @@ -370,6 +370,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rewriting.time.window 5min snapshot-count 0 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler @@ -421,6 +422,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd snapshot-count 0 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG @@ -471,6 +473,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rewriting.time.window -1min snapshot-count 0 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler @@ -575,6 +578,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rewriting.time.window 0min snapshot-count 0 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out index ee3eb58f1850..5505f6151c31 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out @@ -61,21 +61,24 @@ partition struct spec_id int record_count bigint Count of records in data files file_count int Count of data files +total_data_file_size_in_bytes bigint Total size in bytes of data files position_delete_record_count bigint Count of records in position delete files position_delete_file_count int Count of position delete files equality_delete_record_count bigint Count of records in equality delete files equality_delete_file_count int Count of equality delete files -PREHOOK: query: select * from default.ice1.partitions order by `partition` +last_updated_at timestamp with local time zone Commit time of snapshot that last updated this partition +last_updated_snapshot_id bigint Id of snapshot that last updated this partition +PREHOOK: query: select `partition` from default.ice1.partitions order by `partition` PREHOOK: type: QUERY PREHOOK: Input: default@ice1 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice1.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice1.partitions order by `partition` POSTHOOK: type: QUERY POSTHOOK: Input: default@ice1 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"d_part":10,"e_part":5} 0 2 1 0 0 0 0 -{"d_part":2,"e_part":5} 0 1 1 0 0 0 0 -{"d_part":3,"e_part":4} 0 2 1 0 0 0 0 +{"d_part":10,"e_part":5} +{"d_part":2,"e_part":5} +{"d_part":3,"e_part":4} PREHOOK: query: show partitions ice1 PREHOOK: type: SHOWPARTITIONS PREHOOK: Input: default@ice1 @@ -123,9 +126,9 @@ Stage-0 Fetch Operator limit:-1 Select Operator [SEL_1] - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"] TableScan [TS_0] - Output:["partition","spec_id","record_count","file_count","position_delete_record_count","position_delete_file_count","equality_delete_record_count","equality_delete_file_count"] + Output:["partition","spec_id","record_count","file_count","total_data_file_size_in_bytes","position_delete_record_count","position_delete_file_count","equality_delete_record_count","equality_delete_file_count","last_updated_at","last_updated_snapshot_id"] PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') @@ -145,17 +148,17 @@ POSTHOOK: query: insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4) POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@ice2 -PREHOOK: query: select * from default.ice2.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice2.partitions order by `partition` PREHOOK: type: QUERY PREHOOK: Input: default@ice2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice2.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice2.partitions order by `partition` POSTHOOK: type: QUERY POSTHOOK: Input: default@ice2 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"d_part":10,"e_part":5} 0 2 1 0 0 0 0 -{"d_part":2,"e_part":5} 0 1 1 0 0 0 0 -{"d_part":3,"e_part":4} 0 2 1 0 0 0 0 +{"d_part":10,"e_part":5} +{"d_part":2,"e_part":5} +{"d_part":3,"e_part":4} PREHOOK: query: show partitions ice2 PREHOOK: type: SHOWPARTITIONS PREHOOK: Input: default@ice2 @@ -172,17 +175,17 @@ POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c) POSTHOOK: type: ALTERTABLE_SETPARTSPEC POSTHOOK: Input: default@ice2 POSTHOOK: Output: default@ice2 -PREHOOK: query: select * from default.ice2.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice2.partitions order by `partition` PREHOOK: type: QUERY PREHOOK: Input: default@ice2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice2.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice2.partitions order by `partition` POSTHOOK: type: QUERY POSTHOOK: Input: default@ice2 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"d_part":10,"e_part":5,"c":null} 0 2 1 0 0 0 0 -{"d_part":2,"e_part":5,"c":null} 0 1 1 0 0 0 0 -{"d_part":3,"e_part":4,"c":null} 0 2 1 0 0 0 0 +{"d_part":10,"e_part":5,"c":null} +{"d_part":2,"e_part":5,"c":null} +{"d_part":3,"e_part":4,"c":null} PREHOOK: query: show partitions ice2 PREHOOK: type: SHOWPARTITIONS PREHOOK: Input: default@ice2 @@ -202,21 +205,21 @@ POSTHOOK: query: insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4) POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@ice2 -PREHOOK: query: select * from default.ice2.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice2.partitions order by `partition` PREHOOK: type: QUERY PREHOOK: Input: default@ice2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice2.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice2.partitions order by `partition` POSTHOOK: type: QUERY POSTHOOK: Input: default@ice2 POSTHOOK: Output: hdfs://### HDFS PATH ### -{"d_part":10,"e_part":5,"c":null} 0 2 1 0 0 0 0 -{"d_part":2,"e_part":5,"c":null} 0 1 1 0 0 0 0 -{"d_part":3,"e_part":4,"c":null} 0 2 1 0 0 0 0 -{"d_part":null,"e_part":null,"c":2} 1 2 1 0 0 0 0 -{"d_part":null,"e_part":null,"c":3} 1 1 1 0 0 0 0 -{"d_part":null,"e_part":null,"c":4} 1 1 1 0 0 0 0 -{"d_part":null,"e_part":null,"c":5} 1 1 1 0 0 0 0 +{"d_part":10,"e_part":5,"c":null} +{"d_part":2,"e_part":5,"c":null} +{"d_part":3,"e_part":4,"c":null} +{"d_part":null,"e_part":null,"c":2} +{"d_part":null,"e_part":null,"c":3} +{"d_part":null,"e_part":null,"c":4} +{"d_part":null,"e_part":null,"c":5} PREHOOK: query: show partitions ice2 PREHOOK: type: SHOWPARTITIONS PREHOOK: Input: default@ice2 diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out index 0dee689afc2e..72702426b2d3 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_force_iceberg_table.q.out @@ -95,11 +95,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"3\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"3\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge false + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 3 numRows 10 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -165,11 +167,13 @@ Table Parameters: current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"10\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge false + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out index 95a97d8074f1..5bef73eb073e 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_iceberg_table.q.out @@ -95,11 +95,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"3\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"3\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge true + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 3 numRows 10 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -165,11 +167,13 @@ Table Parameters: current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"10\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge true + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -233,11 +237,13 @@ Table Parameters: current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge true + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 1 numRows 5 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -303,11 +309,13 @@ Table Parameters: current-snapshot-summary {\"deleted-data-files\":\"1\",\"deleted-records\":\"5\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge true + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 @@ -389,11 +397,13 @@ Table Parameters: current-snapshot-summary {\"deleted-data-files\":\"1\",\"deleted-records\":\"5\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"0\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"0\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge false + format-version 2 iceberg.orc.files.only true #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### rawDataSize 0 serialization.format 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out index 309cbee993ca..8e15da1c4e6a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/truncate_partitioned_iceberg_table.q.out @@ -102,11 +102,13 @@ Table Parameters: current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} external.table.purge true + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 4 numRows 9 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, @@ -200,11 +202,13 @@ Table Parameters: current-snapshot-timestamp-ms #Masked# default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"b\",\"transform\":\"identity\",\"source-id\":2,\"field-id\":1000}]} external.table.purge true + format-version 2 iceberg.orc.files.only false #### A masked pattern was here #### metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### schema.name-mapping.default [ { \"field-id\" : 1, diff --git a/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out index 257ed797965e..c9c2dacb3aaf 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/use_basic_stats_from_iceberg.q.out @@ -165,6 +165,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 @@ -208,6 +209,7 @@ Table Parameters: metadata_location hdfs://### HDFS PATH ### numFiles 0 numRows 0 + parquet.compression zstd rawDataSize 0 serialization.format 1 snapshot-count 0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out index c031030ee561..b868129c6185 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out @@ -957,7 +957,7 @@ Stage-6 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26"] Filter Operator [FIL_46] (rows=2 width=700) predicate:((ss_sold_date_sk = 2451181) and ss_item_sk is not null and ss_customer_sk is not null) - TableScan [TS_2] (rows=2 width=58###) + TableScan [TS_2] (rows=2 width=55###) default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_sold_time_sk","ss_item_sk","ss_customer_sk","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_quantity","ss_wholesale_cost","ss_list_price","ss_sales_price","ss_ext_discount_amt","ss_ext_sales_price","ss_ext_wholesale_cost","ss_ext_list_price","ss_ext_tax","ss_coupon_amt","ss_net_paid","ss_net_paid_inc_tax","ss_net_profit"] <-Select Operator [SEL_49] (rows=5 width=380) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"] diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out index 5b4a0767dc38..71c7149c6c79 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out @@ -5,12 +5,12 @@ POSTHOOK: query: drop table if exists tbl_ice_mixed POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_ice_mixed(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_mixed POSTHOOK: query: create external table tbl_ice_mixed(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_mixed @@ -283,7 +283,7 @@ PREHOOK: query: create external table tbl_ice_mixed_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_mixed_all_types @@ -299,7 +299,7 @@ POSTHOOK: query: create external table tbl_ice_mixed_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_mixed_all_types @@ -634,7 +634,7 @@ PREHOOK: query: create external table tbl_ice_mixed_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE #### A masked pattern was here #### PREHOOK: Output: database:default @@ -644,7 +644,7 @@ POSTHOOK: query: create external table tbl_ice_mixed_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE #### A masked pattern was here #### POSTHOOK: Output: database:default diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out index c3217c7ce68c..ba851000d4c1 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out @@ -5,12 +5,12 @@ POSTHOOK: query: drop table if exists tbl_ice_orc POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_ice_orc(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_orc POSTHOOK: query: create external table tbl_ice_orc(a int, b string) stored by iceberg stored as orc -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_orc @@ -174,7 +174,7 @@ PREHOOK: query: create external table tbl_ice_orc_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_orc_all_types @@ -190,7 +190,7 @@ POSTHOOK: query: create external table tbl_ice_orc_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_orc_all_types @@ -345,7 +345,7 @@ PREHOOK: query: create external table tbl_ice_orc_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE #### A masked pattern was here #### PREHOOK: Output: database:default @@ -355,7 +355,7 @@ POSTHOOK: query: create external table tbl_ice_orc_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE #### A masked pattern was here #### POSTHOOK: Output: database:default @@ -751,7 +751,7 @@ PREHOOK: query: create external table tbl_ice_orc_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_orc_complex @@ -769,7 +769,7 @@ POSTHOOK: query: create external table tbl_ice_orc_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as orc - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_orc_complex diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out index 1daec87e978d..4cf978f18fef 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out @@ -5,12 +5,12 @@ POSTHOOK: query: drop table if exists tbl_ice_parquet POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_ice_parquet(a int, b string) stored by iceberg stored as parquet -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_parquet POSTHOOK: query: create external table tbl_ice_parquet(a int, b string) stored by iceberg stored as parquet -TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') +TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_parquet @@ -174,7 +174,7 @@ PREHOOK: query: create external table tbl_ice_parquet_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_parquet_all_types @@ -190,7 +190,7 @@ POSTHOOK: query: create external table tbl_ice_parquet_all_types ( t_date DATE, t_decimal DECIMAL(4,2) ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_parquet_all_types @@ -345,7 +345,7 @@ PREHOOK: query: create external table tbl_ice_parquet_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE #### A masked pattern was here #### PREHOOK: Output: database:default @@ -355,7 +355,7 @@ POSTHOOK: query: create external table tbl_ice_parquet_parted ( b string ) partitioned by (p1 string, p2 string) #### A masked pattern was here #### - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE #### A masked pattern was here #### POSTHOOK: Output: database:default @@ -563,7 +563,7 @@ PREHOOK: query: create external table tbl_ice_parquet_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@tbl_ice_parquet_complex @@ -581,7 +581,7 @@ POSTHOOK: query: create external table tbl_ice_parquet_complex ( structofarrays struct, birthdays:array>, structofmaps struct, map2:map> ) stored by iceberg stored as parquet - TBLPROPERTIES ('iceberg.decimal64.vectorization'='true') + TBLPROPERTIES ('iceberg.decimal64.vectorization'='true',"format-version"='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tbl_ice_parquet_complex diff --git a/iceberg/patched-iceberg-core/pom.xml b/iceberg/patched-iceberg-core/pom.xml index a2c7b029fb0d..9095ff93a26c 100644 --- a/iceberg/patched-iceberg-core/pom.xml +++ b/iceberg/patched-iceberg-core/pom.xml @@ -19,6 +19,12 @@ .. + + com.github.ben-manes.caffeine + caffeine + ${caffeine.version} + true + org.apache.iceberg iceberg-core @@ -77,6 +83,7 @@ **/HadoopInputFile.class **/SerializableTable.class + **/PartitionsTable.class diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java new file mode 100644 index 000000000000..904270353d69 --- /dev/null +++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.LoadingCache; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import org.apache.iceberg.expressions.ManifestEvaluator; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ParallelIterable; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.iceberg.util.StructLikeMap; + +/** A {@link Table} implementation that exposes a table's partitions as rows. */ +public class PartitionsTable extends BaseMetadataTable { + + private final Schema schema; + + private final boolean unpartitionedTable; + + PartitionsTable(Table table) { + this(table, table.name() + ".partitions"); + } + + PartitionsTable(Table table, String name) { + super(table, name); + + this.schema = + new Schema( + Types.NestedField.required(1, "partition", Partitioning.partitionType(table)), + Types.NestedField.required(4, "spec_id", Types.IntegerType.get()), + Types.NestedField.required( + 2, "record_count", Types.LongType.get(), "Count of records in data files"), + Types.NestedField.required( + 3, "file_count", Types.IntegerType.get(), "Count of data files"), + Types.NestedField.required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), + Types.NestedField.required( + 5, + "position_delete_record_count", + Types.LongType.get(), + "Count of records in position delete files"), + Types.NestedField.required( + 6, + "position_delete_file_count", + Types.IntegerType.get(), + "Count of position delete files"), + Types.NestedField.required( + 7, + "equality_delete_record_count", + Types.LongType.get(), + "Count of records in equality delete files"), + Types.NestedField.required( + 8, + "equality_delete_file_count", + Types.IntegerType.get(), + "Count of equality delete files"), + Types.NestedField.optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + Types.NestedField.optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); + this.unpartitionedTable = Partitioning.partitionType(table).fields().isEmpty(); + } + + @Override + public TableScan newScan() { + return new PartitionsScan(table()); + } + + @Override + public Schema schema() { + if (unpartitionedTable) { + return schema.select( + "record_count", + "file_count", + "total_data_file_size_in_bytes", + "position_delete_record_count", + "position_delete_file_count", + "equality_delete_record_count", + "equality_delete_file_count", + "last_updated_at", + "last_updated_snapshot_id"); + } + return schema; + } + + @Override + MetadataTableType metadataTableType() { + return MetadataTableType.PARTITIONS; + } + + private DataTask task(StaticTableScan scan) { + Iterable partitions = partitions(table(), scan); + if (unpartitionedTable) { + // the table is unpartitioned, partitions contains only the root partition + return StaticDataTask.of( + io().newInputFile(table().operations().current().metadataFileLocation()), + schema(), + scan.schema(), + partitions, + root -> + StaticDataTask.Row.of( + root.dataRecordCount, + root.dataFileCount, + root.dataFileSizeInBytes, + root.posDeleteRecordCount, + root.posDeleteFileCount, + root.eqDeleteRecordCount, + root.eqDeleteFileCount, + root.lastUpdatedAt, + root.lastUpdatedSnapshotId)); + } else { + return StaticDataTask.of( + io().newInputFile(table().operations().current().metadataFileLocation()), + schema(), + scan.schema(), + partitions, + PartitionsTable::convertPartition); + } + } + + private static StaticDataTask.Row convertPartition(Partition partition) { + return StaticDataTask.Row.of( + partition.partitionData, + partition.specId, + partition.dataRecordCount, + partition.dataFileCount, + partition.dataFileSizeInBytes, + partition.posDeleteRecordCount, + partition.posDeleteFileCount, + partition.eqDeleteRecordCount, + partition.eqDeleteFileCount, + partition.lastUpdatedAt, + partition.lastUpdatedSnapshotId); + } + + private static Iterable partitions(Table table, StaticTableScan scan) { + Types.StructType partitionType = Partitioning.partitionType(table); + PartitionMap partitions = new PartitionMap(partitionType); + try (CloseableIterable>> entries = planEntries(scan)) { + for (ManifestEntry> entry : entries) { + Snapshot snapshot = table.snapshot(entry.snapshotId()); + ContentFile file = entry.file(); + StructLike partition = + PartitionUtil.coercePartition( + partitionType, table.specs().get(file.specId()), file.partition()); + partitions.get(partition).update(file, snapshot); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return partitions.all(); + } + + @VisibleForTesting + static CloseableIterable> planEntries(StaticTableScan scan) { + Table table = scan.table(); + + CloseableIterable filteredManifests = + filteredManifests(scan, table, scan.snapshot().allManifests(table.io())); + + Iterable>> tasks = + CloseableIterable.transform(filteredManifests, manifest -> readEntries(manifest, scan)); + + return new ParallelIterable<>(tasks, scan.planExecutor()); + } + + private static CloseableIterable> readEntries( + ManifestFile manifest, StaticTableScan scan) { + Table table = scan.table(); + return CloseableIterable.transform( + ManifestFiles.open(manifest, table.io(), table.specs()) + .caseSensitive(scan.isCaseSensitive()) + .select(scanColumns(manifest.content())) // don't select stats columns + .liveEntries(), + t -> + (ManifestEntry>) + // defensive copy of manifest entry without stats columns + t.copyWithoutStats()); + } + + private static List scanColumns(ManifestContent content) { + switch (content) { + case DATA: + return BaseScan.SCAN_COLUMNS; + case DELETES: + return BaseScan.DELETE_SCAN_COLUMNS; + default: + throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content); + } + } + + private static CloseableIterable filteredManifests( + StaticTableScan scan, Table table, List manifestFilesList) { + CloseableIterable manifestFiles = + CloseableIterable.withNoopClose(manifestFilesList); + + LoadingCache evalCache = + Caffeine.newBuilder() + .build( + specId -> { + PartitionSpec spec = table.specs().get(specId); + PartitionSpec transformedSpec = transformSpec(scan.tableSchema(), spec); + return ManifestEvaluator.forRowFilter( + scan.filter(), transformedSpec, scan.isCaseSensitive()); + }); + + return CloseableIterable.filter( + manifestFiles, manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); + } + + private class PartitionsScan extends StaticTableScan { + PartitionsScan(Table table) { + super( + table, + PartitionsTable.this.schema(), + MetadataTableType.PARTITIONS, + PartitionsTable.this::task); + } + } + + static class PartitionMap { + private final StructLikeMap partitions; + private final Types.StructType keyType; + + PartitionMap(Types.StructType type) { + this.partitions = StructLikeMap.create(type); + this.keyType = type; + } + + Partition get(StructLike key) { + Partition partition = partitions.get(key); + if (partition == null) { + partition = new Partition(key, keyType); + partitions.put(key, partition); + } + return partition; + } + + Iterable all() { + return partitions.values(); + } + } + + static class Partition { + private final PartitionData partitionData; + private int specId; + private long dataRecordCount; + private int dataFileCount; + private long dataFileSizeInBytes; + private long posDeleteRecordCount; + private int posDeleteFileCount; + private long eqDeleteRecordCount; + private int eqDeleteFileCount; + private Long lastUpdatedAt; + private Long lastUpdatedSnapshotId; + + Partition(StructLike key, Types.StructType keyType) { + this.partitionData = toPartitionData(key, keyType); + this.specId = 0; + this.dataRecordCount = 0L; + this.dataFileCount = 0; + this.dataFileSizeInBytes = 0L; + this.posDeleteRecordCount = 0L; + this.posDeleteFileCount = 0; + this.eqDeleteRecordCount = 0L; + this.eqDeleteFileCount = 0; + } + + void update(ContentFile file, Snapshot snapshot) { + if (snapshot != null) { + long snapshotCommitTime = snapshot.timestampMillis() * 1000; + if (this.lastUpdatedAt == null || snapshotCommitTime > this.lastUpdatedAt) { + this.lastUpdatedAt = snapshotCommitTime; + this.lastUpdatedSnapshotId = snapshot.snapshotId(); + } + } + + switch (file.content()) { + case DATA: + this.dataRecordCount += file.recordCount(); + this.dataFileCount += 1; + this.specId = file.specId(); + this.dataFileSizeInBytes += file.fileSizeInBytes(); + break; + case POSITION_DELETES: + this.posDeleteRecordCount = file.recordCount(); + this.posDeleteFileCount += 1; + this.specId = file.specId(); + break; + case EQUALITY_DELETES: + this.eqDeleteRecordCount = file.recordCount(); + this.eqDeleteFileCount += 1; + this.specId = file.specId(); + break; + default: + throw new UnsupportedOperationException( + "Unsupported file content type: " + file.content()); + } + } + + /** Needed because StructProjection is not serializable */ + private PartitionData toPartitionData(StructLike key, Types.StructType keyType) { + PartitionData data = new PartitionData(keyType); + for (int i = 0; i < keyType.fields().size(); i++) { + Object val = key.get(i, keyType.fields().get(i).type().typeId().javaClass()); + if (val != null) { + data.set(i, val); + } + } + return data; + } + } +} diff --git a/iceberg/pom.xml b/iceberg/pom.xml index 4333554ddeb1..8a2f3e0ee4a6 100644 --- a/iceberg/pom.xml +++ b/iceberg/pom.xml @@ -25,19 +25,20 @@ .. . - 1.3.0 + 1.4.2 4.0.3 3.4.4 - 1.11.1 + 1.11.3 5.5.0 3.1.2 2.5.0 9+181-r4173-1 2.5.1 - 3.19.0 - 5.7.2 - 2.9.2 + 3.24.2 + 5.10.0 + 2.10.0 false + 2.9.3 patched-iceberg-api @@ -206,6 +207,11 @@ junit-jupiter-api ${junit.jupiter.version} + + org.junit.jupiter + junit-jupiter-params + ${junit.jupiter.version} + com.esotericsoftware kryo-shaded From cbc5d2d7d650f90882c5c4ad0026a94d2e586acb Mon Sep 17 00:00:00 2001 From: Gergely Farkas Date: Mon, 13 Nov 2023 09:44:38 +0100 Subject: [PATCH 048/179] HIVE-27866: HttpRequestInterceptorBase should not add an empty "Cookie:" header (#4867) (Gergely Farkas reviewed by Laszlo Bodor) --- .../TestThriftHttpCLIServiceFeatures.java | 54 ++++++++++--------- .../hive/jdbc/HttpRequestInterceptorBase.java | 2 +- 2 files changed, 31 insertions(+), 25 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java index c073ace08162..f04db3635f31 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/thrift/TestThriftHttpCLIServiceFeatures.java @@ -19,6 +19,7 @@ package org.apache.hive.service.cli.thrift; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.verify; @@ -112,9 +113,8 @@ public void process(HttpRequest httpRequest, HttpContext httpContext) requestHeaders.add(currHeaders); Header[] headers = httpRequest.getHeaders("Cookie"); - cookieHeader = ""; for (Header h : headers) { - cookieHeader = cookieHeader + h.getName() + ":" + h.getValue(); + cookieHeader = (cookieHeader == null ? "" : cookieHeader) + h.getName() + ":" + h.getValue(); } } @@ -125,6 +125,10 @@ public ArrayList getRequestHeaders() { public String getCookieHeader() { return cookieHeader; } + + public boolean hasCookieHeader() { + return cookieHeader != null; + } } @@ -245,22 +249,10 @@ private static String getHttpUrl() { */ @Test public void testAdditionalHttpHeaders() throws Exception { - TTransport transport; - DefaultHttpClient hClient = new DefaultHttpClient(); - String httpUrl = getHttpUrl(); - Map additionalHeaders = new HashMap(); + Map additionalHeaders = new HashMap<>(); additionalHeaders.put("key1", "value1"); additionalHeaders.put("key2", "value2"); - HttpBasicAuthInterceptorWithLogging authInt = - new HttpBasicAuthInterceptorWithLogging(ThriftCLIServiceTest.USERNAME, ThriftCLIServiceTest.PASSWORD, null, null, - false, additionalHeaders, null); - hClient.addRequestInterceptor(authInt); - transport = new THttpClient(httpUrl, hClient); - TCLIService.Client httpClient = getClient(transport); - - // Create a new open session request object - TOpenSessionReq openReq = new TOpenSessionReq(); - httpClient.OpenSession(openReq).getSessionHandle(); + HttpBasicAuthInterceptorWithLogging authInt = openSessionWithTestInterceptor(additionalHeaders, null); ArrayList headers = authInt.getRequestHeaders(); for (String h : headers) { @@ -275,13 +267,30 @@ public void testAdditionalHttpHeaders() throws Exception { */ @Test public void testCustomCookies() throws Exception { + + // test if request interceptor adds custom cookies + Map additionalHeaders = new HashMap<>(); + Map cookieHeaders = new HashMap<>(); + cookieHeaders.put("key1", "value1"); + cookieHeaders.put("key2", "value2"); + HttpBasicAuthInterceptorWithLogging authInt = openSessionWithTestInterceptor(additionalHeaders, cookieHeaders); + assertTrue(authInt.hasCookieHeader()); + String cookieHeader = authInt.getCookieHeader(); + assertTrue(cookieHeader.contains("key1=value1")); + assertTrue(cookieHeader.contains("key2=value2")); + + // test if request interceptor does not add empty Cookie header + // when no custom cookies are defined + Map emptyCookieHeaders = new HashMap<>(); + HttpBasicAuthInterceptorWithLogging authInt2 = openSessionWithTestInterceptor(additionalHeaders, emptyCookieHeaders); + assertFalse(authInt2.hasCookieHeader()); + } + + public HttpBasicAuthInterceptorWithLogging openSessionWithTestInterceptor( + Map additionalHeaders, Map cookieHeaders) throws Exception { TTransport transport; DefaultHttpClient hClient = new DefaultHttpClient(); String httpUrl = getHttpUrl(); - Map additionalHeaders = new HashMap(); - Map cookieHeaders = new HashMap(); - cookieHeaders.put("key1", "value1"); - cookieHeaders.put("key2", "value2"); HttpBasicAuthInterceptorWithLogging authInt = new HttpBasicAuthInterceptorWithLogging(ThriftCLIServiceTest.USERNAME, ThriftCLIServiceTest.PASSWORD, null, null, false, additionalHeaders, cookieHeaders); @@ -292,12 +301,9 @@ public void testCustomCookies() throws Exception { // Create a new open session request object TOpenSessionReq openReq = new TOpenSessionReq(); httpClient.OpenSession(openReq).getSessionHandle(); - String cookieHeader = authInt.getCookieHeader(); - assertTrue(cookieHeader.contains("key1=value1")); - assertTrue(cookieHeader.contains("key2=value2")); + return authInt; } - /** * This factory creates a mocked HiveAuthorizer class. * Use the mocked class to capture the argument passed to it in the test case. diff --git a/jdbc/src/java/org/apache/hive/jdbc/HttpRequestInterceptorBase.java b/jdbc/src/java/org/apache/hive/jdbc/HttpRequestInterceptorBase.java index 862d299643ee..a31c157354ee 100644 --- a/jdbc/src/java/org/apache/hive/jdbc/HttpRequestInterceptorBase.java +++ b/jdbc/src/java/org/apache/hive/jdbc/HttpRequestInterceptorBase.java @@ -104,7 +104,7 @@ public void process(HttpRequest httpRequest, HttpContext httpContext) } } // Add custom cookies if passed to the jdbc driver - if (customCookies != null) { + if (customCookies != null && !customCookies.isEmpty()) { String cookieHeaderKeyValues = ""; Header cookieHeaderServer = httpRequest.getFirstHeader("Cookie"); if ((cookieHeaderServer != null) && (cookieHeaderServer.getValue() != null)) { From 17525f169b9a08cd715bfb42899e45b7c689c77a Mon Sep 17 00:00:00 2001 From: Soumyakanti Das Date: Tue, 14 Nov 2023 09:36:22 -0800 Subject: [PATCH 049/179] HIVE-27533: Incorrect FOREIGN KEY constraints in SHOW CREATE TABLE (Soumyakanti Das, reviewed by Krisztian Kasa, Attila Turoczy) --- .../hadoop/hive/ql/exec/DDLPlanUtils.java | 67 +++++++++---------- .../llap/show_create_table.q.out | 3 +- 2 files changed, 33 insertions(+), 37 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java index 10feb13045ff..9e226c24e920 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.NotNullConstraint; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.PrimaryKeyInfo; @@ -74,8 +75,10 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Base64; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -696,12 +699,7 @@ public String getAlterTableStmtPrimaryKeyConstraint(PrimaryKeyInfo pr) { command.add(TABLE_NAME, unparseIdentifier(pr.getTableName())); command.add(DATABASE_NAME, unparseIdentifier(pr.getDatabaseName())); command.add(CONSTRAINT_NAME, unparseIdentifier(pr.getConstraintName())); - command.add( - COL_NAMES, - pr.getColNames().values().stream() - .map(x -> unparseIdentifier(x)) - .collect(Collectors.joining(",")) - ); + command.add(COL_NAMES, unparseListOfIdentifiersAndJoin(pr.getColNames().values(), ",")); command.add(ENABLE, pr.getEnable()); command.add(VALIDATE, pr.getValidate()); command.add(RELY, pr.getRely()); @@ -714,23 +712,35 @@ public void getAlterTableStmtForeignKeyConstraint(ForeignKeyInfo fr, List> all = fr.getForeignKeys(); for (String key : all.keySet()) { - for (ForeignKeyInfo.ForeignKeyCol fkc : all.get(key)) { - ST command = new ST(ALTER_TABLE_ADD_FOREIGN_KEY); - command.add(CHILD_TABLE_NAME, unparseIdentifier(fr.getChildTableName())); - command.add(DATABASE_NAME, unparseIdentifier(fr.getChildDatabaseName())); - command.add(CONSTRAINT_NAME, unparseIdentifier(key)); - command.add(CHILD_COL_NAME, unparseIdentifier(fkc.childColName)); - command.add(DATABASE_NAME_FR, unparseIdentifier(fkc.parentDatabaseName)); - command.add(PARENT_TABLE_NAME, unparseIdentifier(fkc.parentTableName)); - command.add(PARENT_COL_NAME, unparseIdentifier(fkc.parentColName)); - command.add(ENABLE, fkc.enable); - command.add(VALIDATE, fkc.validate); - command.add(RELY, fkc.rely); - constraints.add(command.render()); + List fkCols = all.get(key); + Set parentCols = new LinkedHashSet<>(fkCols.size()); + Set childCols = new LinkedHashSet<>(fkCols.size()); + + for (ForeignKeyInfo.ForeignKeyCol fkc: fkCols) { + parentCols.add(fkc.parentColName); + childCols.add(fkc.childColName); } + ST command = new ST(ALTER_TABLE_ADD_FOREIGN_KEY); + command.add(CHILD_TABLE_NAME, unparseIdentifier(fr.getChildTableName())); + command.add(DATABASE_NAME, unparseIdentifier(fr.getChildDatabaseName())); + command.add(CONSTRAINT_NAME, unparseIdentifier(key)); + command.add(CHILD_COL_NAME, unparseListOfIdentifiersAndJoin(childCols, ", ")); + command.add(DATABASE_NAME_FR, unparseIdentifier(fkCols.get(0).parentDatabaseName)); + command.add(PARENT_TABLE_NAME, unparseIdentifier(fkCols.get(0).parentTableName)); + command.add(PARENT_COL_NAME, unparseListOfIdentifiersAndJoin(parentCols, ", ")); + command.add(ENABLE, fkCols.get(0).enable); + command.add(VALIDATE, fkCols.get(0).validate); + command.add(RELY, fkCols.get(0).rely); + constraints.add(command.render()); } } + private String unparseListOfIdentifiersAndJoin(Collection collection, String delimiter) { + return collection.stream() + .map(HiveUtils::unparseIdentifier) + .collect(Collectors.joining(delimiter)); + } + public void getAlterTableStmtUniqueConstraint(UniqueConstraint uq, List constraints) { if (!UniqueConstraint.isNotEmpty(uq)) { return; @@ -745,12 +755,7 @@ public void getAlterTableStmtUniqueConstraint(UniqueConstraint uq, List for (UniqueConstraint.UniqueConstraintCol col : uniqueConstraints.get(key)) { colNames.add(col.colName); } - command.add( - COLUMN_NAME, - colNames.stream() - .map(x -> unparseIdentifier(x)) - .collect(Collectors.joining(",")) - ); + command.add(COLUMN_NAME, unparseListOfIdentifiersAndJoin(colNames, ",")); command.add(ENABLE, uniqueConstraints.get(key).get(0).enable); command.add(VALIDATE, uniqueConstraints.get(key).get(0).validate); command.add(RELY, uniqueConstraints.get(key).get(0).rely); @@ -1026,11 +1031,7 @@ private String getBuckets(Table table) { if (bucketCols.isEmpty()) { return ""; } - List unparsedBucketCols = bucketCols.stream() - .map(x -> unparseIdentifier(x)) - .collect(Collectors.toList()); - - String buckets = "CLUSTERED BY (\n " + StringUtils.join(unparsedBucketCols, ",\n ") + ")\n"; + String buckets = "CLUSTERED BY (\n " + unparseListOfIdentifiersAndJoin(bucketCols, ",\n ") + ")\n"; List sortColumns = table.getSortCols(); if (!sortColumns.isEmpty()) { @@ -1053,17 +1054,13 @@ private String getSkewed(Table table) { return ""; } - List unparsedSkewedColNames = skewedInfo.getSkewedColNames().stream() - .map(x -> unparseIdentifier(x)) - .collect(Collectors.toList()); - List columnValuesList = new ArrayList<>(); for (List columnValues : skewedInfo.getSkewedColValues()) { columnValuesList.add("('" + StringUtils.join(columnValues, "','") + "')"); } String skewed = - "SKEWED BY (" + StringUtils.join(unparsedSkewedColNames, ",") + ")\n" + + "SKEWED BY (" + unparseListOfIdentifiersAndJoin(skewedInfo.getSkewedColNames(), ",") + ")\n" + " ON (" + StringUtils.join(columnValuesList, ",") + ")"; if (table.isStoredAsSubDirectories()) { skewed += "\n STORED AS DIRECTORIES"; diff --git a/ql/src/test/results/clientpositive/llap/show_create_table.q.out b/ql/src/test/results/clientpositive/llap/show_create_table.q.out index 3d92e504989c..9762dc7d9600 100644 --- a/ql/src/test/results/clientpositive/llap/show_create_table.q.out +++ b/ql/src/test/results/clientpositive/llap/show_create_table.q.out @@ -159,8 +159,7 @@ LOCATION TBLPROPERTIES ( 'bucketing_version'='2', #### A masked pattern was here #### -ALTER TABLE `default`.`test3` ADD CONSTRAINT `#### A masked pattern was here ####` FOREIGN KEY (`col1`) REFERENCES `default`.`test`(`col1`) DISABLE NOVALIDATE RELY; -ALTER TABLE `default`.`test3` ADD CONSTRAINT `#### A masked pattern was here ####` FOREIGN KEY (`col2`) REFERENCES `default`.`test`(`col2`) DISABLE NOVALIDATE RELY; +ALTER TABLE `default`.`test3` ADD CONSTRAINT `#### A masked pattern was here ####` FOREIGN KEY (`col1`, `col2`) REFERENCES `default`.`test`(`col1`, `col2`) DISABLE NOVALIDATE RELY; PREHOOK: query: CREATE TABLE TEST_RESERVED ( `member_nr` varchar(8), `plan_nr` varchar(11), From 3ca97725854df10e7cd67514a3219e996453eab1 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Wed, 15 Nov 2023 23:22:56 +0800 Subject: [PATCH 050/179] HIVE-27869: Iceberg: Select from HadoopTable fails at HiveIcebergStorageHandler#canProvideColStats (Butao Zhang, reviewed by Denys Kuzmenko) Closes #4871 --- .../mr/hive/HiveIcebergStorageHandler.java | 6 ++--- .../mr/hive/TestHiveIcebergStatistics.java | 23 +++++++++++++++++++ 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 538c7316779c..537ffd28d411 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -210,7 +210,7 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H private static final int PART_IDX = 0; public static final String COPY_ON_WRITE = "copy-on-write"; public static final String MERGE_ON_READ = "merge-on-read"; - public static final String STATS = "/stats/"; + public static final String STATS = "/stats/snap-"; /** * Function template for producing a custom sort expression function: @@ -538,7 +538,7 @@ private boolean canProvideColStats(Table table, long snapshotId) { try { FileSystem fs = statsPath.getFileSystem(conf); return fs.exists(statsPath); - } catch (IOException e) { + } catch (Exception e) { LOG.warn("Exception when trying to find Iceberg column stats for table:{} , snapshot:{} , " + "statsPath: {} , stack trace: {}", table.name(), table.currentSnapshot(), statsPath, e); } @@ -593,7 +593,7 @@ private Path getColStatsPath(Table table) { } private Path getColStatsPath(Table table, long snapshotId) { - return new Path(table.location() + STATS + table.name() + snapshotId); + return new Path(table.location() + STATS + snapshotId); } private boolean removeColStatsIfExists(Table tbl) throws IOException { diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java index e09cc732e3b3..d31ebcf80d1e 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.List; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.PartitionSpec; @@ -37,6 +38,8 @@ import org.junit.Assume; import org.junit.Test; +import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandler.STATS; + /** * Tests verifying correct statistics generation behaviour on Iceberg tables triggered by: ANALYZE queries, inserts, * CTAS, etc... @@ -260,6 +263,26 @@ public void testMergeStatsWithInsert() { checkColStatMinMaxDistinctValue(identifier.name(), "customer_id", 0, 5, 6, 0); } + @Test + public void testIcebergColStatsPath() throws IOException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + + shell.setHiveSessionValue(HiveConf.ConfVars.HIVESTATSAUTOGATHER.varname, true); + Table table = testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of()); + + String insert = testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, identifier, false); + shell.executeStatement(insert); + + table.refresh(); + Path tblColPath = new Path(table.location() + STATS + table.currentSnapshot().snapshotId()); + // Check that if colPath is created correctly + Assert.assertTrue(tblColPath.getFileSystem(shell.getHiveConf()).exists(tblColPath)); + List result = shell.executeStatement("SELECT * FROM customers"); + HiveIcebergTestUtils.validateData(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, + HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, result)); + } + private void checkColStat(String tableName, String colName, boolean accurate) { List rows = shell.executeStatement("DESCRIBE " + tableName + " " + colName); From 8a4f5ce7275842ff4f1cc917c7a2a48dde71bf4c Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Thu, 16 Nov 2023 10:38:20 +0100 Subject: [PATCH 051/179] HIVE-27828: Iceberg integration: enable copy on write update when split update is on (Krisztian Kasa, reviewed by Denys Kuzmenko) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 7 - .../queries/negative/merge_split_update_off.q | 1 - ...update_iceberg_copy_on_write_partitioned.q | 1 - ...date_iceberg_copy_on_write_unpartitioned.q | 1 - .../tools/metatool/TestHiveMetaTool.java | 2 +- .../org/apache/hive/jdbc/TestJdbcDriver2.java | 1 - .../org/apache/hadoop/hive/ql/Context.java | 12 +- .../hadoop/hive/ql/hooks/WriteEntity.java | 9 - .../hive/ql/metadata/HiveStorageHandler.java | 10 +- .../ql/parse/AcidExportSemanticAnalyzer.java | 4 +- .../hive/ql/parse/DeleteSemanticAnalyzer.java | 166 +++++++ .../hive/ql/parse/MergeSemanticAnalyzer.java | 417 ++++++------------ .../hadoop/hive/ql/parse/ParseUtils.java | 13 +- .../ql/parse/RewriteSemanticAnalyzer.java | 351 ++------------- .../ql/parse/SemanticAnalyzerFactory.java | 15 +- .../ql/parse/SplitMergeSemanticAnalyzer.java | 107 ----- .../ql/parse/SplitUpdateSemanticAnalyzer.java | 208 --------- .../hadoop/hive/ql/parse/StorageFormat.java | 2 +- .../parse/UpdateDeleteSemanticAnalyzer.java | 379 ---------------- .../hive/ql/parse/UpdateSemanticAnalyzer.java | 90 ++++ .../rewrite/CopyOnWriteDeleteRewriter.java | 87 ++++ .../rewrite/CopyOnWriteUpdateRewriter.java | 134 ++++++ .../hive/ql/parse/rewrite/DeleteRewriter.java | 71 +++ .../parse/rewrite/DeleteRewriterFactory.java | 53 +++ .../ql/parse/rewrite/DeleteStatement.java | 39 ++ .../hive/ql/parse/rewrite/MergeRewriter.java | 270 ++++++++++++ .../parse/rewrite/MergeRewriterFactory.java | 64 +++ .../hive/ql/parse/rewrite/MergeStatement.java | 292 ++++++++++++ .../hive/ql/parse/rewrite/Rewriter.java | 26 ++ .../ql/parse/rewrite/RewriterFactory.java | 25 ++ .../ql/parse/rewrite/SplitMergeRewriter.java | 85 ++++ .../ql/parse/rewrite/SplitUpdateRewriter.java | 135 ++++++ .../hive/ql/parse/rewrite/UpdateRewriter.java | 107 +++++ .../parse/rewrite/UpdateRewriterFactory.java | 66 +++ .../ql/parse/rewrite/UpdateStatement.java | 50 +++ .../rewrite/sql/COWWithClauseBuilder.java | 44 ++ .../rewrite/sql/MultiInsertSqlGenerator.java | 204 +++++++++ .../NativeAcidMultiInsertSqlGenerator.java | 60 +++ .../NonNativeAcidMultiInsertSqlGenerator.java | 84 ++++ .../parse/rewrite/sql/SetClausePatcher.java | 48 ++ .../rewrite/sql/SqlGeneratorFactory.java | 48 ++ .../parse/rewrite/sql/WhereClausePatcher.java | 59 +++ ...yzer.java => TestDMLSemanticAnalyzer.java} | 4 +- .../test/queries/clientpositive/sort_acid.q | 16 +- ql/src/test/queries/clientpositive/sqlmerge.q | 3 +- .../clientpositive/llap/sort_acid.q.out | 79 ++-- .../clientpositive/llap/sqlmerge.q.out | 4 +- 47 files changed, 2566 insertions(+), 1387 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/DeleteSemanticAnalyzer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/SplitMergeSemanticAnalyzer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateSemanticAnalyzer.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteStatement.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/Rewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/RewriterFactory.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitMergeRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateStatement.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SetClausePatcher.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/WhereClausePatcher.java rename ql/src/test/org/apache/hadoop/hive/ql/parse/{TestUpdateDeleteSemanticAnalyzer.java => TestDMLSemanticAnalyzer.java} (99%) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 3e3737ad7574..ec463178912f 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3442,13 +3442,6 @@ public static enum ConfVars { /** * @deprecated Use {@link ConfVars#SPLIT_UPDATE} instead. */ - @Deprecated - MERGE_SPLIT_UPDATE("hive.merge.split.update", true, - "If true, SQL Merge statement will handle WHEN MATCHED UPDATE by splitting it into 2\n" + - "branches of a multi-insert, representing delete of existing row and an insert of\n" + - "the new version of the row. Updating bucketing and partitioning columns should\n" + - "only be permitted if this is true.\n" + - "Deprecated, use hive.split.update instead."), OPTIMIZE_ACID_META_COLUMNS("hive.optimize.acid.meta.columns", true, "If true, don't decode Acid metadata columns from storage unless" + " they are needed."), diff --git a/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q b/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q index f7f64e2c2c94..e7dd10fec231 100644 --- a/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q +++ b/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q @@ -1,5 +1,4 @@ set hive.split.update=false; -set hive.merge.split.update=false; drop table if exists test_merge_target; drop table if exists test_merge_source; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q index f5355d5ca9a8..4e9632944426 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_partitioned.q @@ -1,4 +1,3 @@ -set hive.split.update=false; set hive.explain.user=false; drop table if exists tbl_ice; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q index 85b21f9c45fb..5bf9304000a3 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q @@ -1,4 +1,3 @@ -set hive.split.update=false; set hive.explain.user=false; drop table if exists tbl_ice; diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java index 22e3fe0e4762..af4f4bb36196 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java @@ -117,7 +117,7 @@ public void setUp() throws Exception { hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); hiveConf.setBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK, true); - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE, true); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.SPLIT_UPDATE, true); hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSCOLAUTOGATHER, false); hiveConf.setBoolean("mapred.input.dir.recursive", true); TestTxnDbUtil.setConfValues(hiveConf); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java index 4b4c8d1dc729..d6f285498a1e 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java @@ -268,7 +268,6 @@ private void testExceucteMergeCounts(boolean splitUpdateEarly) throws Exception Statement stmt = con.createStatement(); stmt.execute("set " + ConfVars.SPLIT_UPDATE.varname + "=" + splitUpdateEarly); - stmt.execute("set " + ConfVars.MERGE_SPLIT_UPDATE.varname + "=" + splitUpdateEarly); stmt.execute("set " + ConfVars.HIVE_SUPPORT_CONCURRENCY.varname + "=true"); stmt.execute("set " + ConfVars.HIVE_TXN_MANAGER.varname + "=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java index 1b1c6e23b48a..7e8dc5d3b7ab 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -169,6 +169,7 @@ public class Context { private Map insertBranchToNamePrefix = new HashMap<>(); private int deleteBranchOfUpdateIdx = -1; private Operation operation = Operation.OTHER; + private boolean splitUpdate = false; private WmContext wmContext; private boolean isExplainPlan = false; @@ -200,6 +201,11 @@ public void setOperation(Operation operation) { this.operation = operation; } + public void setOperation(Operation operation, boolean splitUpdate) { + setOperation(operation); + this.splitUpdate = splitUpdate; + } + public Operation getOperation() { return operation; } @@ -305,7 +311,7 @@ public DestClausePrefix getDestNamePrefix(ASTNode curNode, QB queryBlock) { case OTHER: return DestClausePrefix.INSERT; case UPDATE: - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE)) { + if (splitUpdate) { return getMergeDestClausePrefix(curNode); } return DestClausePrefix.UPDATE; @@ -429,6 +435,7 @@ protected Context(Context ctx) { this.isUpdateDeleteMerge = ctx.isUpdateDeleteMerge; this.isLoadingMaterializedView = ctx.isLoadingMaterializedView; this.operation = ctx.operation; + this.splitUpdate = ctx.splitUpdate; this.wmContext = ctx.wmContext; this.isExplainPlan = ctx.isExplainPlan; this.statsSource = ctx.statsSource; @@ -1346,8 +1353,7 @@ public List> getParsedTables() { } public boolean isDeleteBranchOfUpdate(String dest) { - if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE) && - !HiveConf.getBoolVar(conf, HiveConf.ConfVars.MERGE_SPLIT_UPDATE)) { + if (!splitUpdate) { return false; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java index fec4fe7be69c..d3dd2c23504f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java @@ -168,15 +168,6 @@ public WriteType getWriteType() { return writeType; } - /** - * Only use this if you are very sure of what you are doing. This is used by the - * {@link org.apache.hadoop.hive.ql.parse.UpdateDeleteSemanticAnalyzer} to reset the types to - * update or delete after rewriting and reparsing the queries. - * @param type new operation type - */ - public void setWriteType(WriteType type) { - setWriteTypeInternal(type); - } private void setWriteTypeInternal(WriteType type) { writeType = type; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 4548b067364a..e2ef165101c6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -49,10 +49,11 @@ import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; +import org.apache.hadoop.hive.ql.parse.DeleteSemanticAnalyzer; import org.apache.hadoop.hive.ql.parse.StorageFormat.StorageHandlerTypes; import org.apache.hadoop.hive.ql.parse.TransformSpec; import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.ColumnStatsDesc; +import org.apache.hadoop.hive.ql.parse.UpdateSemanticAnalyzer; import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; @@ -444,8 +445,8 @@ default List acidVirtualColumns() { } /** - * {@link org.apache.hadoop.hive.ql.parse.UpdateDeleteSemanticAnalyzer} rewrites DELETE/UPDATE queries into INSERT - * queries. + * {@link UpdateSemanticAnalyzer} rewrites UPDATE and + * {@link DeleteSemanticAnalyzer} rewrites DELETE queries into INSERT queries. * - DELETE FROM T WHERE A = 32 is rewritten into * INSERT INTO T SELECT <selectCols> FROM T WHERE A = 32 SORT BY <sortCols>. * - UPDATE T SET B=12 WHERE A = 32 is rewritten into @@ -470,7 +471,8 @@ default FieldSchema getRowId() { } /** - * {@link org.apache.hadoop.hive.ql.parse.UpdateDeleteSemanticAnalyzer} rewrites DELETE/UPDATE queries into INSERT + * {@link UpdateSemanticAnalyzer} rewrites UPDATE and + * {@link DeleteSemanticAnalyzer} rewrites DELETE queries into INSERT * queries. E.g. DELETE FROM T WHERE A = 32 is rewritten into * INSERT INTO T SELECT <selectCols> FROM T WHERE A = 32 SORT BY <sortCols>. * diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java index 8fbb6c2ae60b..92f2a2ded1d9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AcidExportSemanticAnalyzer.java @@ -58,9 +58,9 @@ * acid export statements. It works by rewriting the acid export into insert statements into a temporary table, * and then export it from there. */ -public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer { +public class AcidExportSemanticAnalyzer extends RewriteSemanticAnalyzer { AcidExportSemanticAnalyzer(QueryState queryState) throws SemanticException { - super(queryState); + super(queryState, null); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DeleteSemanticAnalyzer.java new file mode 100644 index 000000000000..ec2568734ec7 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DeleteSemanticAnalyzer.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse; + +import org.apache.hadoop.hive.common.TableName; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.ddl.DDLWork; +import org.apache.hadoop.hive.ql.ddl.table.execute.AlterTableExecuteDesc; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.TaskFactory; +import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.rewrite.DeleteStatement; +import org.apache.hadoop.hive.ql.parse.rewrite.RewriterFactory; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class DeleteSemanticAnalyzer extends RewriteSemanticAnalyzer { + + public DeleteSemanticAnalyzer(QueryState queryState, RewriterFactory rewriterFactory) + throws SemanticException { + super(queryState, rewriterFactory); + } + + @Override + protected ASTNode getTargetTableNode(ASTNode tree) { + // The first child should be the table we are updating / deleting from + ASTNode tabName = (ASTNode) tree.getChild(0); + assert tabName.getToken().getType() == HiveParser.TOK_TABNAME : + "Expected tablename as first child of " + Context.Operation.DELETE + " but found " + tabName.getName(); + return tabName; + } + + @Override + protected void analyze(ASTNode tree, Table table, ASTNode tableName) throws SemanticException { + List children = tree.getChildren(); + + ASTNode where = null; + if (children.size() > 1) { + where = (ASTNode) children.get(1); + assert where.getToken().getType() == HiveParser.TOK_WHERE : + "Expected where clause, but found " + where.getName(); + } + + boolean shouldTruncate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_REPLACE_DELETE_WITH_TRUNCATE) + && where == null; + if (shouldTruncate) { + genTruncatePlan(table, tableName); + return; + } else if (tryMetadataUpdate(table, tableName, where)) { + return; + } + + rewriteAndAnalyze(new DeleteStatement(table, where), null); + + updateOutputs(table); + } + + private void genTruncatePlan(Table table, ASTNode tabNameNode) throws SemanticException { + String rewrittenQueryStr = "truncate " + getFullTableNameForSQL(tabNameNode); + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, rewrittenQueryStr); + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + + BaseSemanticAnalyzer truncate = SemanticAnalyzerFactory.get(queryState, rewrittenTree); + // Note: this will overwrite this.ctx with rewrittenCtx + rewrittenCtx.setEnableUnparse(false); + truncate.analyze(rewrittenTree, rewrittenCtx); + + rootTasks = truncate.getRootTasks(); + outputs = truncate.getOutputs(); + updateOutputs(table); + } + + private boolean tryMetadataUpdate(Table table, ASTNode tabNameNode, ASTNode whereNode) + throws SemanticException { + // A feature flag on Hive to perform metadata delete on the source table. + if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_DELETE)) { + return false; + } + if (whereNode == null || table.getStorageHandler() == null) { + return false; + } + TableName tableName = getQualifiedTableName(tabNameNode); + String whereClause = ctx.getTokenRewriteStream().toString( + whereNode.getChild(0).getTokenStartIndex(), whereNode.getChild(0).getTokenStopIndex()); + StringBuilder sb = new StringBuilder("select * from ").append(getFullTableNameForSQL(tabNameNode)) + .append(" where ").append(whereClause); + Context context = new Context(conf); + ASTNode rewrittenTree; + try { + rewrittenTree = ParseUtils.parse(sb.toString(), context); + } catch (ParseException pe) { + throw new SemanticException(pe); + } + BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, rewrittenTree); + sem.analyze(rewrittenTree, context); + + SearchArgument sarg = convertFilterExpressionInTS(table, sem); + if (sarg == null) { + return false; + } + if (!table.getStorageHandler().canPerformMetadataDelete(table, tableName.getTableMetaRef(), sarg)) { + return false; + } + + DDLWork ddlWork = createDDLWorkOfMetadataUpdate(tableName, sarg); + rootTasks = Collections.singletonList(TaskFactory.get(ddlWork)); + inputs = sem.getInputs(); + outputs = sem.getOutputs(); + updateOutputs(table); + return true; + } + + private SearchArgument convertFilterExpressionInTS(Table table, BaseSemanticAnalyzer sem) { + Map topOps = sem.getParseContext().getTopOps(); + if (!topOps.containsKey(table.getTableName())) { + return null; + } + ExprNodeGenericFuncDesc hiveFilter = topOps.get(table.getTableName()).getConf().getFilterExpr(); + if (hiveFilter == null) { + return null; + } + ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(ctx.getConf(), hiveFilter); + if (result.isPartial()) { + return null; + } + return result.getSearchArgument(); + } + + private DDLWork createDDLWorkOfMetadataUpdate(TableName tableName, SearchArgument sarg) throws SemanticException { + AlterTableExecuteSpec.DeleteMetadataSpec deleteMetadataSpec = + new AlterTableExecuteSpec.DeleteMetadataSpec(tableName.getTableMetaRef(), sarg); + AlterTableExecuteSpec executeSpec = + new AlterTableExecuteSpec<>(AlterTableExecuteSpec.ExecuteOperationType.DELETE_METADATA, deleteMetadataSpec); + AlterTableExecuteDesc desc = new AlterTableExecuteDesc(tableName, null, executeSpec); + return new DDLWork(getInputs(), getOutputs(), desc); + } + + @Override + protected boolean enableColumnStatsCollecting() { + return false; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java index 1a3d8bdeec87..163dda00d2e5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java @@ -17,43 +17,40 @@ */ package org.apache.hadoop.hive.ql.parse; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.Path; +import org.antlr.runtime.TokenRewriteStream; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.parse.ParseUtils.ReparseResult; -import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.parse.rewrite.MergeStatement; +import org.apache.hadoop.hive.ql.parse.rewrite.RewriterFactory; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + /** * A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles * merge statements. It works by rewriting the updates and deletes into insert statements (since * they are actually inserts) and then doing some patch up to make them work as merges instead. */ -public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer { +public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer { + private int numWhenMatchedUpdateClauses; private int numWhenMatchedDeleteClauses; + private IdentifierQuoter quotedIdentifierHelper; - MergeSemanticAnalyzer(QueryState queryState) throws SemanticException { - super(queryState); + MergeSemanticAnalyzer(QueryState queryState, RewriterFactory rewriterFactory) + throws SemanticException { + super(queryState, rewriterFactory); } @Override @@ -61,15 +58,6 @@ protected ASTNode getTargetTableNode(ASTNode tree) { return (ASTNode)tree.getChild(0); } - @Override - public void analyze(ASTNode tree, Table targetTable, ASTNode tableNameNode) throws SemanticException { - boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable, true); - if (nonNativeAcid) { - throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); - } - analyzeMerge(tree, targetTable, tableNameNode); - } - /** * Here we take a Merge statement AST and generate a semantically equivalent multi-insert * statement to execute. Each Insert leg represents a single WHEN clause. As much as possible, @@ -82,8 +70,10 @@ public void analyze(ASTNode tree, Table targetTable, ASTNode tableNameNode) thro * * @throws SemanticException */ - protected void analyzeMerge(ASTNode tree, Table targetTable, ASTNode targetNameNode) - throws SemanticException { + @Override + public void analyze(ASTNode tree, Table targetTable, ASTNode targetNameNode) throws SemanticException { + quotedIdentifierHelper = new IdentifierQuoter(ctx.getTokenRewriteStream()); + /* * See org.apache.hadoop.hive.ql.parse.TestMergeStatement for some examples of the merge AST For example, given: @@ -125,12 +115,14 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) "MergeSemanticAnalyzer"); } - ctx.setOperation(Context.Operation.MERGE); ASTNode source = (ASTNode)tree.getChild(1); - String targetName = getSimpleTableName(targetNameNode); + String targetAlias = getSimpleTableName(targetNameNode); String sourceName = getSimpleTableName(source); ASTNode onClause = (ASTNode) tree.getChild(2); String onClauseAsText = getMatchedText(onClause); + MergeStatement.MergeStatementBuilder mergeStatementBuilder = MergeStatement + .withTarget(targetTable, getFullTableNameForSQL(targetNameNode), targetAlias) + .onClauseAsText(onClauseAsText); int whenClauseBegins = 3; boolean hasHint = false; @@ -142,38 +134,14 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) } List whenClauses = findWhenClauses(tree, whenClauseBegins); - StringBuilder rewrittenQueryStr = createRewrittenQueryStrBuilder(); - rewrittenQueryStr.append("(SELECT "); - String subQueryAlias = isAliased(targetNameNode) ? targetName : targetTable.getTTable().getTableName(); - ColumnAppender columnAppender = getColumnAppender(subQueryAlias, StringUtils.EMPTY); - columnAppender.appendAcidSelectColumns(rewrittenQueryStr, Context.Operation.MERGE); - - rewrittenQueryStr.deleteCharAt(rewrittenQueryStr.length() - 1); // remove last ',' - addColsToSelect(targetTable.getCols(), rewrittenQueryStr); - addColsToSelect(targetTable.getPartCols(), rewrittenQueryStr); - rewrittenQueryStr.append(" FROM ").append(getFullTableNameForSQL(targetNameNode)).append(") "); - rewrittenQueryStr.append(subQueryAlias); - rewrittenQueryStr.append('\n'); - - rewrittenQueryStr.append(INDENT).append(chooseJoinType(whenClauses)).append("\n"); - if (source.getType() == HiveParser.TOK_SUBQUERY) { - //this includes the mandatory alias - rewrittenQueryStr.append(INDENT).append(getMatchedText(source)); - } else { - rewrittenQueryStr.append(INDENT).append(getFullTableNameForSQL(source)); - if (isAliased(source)) { - rewrittenQueryStr.append(" ").append(sourceName); - } - } - rewrittenQueryStr.append('\n'); - rewrittenQueryStr.append(INDENT).append("ON ").append(onClauseAsText).append('\n'); + mergeStatementBuilder.sourceAlias(getSourceAlias(source, sourceName)); // Add the hint if any - String hintStr = null; if (hasHint) { - hintStr = " /*+ " + qHint.getText() + " */ "; + mergeStatementBuilder.hintStr(String.format(" /*+ %s */ ", qHint.getText())); } - /** + + /* * We allow at most 2 WHEN MATCHED clause, in which case 1 must be Update the other Delete * If we have both update and delete, the 1st one (in SQL code) must have "AND " * so that the 2nd can ensure not to process the same rows. @@ -183,31 +151,27 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) int numInsertClauses = 0; numWhenMatchedUpdateClauses = 0; numWhenMatchedDeleteClauses = 0; - boolean hintProcessed = false; for (ASTNode whenClause : whenClauses) { switch (getWhenClauseOperation(whenClause).getType()) { case HiveParser.TOK_INSERT: numInsertClauses++; - handleInsert(whenClause, rewrittenQueryStr, targetNameNode, onClause, - targetTable, targetName, onClauseAsText, hintProcessed ? null : hintStr); - hintProcessed = true; + mergeStatementBuilder.addWhenClause( + handleInsert(whenClause, onClause, targetTable, targetAlias, onClauseAsText)); break; case HiveParser.TOK_UPDATE: numWhenMatchedUpdateClauses++; - String s = handleUpdate(whenClause, rewrittenQueryStr, targetNameNode, - onClauseAsText, targetTable, extraPredicate, hintProcessed ? null : hintStr, columnAppender); - hintProcessed = true; + MergeStatement.UpdateClause updateClause = handleUpdate(whenClause, targetTable, extraPredicate); + mergeStatementBuilder.addWhenClause(updateClause); if (numWhenMatchedUpdateClauses + numWhenMatchedDeleteClauses == 1) { - extraPredicate = s; //i.e. it's the 1st WHEN MATCHED + extraPredicate = updateClause.getExtraPredicate(); //i.e. it's the 1st WHEN MATCHED } break; case HiveParser.TOK_DELETE: numWhenMatchedDeleteClauses++; - String s1 = handleDelete(whenClause, rewrittenQueryStr, - onClauseAsText, extraPredicate, hintProcessed ? null : hintStr, columnAppender); - hintProcessed = true; + MergeStatement.DeleteClause deleteClause = handleDelete(whenClause, extraPredicate); + mergeStatementBuilder.addWhenClause(deleteClause); if (numWhenMatchedUpdateClauses + numWhenMatchedDeleteClauses == 1) { - extraPredicate = s1; //i.e. it's the 1st WHEN MATCHED + extraPredicate = deleteClause.getExtraPredicate(); //i.e. it's the 1st WHEN MATCHED } break; default: @@ -226,141 +190,35 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) throw new SemanticException(ErrorMsg.MERGE_PREDIACTE_REQUIRED, ctx.getCmd()); } - boolean validating = handleCardinalityViolation(rewrittenQueryStr, targetNameNode, onClauseAsText, targetTable, - numWhenMatchedDeleteClauses == 0 && numWhenMatchedUpdateClauses == 0, columnAppender); - ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, rewrittenQueryStr); - Context rewrittenCtx = rr.rewrittenCtx; - ASTNode rewrittenTree = rr.rewrittenTree; - rewrittenCtx.setOperation(Context.Operation.MERGE); - - //set dest name mapping on new context; 1st child is TOK_FROM - int insClauseIdx = 1; - for (int whenClauseIdx = 0; - insClauseIdx < rewrittenTree.getChildCount() - (validating ? 1 : 0/*skip cardinality violation clause*/); - whenClauseIdx++) { - //we've added Insert clauses in order or WHEN items in whenClauses - switch (getWhenClauseOperation(whenClauses.get(whenClauseIdx)).getType()) { - case HiveParser.TOK_INSERT: - rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.INSERT); - ++insClauseIdx; - break; - case HiveParser.TOK_UPDATE: - insClauseIdx += addDestNamePrefixOfUpdate(insClauseIdx, rewrittenCtx); - break; - case HiveParser.TOK_DELETE: - rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.DELETE); - ++insClauseIdx; - break; - default: - assert false; - } - } - if (validating) { - //here means the last branch of the multi-insert is Cardinality Validation - rewrittenCtx.addDestNamePrefix(rewrittenTree.getChildCount() - 1, Context.DestClausePrefix.INSERT); - } + String subQueryAlias = isAliased(targetNameNode) ? targetAlias : targetTable.getTTable().getTableName(); - analyzeRewrittenTree(rewrittenTree, rewrittenCtx); - updateOutputs(targetTable); - } + rewriteAndAnalyze(mergeStatementBuilder.build(), subQueryAlias); - /** - * This sets the destination name prefix for update clause. - * @param insClauseIdx index of insert clause in the rewritten multi-insert represents the merge update clause. - * @param rewrittenCtx the {@link Context} stores the prefixes - * @return the number of prefixes set. - */ - protected int addDestNamePrefixOfUpdate(int insClauseIdx, Context rewrittenCtx) { - rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.UPDATE); - return 1; - } - - /** - * If there is no WHEN NOT MATCHED THEN INSERT, we don't outer join. - */ - private String chooseJoinType(List whenClauses) { - for (ASTNode whenClause : whenClauses) { - if (getWhenClauseOperation(whenClause).getType() == HiveParser.TOK_INSERT) { - return "RIGHT OUTER JOIN"; - } - } - return "INNER JOIN"; + updateOutputs(targetTable); } - /** - * Per SQL Spec ISO/IEC 9075-2:2011(E) Section 14.2 under "General Rules" Item 6/Subitem a/Subitem 2/Subitem B, - * an error should be raised if > 1 row of "source" matches the same row in "target". - * This should not affect the runtime of the query as it's running in parallel with other - * branches of the multi-insert. It won't actually write any data to merge_tmp_table since the - * cardinality_violation() UDF throws an error whenever it's called killing the query - * @return true if another Insert clause was added - */ - private boolean handleCardinalityViolation(StringBuilder rewrittenQueryStr, ASTNode target, - String onClauseAsString, Table targetTable, boolean onlyHaveWhenNotMatchedClause, ColumnAppender columnAppender) - throws SemanticException { - if (!conf.getBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK)) { - LOG.info("Merge statement cardinality violation check is disabled: " + - HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK.varname); - return false; - } - if (onlyHaveWhenNotMatchedClause) { - //if no update or delete in Merge, there is no need to to do cardinality check - return false; - } - //this is a tmp table and thus Session scoped and acid requires SQL statement to be serial in a - // given session, i.e. the name can be fixed across all invocations - String tableName = "merge_tmp_table"; - List sortKeys = columnAppender.getSortKeys(); - rewrittenQueryStr.append("INSERT INTO ").append(tableName) - .append("\n SELECT cardinality_violation(") - .append(StringUtils.join(sortKeys, ",")); - addColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target); - - rewrittenQueryStr.append(")\n WHERE ").append(onClauseAsString) - .append(" GROUP BY ").append(StringUtils.join(sortKeys, ",")); - - addColsToSelect(targetTable.getPartCols(), rewrittenQueryStr, target); - - rewrittenQueryStr.append(" HAVING count(*) > 1"); - //say table T has partition p, we are generating - //select cardinality_violation(ROW_ID, p) WHERE ... GROUP BY ROW__ID, p - //the Group By args are passed to cardinality_violation to add the violating value to the error msg - try { - if (null == db.getTable(tableName, false)) { - StorageFormat format = new StorageFormat(conf); - format.processStorageFormat("TextFile"); - Table table = db.newTable(tableName); - table.setSerializationLib(format.getSerde()); - List fields = new ArrayList(); - fields.add(new FieldSchema("val", "int", null)); - table.setFields(fields); - table.setDataLocation(Warehouse.getDnsPath(new Path(SessionState.get().getTempTableSpace(), - tableName), conf)); - table.getTTable().setTemporary(true); - table.setStoredAsSubDirectories(false); - table.setInputFormatClass(format.getInputFormat()); - table.setOutputFormatClass(format.getOutputFormat()); - db.createTable(table, true); + private String getSourceAlias(ASTNode source, String sourceName) throws SemanticException { + String sourceAlias; + if (source.getType() == HiveParser.TOK_SUBQUERY) { + //this includes the mandatory alias + sourceAlias = getMatchedText(source); + } else { + sourceAlias = getFullTableNameForSQL(source); + if (isAliased(source)) { + sourceAlias = String.format("%s %s", sourceAlias, sourceName); } - } catch(HiveException|MetaException e) { - throw new SemanticException(e.getMessage(), e); } - return true; + return sourceAlias; } /** - * @param onClauseAsString - because there is no clone() and we need to use in multiple places * @param deleteExtraPredicate - see notes at caller */ - private String handleUpdate(ASTNode whenMatchedUpdateClause, StringBuilder rewrittenQueryStr, ASTNode target, - String onClauseAsString, Table targetTable, String deleteExtraPredicate, String hintStr, - ColumnAppender columnAppender) - throws SemanticException { + private MergeStatement.UpdateClause handleUpdate(ASTNode whenMatchedUpdateClause, Table targetTable, + String deleteExtraPredicate) throws SemanticException { assert whenMatchedUpdateClause.getType() == HiveParser.TOK_MATCHED; assert getWhenClauseOperation(whenMatchedUpdateClause).getType() == HiveParser.TOK_UPDATE; - String targetName = getSimpleTableName(target); - List values = new ArrayList<>(targetTable.getCols().size()); - + Map newValuesMap = new HashMap<>(targetTable.getCols().size() + targetTable.getPartCols().size()); ASTNode setClause = (ASTNode)getWhenClauseOperation(whenMatchedUpdateClause).getChild(0); //columns being updated -> update expressions; "setRCols" (last param) is null because we use actual expressions //before re-parsing, i.e. they are known to SemanticAnalyzer logic @@ -396,77 +254,24 @@ private String handleUpdate(ASTNode whenMatchedUpdateClause, StringBuilder rewri //do nothing } - values.add(rhsExp); - } else { - values.add(targetName + "." + HiveUtils.unparseIdentifier(name, this.conf)); + newValuesMap.put(name, rhsExp); } } - addPartitionColsAsValues(targetTable.getPartCols(), targetName, values); - - String extraPredicate = handleUpdate(whenMatchedUpdateClause, rewrittenQueryStr, onClauseAsString, - deleteExtraPredicate, hintStr, columnAppender, targetName, values); - - setUpAccessControlInfoForUpdate(targetTable, setColsExprs); - return extraPredicate; - } - - protected String handleUpdate(ASTNode whenMatchedUpdateClause, StringBuilder rewrittenQueryStr, - String onClauseAsString, String deleteExtraPredicate, String hintStr, - ColumnAppender columnAppender, String targetName, List values) { - values.add(0, targetName + ".ROW__ID"); - - rewrittenQueryStr.append(" -- update clause").append("\n"); - appendInsertBranch(rewrittenQueryStr, hintStr, values); - - String extraPredicate = addWhereClauseOfUpdate( - rewrittenQueryStr, onClauseAsString, whenMatchedUpdateClause, deleteExtraPredicate); - appendSortBy(rewrittenQueryStr, Collections.singletonList(targetName + ".ROW__ID ")); - rewrittenQueryStr.append("\n"); - - return extraPredicate; - } - - protected String addWhereClauseOfUpdate(StringBuilder rewrittenQueryStr, String onClauseAsString, - ASTNode whenMatchedUpdateClause, String deleteExtraPredicate) { - rewrittenQueryStr.append(INDENT).append("WHERE ").append(onClauseAsString); String extraPredicate = getWhenClausePredicate(whenMatchedUpdateClause); - if (extraPredicate != null) { - //we have WHEN MATCHED AND THEN DELETE - rewrittenQueryStr.append(" AND ").append(extraPredicate); - } - if (deleteExtraPredicate != null) { - rewrittenQueryStr.append(" AND NOT(").append(deleteExtraPredicate).append(")"); - } - return extraPredicate; + setUpAccessControlInfoForUpdate(targetTable, setColsExprs); + return new MergeStatement.UpdateClause(extraPredicate, deleteExtraPredicate, newValuesMap); } /** - * @param onClauseAsString - because there is no clone() and we need to use in multiple places * @param updateExtraPredicate - see notes at caller */ - protected String handleDelete(ASTNode whenMatchedDeleteClause, StringBuilder rewrittenQueryStr, - String onClauseAsString, String updateExtraPredicate, - String hintStr, ColumnAppender columnAppender) { + protected MergeStatement.DeleteClause handleDelete( + ASTNode whenMatchedDeleteClause, String updateExtraPredicate) { assert whenMatchedDeleteClause.getType() == HiveParser.TOK_MATCHED; - - List deleteValues = columnAppender.getDeleteValues(Context.Operation.DELETE); - appendInsertBranch(rewrittenQueryStr, hintStr, deleteValues); - - rewrittenQueryStr.append(INDENT).append("WHERE ").append(onClauseAsString); String extraPredicate = getWhenClausePredicate(whenMatchedDeleteClause); - if (extraPredicate != null) { - //we have WHEN MATCHED AND THEN DELETE - rewrittenQueryStr.append(" AND ").append(extraPredicate); - } - if (updateExtraPredicate != null) { - rewrittenQueryStr.append(" AND NOT(").append(updateExtraPredicate).append(")"); - } - List sortKeys = columnAppender.getSortKeys(); - rewrittenQueryStr.append("\n").append(INDENT); - appendSortBy(rewrittenQueryStr, sortKeys); - return extraPredicate; + return new MergeStatement.DeleteClause(extraPredicate, updateExtraPredicate); } private static String addParseInfo(ASTNode n) { @@ -519,9 +324,9 @@ private String getWhenClausePredicate(ASTNode whenClause) { * @param targetTableNameInSourceQuery - simple name/alias * @throws SemanticException */ - private void handleInsert(ASTNode whenNotMatchedClause, StringBuilder rewrittenQueryStr, ASTNode target, - ASTNode onClause, Table targetTable, String targetTableNameInSourceQuery, String onClauseAsString, - String hintStr) throws SemanticException { + private MergeStatement.InsertClause handleInsert(ASTNode whenNotMatchedClause, ASTNode onClause, + Table targetTable, String targetTableNameInSourceQuery, + String onClauseAsString) throws SemanticException { ASTNode whenClauseOperation = getWhenClauseOperation(whenNotMatchedClause); assert whenNotMatchedClause.getType() == HiveParser.TOK_NOT_MATCHED; assert whenClauseOperation.getType() == HiveParser.TOK_INSERT; @@ -541,16 +346,6 @@ private void handleInsert(ASTNode whenNotMatchedClause, StringBuilder rewrittenQ columnListNode.getChildCount(), valuesNode.getChildCount() - 1)); } - rewrittenQueryStr.append("INSERT INTO ").append(getFullTableNameForSQL(target)); - if (columnListNode != null) { - rewrittenQueryStr.append(' ').append(getMatchedText(columnListNode)); - } - - rewrittenQueryStr.append(" -- insert clause\n SELECT "); - if (hintStr != null) { - rewrittenQueryStr.append(hintStr); - } - OnClauseAnalyzer oca = new OnClauseAnalyzer(onClause, targetTable, targetTableNameInSourceQuery, conf, onClauseAsString); oca.analyze(); @@ -562,15 +357,10 @@ private void handleInsert(ASTNode whenNotMatchedClause, StringBuilder rewrittenQ defaultValuesTranslator.applyTranslations(ctx.getTokenRewriteStream()); String valuesClause = getMatchedText(valuesNode); valuesClause = valuesClause.substring(1, valuesClause.length() - 1); //strip '(' and ')' - rewrittenQueryStr.append(valuesClause).append("\n WHERE ").append(oca.getPredicate()); String extraPredicate = getWhenClausePredicate(whenNotMatchedClause); - if (extraPredicate != null) { - //we have WHEN NOT MATCHED AND THEN INSERT - rewrittenQueryStr.append(" AND ") - .append(getMatchedText(((ASTNode)whenNotMatchedClause.getChild(1)))); - } - rewrittenQueryStr.append('\n'); + return new MergeStatement.InsertClause( + getMatchedText(columnListNode), valuesClause, oca.getPredicate(), extraPredicate); } private void collectDefaultValues( @@ -723,11 +513,88 @@ private String getPredicate() { @Override protected boolean allowOutputMultipleTimes() { - return conf.getBoolVar(HiveConf.ConfVars.SPLIT_UPDATE) || conf.getBoolVar(HiveConf.ConfVars.MERGE_SPLIT_UPDATE); + return conf.getBoolVar(HiveConf.ConfVars.SPLIT_UPDATE); } @Override protected boolean enableColumnStatsCollecting() { return numWhenMatchedUpdateClauses == 0 && numWhenMatchedDeleteClauses == 0; } + + /** + * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it. + * Since HiveLexer.g is written such that it strips away any ` (back ticks) around + * quoted identifiers we need to add those back to generated SQL. + * Additionally, the parser only produces tokens of type Identifier and never + * QuotedIdentifier (HIVE-6013). So here we just quote all identifiers. + * (') around String literals are retained w/o issues + */ + private static final class IdentifierQuoter { + private final TokenRewriteStream trs; + private final IdentityHashMap visitedNodes = new IdentityHashMap<>(); + + IdentifierQuoter(TokenRewriteStream trs) { + this.trs = trs; + if (trs == null) { + throw new IllegalArgumentException("Must have a TokenRewriteStream"); + } + } + + private void visit(ASTNode n) { + if (n.getType() == HiveParser.Identifier) { + if (visitedNodes.containsKey(n)) { + /** + * Since we are modifying the stream, it's not idempotent. Ideally, the caller would take + * care to only quote Identifiers in each subtree once, but this makes it safe + */ + return; + } + visitedNodes.put(n, n); + trs.insertBefore(n.getToken(), "`"); + trs.insertAfter(n.getToken(), "`"); + } + if (n.getChildCount() <= 0) { + return; + } + for (Node c : n.getChildren()) { + visit((ASTNode)c); + } + } + } + + /** + * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it without + * needing to understand what it is (except for QuotedIdentifiers). + */ + protected String getMatchedText(ASTNode n) { + if (n == null) { + return null; + } + + quotedIdentifierHelper.visit(n); + return ctx.getTokenRewriteStream().toString(n.getTokenStartIndex(), + n.getTokenStopIndex() + 1).trim(); + } + + protected boolean isAliased(ASTNode n) { + switch (n.getType()) { + case HiveParser.TOK_TABREF: + return findTabRefIdxs(n)[0] != 0; + case HiveParser.TOK_TABNAME: + return false; + case HiveParser.TOK_SUBQUERY: + assert n.getChildCount() > 1 : "Expected Derived Table to be aliased"; + return true; + default: + throw raiseWrongType("TOK_TABREF|TOK_TABNAME", n); + } + } + + /** + * Returns the table name to use in the generated query preserving original quotes/escapes if any. + * @see #getFullTableNameForSQL(ASTNode) + */ + protected String getSimpleTableName(ASTNode n) throws SemanticException { + return HiveUtils.unparseIdentifier(getSimpleTableNameBase(n), conf); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java index a246e3af2a84..a3a60f3d5c41 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java @@ -675,11 +675,16 @@ public static String stripIdentifierQuotes(String val) { return val; } + public static ReparseResult parseRewrittenQuery(Context ctx, StringBuilder rewrittenQueryStr) + throws SemanticException { + return parseRewrittenQuery(ctx, rewrittenQueryStr.toString()); + } + /** * Parse the newly generated SQL statement to get a new AST. */ public static ReparseResult parseRewrittenQuery(Context ctx, - StringBuilder rewrittenQueryStr) + String rewrittenQueryStr) throws SemanticException { // Set dynamic partitioning to nonstrict so that queries do not need any partition // references. @@ -699,12 +704,12 @@ public static ReparseResult parseRewrittenQuery(Context ctx, rewrittenCtx.setStatsSource(ctx.getStatsSource()); rewrittenCtx.setPlanMapper(ctx.getPlanMapper()); rewrittenCtx.setIsUpdateDeleteMerge(true); - rewrittenCtx.setCmd(rewrittenQueryStr.toString()); + rewrittenCtx.setCmd(rewrittenQueryStr); ASTNode rewrittenTree; try { - LOG.info("Going to reparse <" + ctx.getCmd() + "> as \n<" + rewrittenQueryStr.toString() + ">"); - rewrittenTree = ParseUtils.parse(rewrittenQueryStr.toString(), rewrittenCtx); + LOG.info("Going to reparse <{}> as \n<{}>", ctx.getCmd(), rewrittenQueryStr); + rewrittenTree = ParseUtils.parse(rewrittenQueryStr, rewrittenCtx); } catch (ParseException e) { throw new SemanticException(ErrorMsg.UPDATEDELETE_PARSE_ERROR.getMsg(), e); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java index 8bd4c9611d5e..f968c65443c5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java @@ -17,18 +17,7 @@ */ package org.apache.hadoop.hive.ql.parse; -import java.util.ArrayList; -import java.util.IdentityHashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import org.antlr.runtime.TokenRewriteStream; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.common.TableName; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.Context; @@ -41,17 +30,19 @@ import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.InvalidTableException; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; +import org.apache.hadoop.hive.ql.parse.rewrite.Rewriter; +import org.apache.hadoop.hive.ql.parse.rewrite.RewriterFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.commons.lang3.StringUtils.isNotBlank; - +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles @@ -59,18 +50,18 @@ * statements (since they are actually inserts) and then doing some patch up to make them work as * updates and deletes instead. */ -public abstract class RewriteSemanticAnalyzer extends CalcitePlanner { +public abstract class RewriteSemanticAnalyzer extends CalcitePlanner { protected static final Logger LOG = LoggerFactory.getLogger(RewriteSemanticAnalyzer.class); + private final RewriterFactory rewriterFactory; protected boolean useSuper = false; - protected static final String INDENT = " "; - private IdentifierQuoter quotedIdentifierHelper; + private ASTNode tableName; private Table targetTable; - private String targetTableFullName; - RewriteSemanticAnalyzer(QueryState queryState) throws SemanticException { + RewriteSemanticAnalyzer(QueryState queryState, RewriterFactory rewriterFactory) throws SemanticException { super(queryState); + this.rewriterFactory = rewriterFactory; } @Override @@ -78,7 +69,6 @@ public void analyzeInternal(ASTNode tree) throws SemanticException { if (useSuper) { super.analyzeInternal(tree); } else { - quotedIdentifierHelper = new IdentifierQuoter(ctx.getTokenRewriteStream()); analyze(tree); cleanUpMetaColumnAccessControl(); } @@ -87,9 +77,7 @@ public void analyzeInternal(ASTNode tree) throws SemanticException { protected abstract ASTNode getTargetTableNode(ASTNode tree); private void analyze(ASTNode tree) throws SemanticException { - ASTNode tableName = getTargetTableNode(tree); - - targetTableFullName = getFullTableNameForSQL(tableName); + tableName = getTargetTableNode(tree); targetTable = getTable(tableName, db, true); validateTxnManager(targetTable); validateTargetTable(targetTable); @@ -98,7 +86,19 @@ private void analyze(ASTNode tree) throws SemanticException { protected abstract void analyze(ASTNode tree, Table table, ASTNode tableName) throws SemanticException; - public void analyzeRewrittenTree(ASTNode rewrittenTree, Context rewrittenCtx) throws SemanticException { + protected void rewriteAndAnalyze(T statementData, String subQueryAlias) throws SemanticException { + Rewriter rewriter = + rewriterFactory.createRewriter(targetTable, getFullTableNameForSQL(tableName), subQueryAlias); + + ParseUtils.ReparseResult rr = rewriter.rewrite(ctx, statementData); + + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + + analyzeRewrittenTree(rewrittenTree, rewrittenCtx); + } + + protected void analyzeRewrittenTree(ASTNode rewrittenTree, Context rewrittenCtx) throws SemanticException { try { if (LOG.isDebugEnabled()) { LOG.debug("Rewritten AST {}", rewrittenTree.dump()); @@ -110,52 +110,6 @@ public void analyzeRewrittenTree(ASTNode rewrittenTree, Context rewrittenCtx) th } } - /** - * Append list of partition columns to Insert statement, i.e. the 2nd set of partCol1,partCol2 - * INSERT INTO T PARTITION(partCol1,partCol2...) SELECT col1, ... partCol1,partCol2... - */ - protected void addColsToSelect(List partCols, StringBuilder rewrittenQueryStr) - throws SemanticException { - // If the table is partitioned, we need to select the partition columns as well. - if (partCols != null) { - for (FieldSchema fschema : partCols) { - rewrittenQueryStr.append(", "); - rewrittenQueryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf)); - } - } - } - - /** - * Append list of columns to rewritten statement. - */ - protected void addColsToSelect(List cols, StringBuilder rewrittenQueryStr, - ASTNode target) throws SemanticException { - addColsToSelect(cols, rewrittenQueryStr, getSimpleTableName(target)); - } - - /** - * Append list of columns to rewritten statement. - * Column names are qualified with the specified alias and quoted. - */ - protected void addColsToSelect(List cols, StringBuilder rewrittenQueryStr, String alias) { - // If the table is partitioned, we need to select the partition columns as well. - if (cols != null) { - for (FieldSchema fschema : cols) { - rewrittenQueryStr.append(", "); - rewrittenQueryStr.append(alias).append('.'); - rewrittenQueryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf)); - } - } - } - - protected void addPartitionColsAsValues(List partCols, String alias, List values) { - if (partCols == null) { - return; - } - partCols.forEach( - fieldSchema -> values.add(alias + "." + HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf))); - } - /** * Assert that we are not asked to update a bucketing column or partition column. * @param colName it's the A in "SET A = B" @@ -359,7 +313,7 @@ private static String normalizeColName(String colName) { * SemanticAnalyzer will generate a WriteEntity for the target table since it doesn't know/check * if the read and write are of the same table in "insert ... select ....". Since DbTxnManager * uses Read/WriteEntity objects to decide which locks to acquire, we get more concurrency if we - * have change the table WriteEntity to a set of partition WriteEntity objects based on + * have changed the table WriteEntity to a set of partition WriteEntity objects based on * ReadEntity objects computed for this table. */ protected void updateOutputs(Table targetTable) { @@ -431,255 +385,4 @@ private boolean isTargetTable(Entity entity, Table targetTable) { // while comparing two tables return targetTable.equalsWithIgnoreWriteId(entity.getTable()); } - - /** - * Returns the table name to use in the generated query preserving original quotes/escapes if any. - * @see #getFullTableNameForSQL(ASTNode) - */ - protected String getSimpleTableName(ASTNode n) throws SemanticException { - return HiveUtils.unparseIdentifier(getSimpleTableNameBase(n), this.conf); - } - - // Patch up the projection list for updates, putting back the original set expressions. - // Walk through the projection list and replace the column names with the - // expressions from the original update. Under the TOK_SELECT (see above) the structure - // looks like: - // TOK_SELECT -> TOK_SELEXPR -> expr - // \-> TOK_SELEXPR -> expr ... - protected void patchProjectionForUpdate(ASTNode insertBranch, Map setColExprs) { - ASTNode rewrittenSelect = (ASTNode) insertBranch.getChildren().get(1); - assert rewrittenSelect.getToken().getType() == HiveParser.TOK_SELECT : - "Expected TOK_SELECT as second child of TOK_INSERT but found " + rewrittenSelect.getName(); - for (Map.Entry entry : setColExprs.entrySet()) { - ASTNode selExpr = (ASTNode) rewrittenSelect.getChildren().get(entry.getKey()); - assert selExpr.getToken().getType() == HiveParser.TOK_SELEXPR : - "Expected child of TOK_SELECT to be TOK_SELEXPR but was " + selExpr.getName(); - // Now, change it's child - selExpr.setChild(0, entry.getValue()); - } - } - - protected StringBuilder createRewrittenQueryStrBuilder() { - return new StringBuilder("FROM\n"); - } - - protected void appendTarget(StringBuilder rewrittenQueryStr, ASTNode target, String targetName) { - rewrittenQueryStr.append(INDENT).append(targetTableFullName); - if (isAliased(target)) { - rewrittenQueryStr.append(" ").append(targetName); - } - rewrittenQueryStr.append('\n'); - } - - protected boolean isAliased(ASTNode n) { - switch (n.getType()) { - case HiveParser.TOK_TABREF: - return findTabRefIdxs(n)[0] != 0; - case HiveParser.TOK_TABNAME: - return false; - case HiveParser.TOK_SUBQUERY: - assert n.getChildCount() > 1 : "Expected Derived Table to be aliased"; - return true; - default: - throw raiseWrongType("TOK_TABREF|TOK_TABNAME", n); - } - } - - protected void appendInsertBranch(StringBuilder rewrittenQueryStr, String hintStr, List values) { - rewrittenQueryStr.append("INSERT INTO ").append(targetTableFullName); - addPartitionColsToInsert(targetTable.getPartCols(), rewrittenQueryStr); - rewrittenQueryStr.append("\n"); - - rewrittenQueryStr.append(INDENT); - rewrittenQueryStr.append("SELECT "); - if (isNotBlank(hintStr)) { - rewrittenQueryStr.append(hintStr); - } - - rewrittenQueryStr.append(StringUtils.join(values, ",")); - rewrittenQueryStr.append("\n"); - } - - protected void appendSortBy(StringBuilder rewrittenQueryStr, List keys) { - if (keys.isEmpty()) { - return; - } - rewrittenQueryStr.append(INDENT).append("SORT BY "); - rewrittenQueryStr.append(StringUtils.join(keys, ",")); - rewrittenQueryStr.append("\n"); - } - - /** - * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it. - * Since HiveLexer.g is written such that it strips away any ` (back ticks) around - * quoted identifiers we need to add those back to generated SQL. - * Additionally, the parser only produces tokens of type Identifier and never - * QuotedIdentifier (HIVE-6013). So here we just quote all identifiers. - * (') around String literals are retained w/o issues - */ - private static class IdentifierQuoter { - private final TokenRewriteStream trs; - private final IdentityHashMap visitedNodes = new IdentityHashMap<>(); - - IdentifierQuoter(TokenRewriteStream trs) { - this.trs = trs; - if (trs == null) { - throw new IllegalArgumentException("Must have a TokenRewriteStream"); - } - } - - private void visit(ASTNode n) { - if (n.getType() == HiveParser.Identifier) { - if (visitedNodes.containsKey(n)) { - /** - * Since we are modifying the stream, it's not idempotent. Ideally, the caller would take - * care to only quote Identifiers in each subtree once, but this makes it safe - */ - return; - } - visitedNodes.put(n, n); - trs.insertBefore(n.getToken(), "`"); - trs.insertAfter(n.getToken(), "`"); - } - if (n.getChildCount() <= 0) { - return; - } - for (Node c : n.getChildren()) { - visit((ASTNode)c); - } - } - } - - /** - * This allows us to take an arbitrary ASTNode and turn it back into SQL that produced it without - * needing to understand what it is (except for QuotedIdentifiers). - */ - protected String getMatchedText(ASTNode n) { - quotedIdentifierHelper.visit(n); - return ctx.getTokenRewriteStream().toString(n.getTokenStartIndex(), - n.getTokenStopIndex() + 1).trim(); - } - - public static final String DELETE_PREFIX = "__d__"; - public static final String SUB_QUERY_ALIAS = "s"; - - protected ColumnAppender getColumnAppender(String subQueryAlias, String deletePrefix) { - boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable, true); - return nonNativeAcid ? new NonNativeAcidColumnAppender(targetTable, conf, subQueryAlias, deletePrefix) : - new NativeAcidColumnAppender(targetTable, conf, subQueryAlias); - } - - protected static abstract class ColumnAppender { - protected final Table table; - protected final HiveConf conf; - protected final String subQueryAlias; - - protected ColumnAppender(Table table, HiveConf conf, String subQueryAlias) { - this.table = table; - this.conf = conf; - this.subQueryAlias = subQueryAlias; - } - - public abstract void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation); - - public void appendAcidSelectColumnsForDeletedRecords(StringBuilder stringBuilder, Context.Operation operation) { - throw new UnsupportedOperationException(); - } - - public abstract List getDeleteValues(Context.Operation operation); - public abstract List getSortKeys(); - - protected String qualify(String columnName) { - if (isBlank(subQueryAlias)) { - return columnName; - } - return String.format("%s.%s", subQueryAlias, columnName); - } - } - - protected static class NativeAcidColumnAppender extends ColumnAppender { - - public NativeAcidColumnAppender(Table table, HiveConf conf, String subQueryAlias) { - super(table, conf, subQueryAlias); - } - - @Override - public void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation) { - stringBuilder.append("ROW__ID,"); - for (FieldSchema fieldSchema : table.getPartCols()) { - String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); - stringBuilder.append(identifier); - stringBuilder.append(","); - } - } - - @Override - public List getDeleteValues(Context.Operation operation) { - List deleteValues = new ArrayList<>(1 + table.getPartCols().size()); - deleteValues.add(qualify("ROW__ID")); - for (FieldSchema fieldSchema : table.getPartCols()) { - deleteValues.add(qualify(HiveUtils.unparseIdentifier(fieldSchema.getName(), conf))); - } - return deleteValues; - } - - @Override - public List getSortKeys() { - return singletonList(qualify("ROW__ID")); - } - } - - protected static class NonNativeAcidColumnAppender extends ColumnAppender { - private final String deletePrefix; - - public NonNativeAcidColumnAppender(Table table, HiveConf conf, String subQueryAlias, String deletePrefix) { - super(table, conf, subQueryAlias); - this.deletePrefix = deletePrefix; - } - - @Override - public void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation) { - appendAcidSelectColumns(stringBuilder, operation, false); - } - - @Override - public void appendAcidSelectColumnsForDeletedRecords(StringBuilder stringBuilder, Context.Operation operation) { - appendAcidSelectColumns(stringBuilder, operation, true); - } - - private void appendAcidSelectColumns(StringBuilder stringBuilder, Context.Operation operation, boolean markRowIdAsDeleted) { - List acidSelectColumns = table.getStorageHandler().acidSelectColumns(table, operation); - for (FieldSchema fieldSchema : acidSelectColumns) { - String identifier = markRowIdAsDeleted && fieldSchema.equals(table.getStorageHandler().getRowId()) ? - "-1" : HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); - stringBuilder.append(identifier); - - if (StringUtils.isNotEmpty(deletePrefix) && !markRowIdAsDeleted) { - stringBuilder.append(" AS "); - String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); - stringBuilder.append(prefixedIdentifier); - } - stringBuilder.append(","); - } - } - - @Override - public List getDeleteValues(Context.Operation operation) { - List acidSelectColumns = table.getStorageHandler().acidSelectColumns(table, operation); - List deleteValues = new ArrayList<>(acidSelectColumns.size()); - for (FieldSchema fieldSchema : acidSelectColumns) { - String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); - deleteValues.add(qualify(prefixedIdentifier)); - } - return deleteValues; - } - - @Override - public List getSortKeys() { - return table.getStorageHandler().acidSortColumns(table, Context.Operation.DELETE).stream() - .map(fieldSchema -> qualify( - HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf))) - .collect(Collectors.toList()); - } - } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java index aa6c93ce1c69..bd3c720f356a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java @@ -22,6 +22,9 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.DeleteRewriterFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.MergeRewriterFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.UpdateRewriterFactory; import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,18 +97,12 @@ private static BaseSemanticAnalyzer getInternal(QueryState queryState, ASTNode t return new ColumnStatsSemanticAnalyzer(queryState); case HiveParser.TOK_UPDATE_TABLE: - if (HiveConf.getBoolVar(queryState.getConf(), HiveConf.ConfVars.SPLIT_UPDATE)) { - return new SplitUpdateSemanticAnalyzer(queryState); - } + return new UpdateSemanticAnalyzer(queryState, new UpdateRewriterFactory(queryState.getConf())); case HiveParser.TOK_DELETE_FROM: - return new UpdateDeleteSemanticAnalyzer(queryState); + return new DeleteSemanticAnalyzer(queryState, new DeleteRewriterFactory(queryState.getConf())); case HiveParser.TOK_MERGE: - if (HiveConf.getBoolVar(queryState.getConf(), HiveConf.ConfVars.SPLIT_UPDATE) || - HiveConf.getBoolVar(queryState.getConf(), HiveConf.ConfVars.MERGE_SPLIT_UPDATE)) { - return new SplitMergeSemanticAnalyzer(queryState); - } - return new MergeSemanticAnalyzer(queryState); + return new MergeSemanticAnalyzer(queryState, new MergeRewriterFactory(queryState.getConf())); case HiveParser.TOK_ALTER_SCHEDULED_QUERY: case HiveParser.TOK_CREATE_SCHEDULED_QUERY: diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitMergeSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitMergeSemanticAnalyzer.java deleted file mode 100644 index 8efe712f58ea..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitMergeSemanticAnalyzer.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.parse; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; -import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.session.SessionState; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - - -/** - * A subclass of the {@link MergeSemanticAnalyzer} that just handles - * merge statements. This version of rewrite adds two insert branches for the update clause one for - * inserting new values of updated records and one for inserting the deleted delta records of updated records. - */ -public class SplitMergeSemanticAnalyzer extends MergeSemanticAnalyzer { - - SplitMergeSemanticAnalyzer(QueryState queryState) throws SemanticException { - super(queryState); - } - - @Override - protected int addDestNamePrefixOfUpdate(int insClauseIdx, Context rewrittenCtx) { - rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.INSERT); - rewrittenCtx.addDeleteOfUpdateDestNamePrefix(insClauseIdx + 1, Context.DestClausePrefix.DELETE); - return 2; - } - - @Override - public void analyze(ASTNode tree, Table targetTable, ASTNode tableNameNode) throws SemanticException { - analyzeMerge(tree, targetTable, tableNameNode); - } - - @Override - protected String handleUpdate(ASTNode whenMatchedUpdateClause, StringBuilder rewrittenQueryStr, - String onClauseAsString, String deleteExtraPredicate, String hintStr, - ColumnAppender columnAppender, String targetName, List values) { - rewrittenQueryStr.append(" -- update clause (insert part)\n"); - appendInsertBranch(rewrittenQueryStr, hintStr, values); - - String extraPredicate = addWhereClauseOfUpdate( - rewrittenQueryStr, onClauseAsString, whenMatchedUpdateClause, deleteExtraPredicate); - - rewrittenQueryStr.append("\n"); - - rewrittenQueryStr.append(" -- update clause (delete part)\n"); - handleDelete(whenMatchedUpdateClause, rewrittenQueryStr, onClauseAsString, - deleteExtraPredicate, hintStr, columnAppender); - - return extraPredicate; - } - - /** - * @param onClauseAsString - because there is no clone() and we need to use in multiple places - * @param updateExtraPredicate - see notes at caller - */ - @Override - protected String handleDelete(ASTNode whenMatchedDeleteClause, StringBuilder rewrittenQueryStr, - String onClauseAsString, String updateExtraPredicate, - String hintStr, ColumnAppender columnAppender) { - assert ( - getWhenClauseOperation(whenMatchedDeleteClause).getType() == HiveParser.TOK_UPDATE) || - getWhenClauseOperation(whenMatchedDeleteClause).getType() == HiveParser.TOK_DELETE; - - return super.handleDelete( - whenMatchedDeleteClause, rewrittenQueryStr, onClauseAsString, updateExtraPredicate, hintStr, columnAppender); - } - - @Override - protected boolean allowOutputMultipleTimes() { - return true; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java deleted file mode 100644 index d36b1129067a..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SplitUpdateSemanticAnalyzer.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.parse; - -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; -import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.parse.ParseUtils.ReparseResult; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * A subclass of the {@link SemanticAnalyzer} that just handles - * update statements. It works by rewriting the updates into multi-insert - * statements (since they are actually inserts). - * One insert branch for inserting the new values of the updated records. - * And another insert branch for inserting delete delta records of the updated records. - * - * From - * UPDATE acidtlb SET b=350 - * WHERE a = 30 - * - * To - * FROM - * (SELECT ROW__ID,`a` AS `a`,350 AS `b` FROM `default`.`acidtlb` WHERE a = 30) s - * INSERT INTO `default`.`acidtlb` -- insert delta - * SELECT s.`a`,s.`b` - * INSERT INTO `default`.`acidtlb` -- delete delta - * SELECT s.ROW__ID - * SORT BY s.ROW__ID - */ -public class SplitUpdateSemanticAnalyzer extends RewriteSemanticAnalyzer { - - private Context.Operation operation = Context.Operation.OTHER; - - SplitUpdateSemanticAnalyzer(QueryState queryState) throws SemanticException { - super(queryState); - } - - @Override - protected ASTNode getTargetTableNode(ASTNode tree) { - // The first child should be the table we are updating / deleting from - ASTNode tabName = (ASTNode)tree.getChild(0); - assert tabName.getToken().getType() == HiveParser.TOK_TABNAME : - "Expected tablename as first child of " + operation + " but found " + tabName.getName(); - return tabName; - } - - protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws SemanticException { - switch (tree.getToken().getType()) { - case HiveParser.TOK_UPDATE_TABLE: - boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); - if (nonNativeAcid && AcidUtils.isCopyOnWriteMode(table, Context.Operation.UPDATE)) { - throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_COW_UPDATE.getErrorCodedMsg()); - } - analyzeUpdate(tree, table, tabNameNode); - break; - default: - throw new RuntimeException("Asked to parse token " + tree.getName() + " in " + - "SplitUpdateSemanticAnalyzer"); - } - } - - private void analyzeUpdate(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException { - operation = Context.Operation.UPDATE; - - List children = tree.getChildren(); - - ASTNode where = null; - int whereIndex = 2; - if (children.size() > whereIndex) { - where = (ASTNode) children.get(whereIndex); - assert where.getToken().getType() == HiveParser.TOK_WHERE : - "Expected where clause, but found " + where.getName(); - } - - Set setRCols = new LinkedHashSet<>(); -// TOK_UPDATE_TABLE -// TOK_TABNAME -// ... -// TOK_SET_COLUMNS_CLAUSE <- The set list from update should be the second child (index 1) - assert children.size() >= 2 : "Expected update token to have at least two children"; - ASTNode setClause = (ASTNode) children.get(1); - Map setCols = collectSetColumnsAndExpressions(setClause, setRCols, mTable); - Map setColExprs = new HashMap<>(setClause.getChildCount()); - - List nonPartCols = mTable.getCols(); - Map colNameToDefaultConstraint = getColNameToDefaultValueMap(mTable); - StringBuilder rewrittenQueryStr = createRewrittenQueryStrBuilder(); - rewrittenQueryStr.append("(SELECT "); - - ColumnAppender columnAppender = getColumnAppender(SUB_QUERY_ALIAS, DELETE_PREFIX); - columnAppender.appendAcidSelectColumns(rewrittenQueryStr, operation); - List deleteValues = columnAppender.getDeleteValues(operation); - int columnOffset = deleteValues.size(); - - List insertValues = new ArrayList<>(mTable.getCols().size()); - boolean first = true; - - for (int i = 0; i < nonPartCols.size(); i++) { - if (first) { - first = false; - } else { - rewrittenQueryStr.append(","); - } - - String name = nonPartCols.get(i).getName(); - ASTNode setCol = setCols.get(name); - String identifier = HiveUtils.unparseIdentifier(name, this.conf); - - if (setCol != null) { - if (setCol.getType() == HiveParser.TOK_TABLE_OR_COL && - setCol.getChildCount() == 1 && setCol.getChild(0).getType() == HiveParser.TOK_DEFAULT_VALUE) { - rewrittenQueryStr.append(colNameToDefaultConstraint.get(name)); - } else { - rewrittenQueryStr.append(identifier); - // This is one of the columns we're setting, record it's position so we can come back - // later and patch it up. 0th is ROW_ID - setColExprs.put(i + columnOffset, setCol); - } - } else { - rewrittenQueryStr.append(identifier); - } - rewrittenQueryStr.append(" AS "); - rewrittenQueryStr.append(identifier); - - insertValues.add(SUB_QUERY_ALIAS + "." + identifier); - } - addPartitionColsAsValues(mTable.getPartCols(), SUB_QUERY_ALIAS, insertValues); - rewrittenQueryStr.append(" FROM ").append(getFullTableNameForSQL(tabNameNode)).append(") "); - rewrittenQueryStr.append(SUB_QUERY_ALIAS).append("\n"); - - appendInsertBranch(rewrittenQueryStr, null, insertValues); - appendInsertBranch(rewrittenQueryStr, null, deleteValues); - - List sortKeys = columnAppender.getSortKeys(); - appendSortBy(rewrittenQueryStr, sortKeys); - - ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, rewrittenQueryStr); - Context rewrittenCtx = rr.rewrittenCtx; - ASTNode rewrittenTree = rr.rewrittenTree; - - ASTNode rewrittenInsert = new ASTSearcher().simpleBreadthFirstSearch( - rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_INSERT); - - rewrittenCtx.setOperation(Context.Operation.UPDATE); - rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.INSERT); - rewrittenCtx.addDeleteOfUpdateDestNamePrefix(2, Context.DestClausePrefix.DELETE); - - if (where != null) { - rewrittenInsert.addChild(where); - } - - patchProjectionForUpdate(rewrittenInsert, setColExprs); - - // Note: this will overwrite this.ctx with rewrittenCtx - rewrittenCtx.setEnableUnparse(false); - analyzeRewrittenTree(rewrittenTree, rewrittenCtx); - - updateOutputs(mTable); - - setUpAccessControlInfoForUpdate(mTable, setCols); - - // Add the setRCols to the input list - if (columnAccessInfo == null) { //assuming this means we are not doing Auth - return; - } - - for (String colName : setRCols) { - columnAccessInfo.add(Table.getCompleteName(mTable.getDbName(), mTable.getTableName()), colName); - } - } - - @Override - protected boolean allowOutputMultipleTimes() { - return true; - } - - @Override - protected boolean enableColumnStatsCollecting() { - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java index f330ed6adfde..b01be0740631 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java @@ -170,7 +170,7 @@ private String processStorageHandler(String name) throws SemanticException { return ensureClassExists(BaseSemanticAnalyzer.unescapeSQLString(name)); } - protected void processStorageFormat(String name) throws SemanticException { + public void processStorageFormat(String name) throws SemanticException { StorageFormatDescriptor descriptor = getDescriptor(name, "STORED AS clause"); inputFormat = ensureClassExists(descriptor.getInputFormat()); outputFormat = ensureClassExists(descriptor.getOutputFormat()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java deleted file mode 100644 index 9ed1cc2db034..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java +++ /dev/null @@ -1,379 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.parse; - -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.hadoop.hive.common.TableName; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.ddl.DDLWork; -import org.apache.hadoop.hive.ql.ddl.table.execute.AlterTableExecuteDesc; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; -import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; -import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.parse.ParseUtils.ReparseResult; -import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; - -/** - * A subclass of the {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer} that just handles - * update and delete statements. It works by rewriting the updates and deletes into insert - * statements (since they are actually inserts) and then doing some patch up to make them work as - * updates and deletes instead. - */ -public class UpdateDeleteSemanticAnalyzer extends RewriteSemanticAnalyzer { - - private Context.Operation operation = Context.Operation.OTHER; - - UpdateDeleteSemanticAnalyzer(QueryState queryState) throws SemanticException { - super(queryState); - } - - @Override - protected ASTNode getTargetTableNode(ASTNode tree) { - // The first child should be the table we are updating / deleting from - ASTNode tabName = (ASTNode)tree.getChild(0); - assert tabName.getToken().getType() == HiveParser.TOK_TABNAME : - "Expected tablename as first child of " + operation + " but found " + tabName.getName(); - return tabName; - } - - protected void analyze(ASTNode tree, Table table, ASTNode tabNameNode) throws SemanticException { - switch (tree.getToken().getType()) { - case HiveParser.TOK_DELETE_FROM: - operation = Context.Operation.DELETE; - if (!tryMetadataUpdate(tree, table, tabNameNode)) { - reparseAndSuperAnalyze(tree, table, tabNameNode); - } - break; - case HiveParser.TOK_UPDATE_TABLE: - boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); - if (nonNativeAcid && !AcidUtils.isCopyOnWriteMode(table, Context.Operation.UPDATE)) { - throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); - } - operation = Context.Operation.UPDATE; - reparseAndSuperAnalyze(tree, table, tabNameNode); - break; - default: - throw new RuntimeException("Asked to parse token " + tree.getName() + " in " + - "UpdateDeleteSemanticAnalyzer"); - } - } - - /** - * This supports update and delete statements - * Rewrite the delete or update into an insert. Crazy, but it works as deletes and update - * actually are inserts into the delta file in Hive. A delete - * DELETE FROM _tablename_ [WHERE ...] - * will be rewritten as - * INSERT INTO TABLE _tablename_ [PARTITION (_partcols_)] SELECT ROW__ID[, - * _partcols_] from _tablename_ SORT BY ROW__ID - * An update - * UPDATE _tablename_ SET x = _expr_ [WHERE...] - * will be rewritten as - * INSERT INTO TABLE _tablename_ [PARTITION (_partcols_)] SELECT _all_, - * _partcols_from _tablename_ SORT BY ROW__ID - * where _all_ is all the non-partition columns. The expressions from the set clause will be - * re-attached later. - * The where clause will also be re-attached later. - * The sort by clause is put in there so that records come out in the right order to enable - * merge on read. - */ - private void reparseAndSuperAnalyze(ASTNode tree, Table mTable, ASTNode tabNameNode) throws SemanticException { - List children = tree.getChildren(); - - boolean shouldTruncate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_REPLACE_DELETE_WITH_TRUNCATE) - && children.size() == 1 && deleting(); - if (shouldTruncate) { - StringBuilder rewrittenQueryStr = new StringBuilder("truncate ").append(getFullTableNameForSQL(tabNameNode)); - ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, rewrittenQueryStr); - Context rewrittenCtx = rr.rewrittenCtx; - ASTNode rewrittenTree = rr.rewrittenTree; - - BaseSemanticAnalyzer truncate = SemanticAnalyzerFactory.get(queryState, rewrittenTree); - // Note: this will overwrite this.ctx with rewrittenCtx - rewrittenCtx.setEnableUnparse(false); - truncate.analyze(rewrittenTree, rewrittenCtx); - - rootTasks = truncate.getRootTasks(); - outputs = truncate.getOutputs(); - updateOutputs(mTable); - return; - } - - StringBuilder rewrittenQueryStr = new StringBuilder(); - rewrittenQueryStr.append("insert into table "); - rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); - addPartitionColsToInsert(mTable.getPartCols(), rewrittenQueryStr); - - ColumnAppender columnAppender = getColumnAppender(null, DELETE_PREFIX); - int columnOffset = columnAppender.getDeleteValues(operation).size(); - rewrittenQueryStr.append(" select "); - columnAppender.appendAcidSelectColumns(rewrittenQueryStr, operation); - rewrittenQueryStr.setLength(rewrittenQueryStr.length() - 1); - - boolean copyOnWriteMode = AcidUtils.isCopyOnWriteMode(mTable, operation); - Map setColExprs = null; - Map setCols = null; - // Must be deterministic order set for consistent q-test output across Java versions - Set setRCols = new LinkedHashSet(); - if (updating()) { - // We won't write the set - // expressions in the rewritten query. We'll patch that up later. - // The set list from update should be the second child (index 1) - assert children.size() >= 2 : "Expected update token to have at least two children"; - ASTNode setClause = (ASTNode)children.get(1); - setCols = collectSetColumnsAndExpressions(setClause, setRCols, mTable); - setColExprs = new HashMap<>(setClause.getChildCount()); - - List nonPartCols = mTable.getCols(); - for (int i = 0; i < nonPartCols.size(); i++) { - rewrittenQueryStr.append(','); - String name = nonPartCols.get(i).getName(); - ASTNode setCol = setCols.get(name); - String identifier = HiveUtils.unparseIdentifier(name, this.conf); - rewrittenQueryStr.append(identifier); - if (copyOnWriteMode) { - rewrittenQueryStr.append(" AS ").append(identifier); - } - if (setCol != null) { - // This is one of the columns we're setting, record it's position so we can come back - // later and patch it up. - // Add one to the index because the select has the ROW__ID as the first column. - setColExprs.put(columnOffset + i, setCol); - } - } - } - - rewrittenQueryStr.append(" from "); - rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); - - ASTNode where = null; - int whereIndex = deleting() ? 1 : 2; - - if (children.size() > whereIndex) { - where = (ASTNode)children.get(whereIndex); - assert where.getToken().getType() == HiveParser.TOK_WHERE : - "Expected where clause, but found " + where.getName(); - - if (copyOnWriteMode) { - String whereClause = ctx.getTokenRewriteStream().toString( - where.getChild(0).getTokenStartIndex(), where.getChild(0).getTokenStopIndex()); - String filePathCol = HiveUtils.unparseIdentifier("FILE__PATH", conf); - - if (updating()) { - rewrittenQueryStr.append("\nunion all"); - rewrittenQueryStr.append("\nselect "); - columnAppender.appendAcidSelectColumns(rewrittenQueryStr, Context.Operation.DELETE); - rewrittenQueryStr.setLength(rewrittenQueryStr.length() - 1); - rewrittenQueryStr.append(" from "); - rewrittenQueryStr.append(getFullTableNameForSQL(tabNameNode)); - } - // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. - rewrittenQueryStr.append("\nwhere NOT (").append(whereClause).append(")"); - rewrittenQueryStr.append("\n").append(INDENT); - // Add the file path filter that matches the delete condition. - rewrittenQueryStr.append("AND ").append(filePathCol); - rewrittenQueryStr.append(" IN ( select ").append(filePathCol).append(" from t )"); - rewrittenQueryStr.append("\nunion all"); - rewrittenQueryStr.append("\nselect * from t"); - - StringBuilder withQueryStr = new StringBuilder(); - withQueryStr.append("WITH t AS ("); - withQueryStr.append("\n").append(INDENT); - withQueryStr.append("select "); - columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, Context.Operation.DELETE); - withQueryStr.setLength(withQueryStr.length() - 1); - withQueryStr.append(" from ("); - withQueryStr.append("\n").append(INDENT).append(INDENT); - withQueryStr.append("select "); - columnAppender.appendAcidSelectColumnsForDeletedRecords(withQueryStr, Context.Operation.DELETE); - withQueryStr.append(" row_number() OVER (partition by ").append(filePathCol).append(") rn"); - withQueryStr.append(" from ").append(getFullTableNameForSQL(tabNameNode)); - withQueryStr.append("\n").append(INDENT).append(INDENT); - withQueryStr.append("where ").append(whereClause); - withQueryStr.append("\n").append(INDENT); - withQueryStr.append(") q"); - withQueryStr.append("\n").append(INDENT); - withQueryStr.append("where rn=1\n)\n"); - - rewrittenQueryStr.insert(0, withQueryStr.toString()); - } - } - - if (!copyOnWriteMode) { - // Add a sort by clause so that the row ids come out in the correct order - appendSortBy(rewrittenQueryStr, columnAppender.getSortKeys()); - } - ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, rewrittenQueryStr); - Context rewrittenCtx = rr.rewrittenCtx; - ASTNode rewrittenTree = rr.rewrittenTree; - - ASTNode rewrittenInsert = (ASTNode) (copyOnWriteMode && updating() ? - new ASTSearcher().simpleBreadthFirstSearch(rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, - HiveParser.TOK_UNIONALL).getChild(0).getChild(0) : rewrittenTree) - .getChild(1); - - if (updating()) { - rewrittenCtx.setOperation(Context.Operation.UPDATE); - rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); - } else if (deleting()) { - rewrittenCtx.setOperation(Context.Operation.DELETE); - rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE); - } - - if (where != null && (!copyOnWriteMode || updating())) { - assert rewrittenInsert.getToken().getType() == HiveParser.TOK_INSERT : - "Expected TOK_INSERT as second child of TOK_QUERY but found " + rewrittenInsert.getName(); - // The structure of the AST for the rewritten insert statement is: - // TOK_QUERY -> TOK_FROM - // \-> TOK_INSERT -> TOK_INSERT_INTO - // \-> TOK_SELECT - // \-> TOK_SORTBY - // Or - // TOK_QUERY -> TOK_FROM - // \-> TOK_INSERT -> TOK_INSERT_INTO - // \-> TOK_SELECT - // - // The following adds the TOK_WHERE and its subtree from the original query as a child of - // TOK_INSERT, which is where it would have landed if it had been there originally in the - // string. We do it this way because it's easy then turning the original AST back into a - // string and reparsing it. - if (rewrittenInsert.getChildren().size() == 3) { - // We have to move the SORT_BY over one, so grab it and then push it to the second slot, - // and put the where in the first slot - ASTNode sortBy = (ASTNode) rewrittenInsert.getChildren().get(2); - assert sortBy.getToken().getType() == HiveParser.TOK_SORTBY : - "Expected TOK_SORTBY to be third child of TOK_INSERT, but found " + sortBy.getName(); - rewrittenInsert.addChild(sortBy); - rewrittenInsert.setChild(2, where); - } else { - ASTNode select = (ASTNode) rewrittenInsert.getChildren().get(1); - assert select.getToken().getType() == HiveParser.TOK_SELECT : - "Expected TOK_SELECT to be second child of TOK_INSERT, but found " + select.getName(); - rewrittenInsert.addChild(where); - } - } - - if (updating() && setColExprs != null) { - patchProjectionForUpdate(rewrittenInsert, setColExprs); - } - - // Note: this will overwrite this.ctx with rewrittenCtx - rewrittenCtx.setEnableUnparse(false); - analyzeRewrittenTree(rewrittenTree, rewrittenCtx); - - updateOutputs(mTable); - - if (updating()) { - setUpAccessControlInfoForUpdate(mTable, setCols); - - // Add the setRCols to the input list - for (String colName : setRCols) { - if (columnAccessInfo != null) { //assuming this means we are not doing Auth - columnAccessInfo.add(Table.getCompleteName(mTable.getDbName(), mTable.getTableName()), - colName); - } - } - } - } - - private boolean tryMetadataUpdate(ASTNode tree, Table table, ASTNode tabNameNode) throws SemanticException { - // A feature flag on Hive to perform metadata delete on the source table. - if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_DELETE)) { - return false; - } - TableName tableName = getQualifiedTableName(tabNameNode); - if (!deleting() || table.getStorageHandler() == null) { - return false; - } - int whereIndex = 1; - List children = tree.getChildren(); - if (children.size() <= whereIndex) { - return false; - } - ASTNode whereNode = (ASTNode) children.get(whereIndex); - String whereClause = ctx.getTokenRewriteStream().toString( - whereNode.getChild(0).getTokenStartIndex(), whereNode.getChild(0).getTokenStopIndex()); - StringBuilder sb = new StringBuilder("select * from ").append(getFullTableNameForSQL(tabNameNode)) - .append(" where ").append(whereClause); - Context context = new Context(conf); - ASTNode rewrittenTree; - try { - rewrittenTree = ParseUtils.parse(sb.toString(), context); - } catch (ParseException pe) { - throw new SemanticException(pe); - } - BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, rewrittenTree); - sem.analyze(rewrittenTree, context); - - Map topOps = sem.getParseContext().getTopOps(); - if (!topOps.containsKey(table.getTableName())) { - return false; - } - ExprNodeGenericFuncDesc hiveFilter = topOps.get(table.getTableName()).getConf().getFilterExpr(); - if (hiveFilter == null) { - return false; - } - ConvertAstToSearchArg.Result result = ConvertAstToSearchArg.createSearchArgument(ctx.getConf(), hiveFilter); - if (result.isPartial()) { - return false; - } - SearchArgument sarg = result.getSearchArgument(); - if (!table.getStorageHandler().canPerformMetadataDelete(table, tableName.getTableMetaRef(), sarg)) { - return false; - } - - AlterTableExecuteSpec.DeleteMetadataSpec deleteMetadataSpec = - new AlterTableExecuteSpec.DeleteMetadataSpec(tableName.getTableMetaRef(), sarg); - AlterTableExecuteSpec executeSpec = - new AlterTableExecuteSpec<>(AlterTableExecuteSpec.ExecuteOperationType.DELETE_METADATA, deleteMetadataSpec); - AlterTableExecuteDesc desc = new AlterTableExecuteDesc(tableName, null, executeSpec); - DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), desc); - rootTasks = Collections.singletonList(TaskFactory.get(ddlWork)); - inputs = sem.getInputs(); - outputs = sem.getOutputs(); - updateOutputs(table); - return true; - } - - private boolean updating() { - return operation == Context.Operation.UPDATE; - } - private boolean deleting() { - return operation == Context.Operation.DELETE; - } - - @Override - protected boolean enableColumnStatsCollecting() { - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateSemanticAnalyzer.java new file mode 100644 index 000000000000..e140bb320b98 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateSemanticAnalyzer.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse; + +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.rewrite.RewriterFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.UpdateStatement; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class UpdateSemanticAnalyzer extends RewriteSemanticAnalyzer { + + public UpdateSemanticAnalyzer(QueryState queryState, RewriterFactory rewriterFactory) + throws SemanticException { + super(queryState, rewriterFactory); + } + + @Override + protected ASTNode getTargetTableNode(ASTNode tree) { + // The first child should be the table we are updating / deleting from + ASTNode tabName = (ASTNode) tree.getChild(0); + assert tabName.getToken().getType() == HiveParser.TOK_TABNAME : + "Expected tablename as first child of " + Context.Operation.UPDATE + " but found " + tabName.getName(); + return tabName; + } + + @Override + protected void analyze(ASTNode tree, Table table, ASTNode tableName) throws SemanticException { + List children = tree.getChildren(); + + ASTNode where = null; + int whereIndex = 2; + if (children.size() > whereIndex) { + where = (ASTNode) children.get(whereIndex); + assert where.getToken().getType() == HiveParser.TOK_WHERE : + "Expected where clause, but found " + where.getName(); + } + +// TOK_UPDATE_TABLE +// TOK_TABNAME +// ... +// TOK_SET_COLUMNS_CLAUSE <- The set list from update should be the second child (index 1) + assert children.size() >= 2 : "Expected update token to have at least two children"; + ASTNode setClause = (ASTNode) children.get(1); + // Must be deterministic order set for consistent q-test output across Java versions (HIVE-9239) + Set setRCols = new LinkedHashSet<>(); + Map setCols = collectSetColumnsAndExpressions(setClause, setRCols, table); + Map colNameToDefaultConstraint = getColNameToDefaultValueMap(table); + + rewriteAndAnalyze(new UpdateStatement(table, where, setClause, setCols, colNameToDefaultConstraint), null); + + updateOutputs(table); + setUpAccessControlInfoForUpdate(table, setCols); + + // Add the setRCols to the input list + if (columnAccessInfo == null) { //assuming this means we are not doing Auth + return; + } + + for (String colName : setRCols) { + columnAccessInfo.add(Table.getCompleteName(table.getDbName(), table.getTableName()), colName); + } + } + + @Override + protected boolean enableColumnStatsCollecting() { + return false; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java new file mode 100644 index 000000000000..f5fbeaf0449b --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.antlr.runtime.tree.Tree; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.VirtualColumn; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; + +public class CopyOnWriteDeleteRewriter implements Rewriter { + + private final HiveConf conf; + protected final SqlGeneratorFactory sqlGeneratorFactory; + private final COWWithClauseBuilder cowWithClauseBuilder; + + public CopyOnWriteDeleteRewriter( + HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, COWWithClauseBuilder cowWithClauseBuilder) { + this.sqlGeneratorFactory = sqlGeneratorFactory; + this.conf = conf; + this.cowWithClauseBuilder = cowWithClauseBuilder; + } + + @Override + public ParseUtils.ReparseResult rewrite(Context context, DeleteStatement deleteBlock) + throws SemanticException { + + Tree wherePredicateNode = deleteBlock.getWhereTree().getChild(0); + String whereClause = context.getTokenRewriteStream().toString( + wherePredicateNode.getTokenStartIndex(), wherePredicateNode.getTokenStopIndex()); + String filePathCol = HiveUtils.unparseIdentifier(VirtualColumn.FILE_PATH.getName(), conf); + + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + + cowWithClauseBuilder.appendWith(sqlGenerator, filePathCol, whereClause); + + sqlGenerator.append("insert into table "); + sqlGenerator.append(sqlGenerator.getTargetTableFullName()); + sqlGenerator.appendPartitionColsOfTarget(); + + sqlGenerator.append(" select "); + sqlGenerator.appendAcidSelectColumns(Context.Operation.DELETE); + sqlGenerator.removeLastChar(); + + sqlGenerator.append(" from "); + sqlGenerator.append(sqlGenerator.getTargetTableFullName()); + + // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. + sqlGenerator.append("\nwhere NOT (").append(whereClause).append(")"); + sqlGenerator.append("\n"); + // Add the file path filter that matches the delete condition. + sqlGenerator.append("AND ").append(filePathCol); + sqlGenerator.append(" IN ( select ").append(filePathCol).append(" from t )"); + sqlGenerator.append("\nunion all"); + sqlGenerator.append("\nselect * from t"); + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + + rewrittenCtx.setOperation(Context.Operation.DELETE); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE); + + // Note: this will overwrite this.ctx with rewrittenCtx + rewrittenCtx.setEnableUnparse(false); + return rr; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java new file mode 100644 index 000000000000..dc53a0304970 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.antlr.runtime.tree.Tree; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.VirtualColumn; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.CalcitePlanner; +import org.apache.hadoop.hive.ql.parse.HiveParser; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SetClausePatcher; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class CopyOnWriteUpdateRewriter implements Rewriter { + + private final HiveConf conf; + private final SqlGeneratorFactory sqlGeneratorFactory; + private final COWWithClauseBuilder cowWithClauseBuilder; + private final SetClausePatcher setClausePatcher; + + + public CopyOnWriteUpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, + COWWithClauseBuilder cowWithClauseBuilder, SetClausePatcher setClausePatcher) { + this.conf = conf; + this.sqlGeneratorFactory = sqlGeneratorFactory; + this.cowWithClauseBuilder = cowWithClauseBuilder; + this.setClausePatcher = setClausePatcher; + } + + @Override + public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateBlock) + throws SemanticException { + + Tree wherePredicateNode = updateBlock.getWhereTree().getChild(0); + String whereClause = context.getTokenRewriteStream().toString( + wherePredicateNode.getTokenStartIndex(), wherePredicateNode.getTokenStopIndex()); + String filePathCol = HiveUtils.unparseIdentifier(VirtualColumn.FILE_PATH.getName(), conf); + + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + + cowWithClauseBuilder.appendWith(sqlGenerator, filePathCol, whereClause); + + sqlGenerator.append("insert into table "); + sqlGenerator.appendTargetTableName(); + sqlGenerator.appendPartitionColsOfTarget(); + + int columnOffset = sqlGenerator.getDeleteValues(Context.Operation.UPDATE).size(); + sqlGenerator.append(" select "); + sqlGenerator.appendAcidSelectColumns(Context.Operation.UPDATE); + sqlGenerator.removeLastChar(); + + Map setColExprs = new HashMap<>(updateBlock.getSetCols().size()); + List nonPartCols = updateBlock.getTargetTable().getCols(); + for (int i = 0; i < nonPartCols.size(); i++) { + sqlGenerator.append(','); + String name = nonPartCols.get(i).getName(); + ASTNode setCol = updateBlock.getSetCols().get(name); + String identifier = HiveUtils.unparseIdentifier(name, this.conf); + sqlGenerator.append(identifier); + sqlGenerator.append(" AS ").append(identifier); + if (setCol != null) { + // This is one of the columns we're setting, record it's position so we can come back + // later and patch it up. + // Add one to the index because the select has the ROW__ID as the first column. + setColExprs.put(columnOffset + i, setCol); + } + } + + sqlGenerator.append(" from "); + sqlGenerator.appendTargetTableName(); + + if (updateBlock.getWhereTree() != null) { + sqlGenerator.append("\nwhere "); + sqlGenerator.append(whereClause); + sqlGenerator.append("\nunion all"); + sqlGenerator.append("\nselect "); + sqlGenerator.appendAcidSelectColumns(Context.Operation.DELETE); + sqlGenerator.removeLastChar(); + sqlGenerator.append(" from "); + sqlGenerator.appendTargetTableName(); + // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. + sqlGenerator.append("\nwhere NOT (").append(whereClause).append(")"); + sqlGenerator.append("\n").indent(); + // Add the file path filter that matches the delete condition. + sqlGenerator.append("AND ").append(filePathCol); + sqlGenerator.append(" IN ( select ").append(filePathCol).append(" from t )"); + sqlGenerator.append("\nunion all"); + sqlGenerator.append("\nselect * from t"); + } + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + + ASTNode rewrittenInsert = (ASTNode) new CalcitePlanner.ASTSearcher().simpleBreadthFirstSearch( + rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_UNIONALL).getChild(0).getChild(0) + .getChild(1); + + rewrittenCtx.setOperation(Context.Operation.UPDATE); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); + + setClausePatcher.patchProjectionForUpdate(rewrittenInsert, setColExprs); + + // Note: this will overwrite this.ctx with rewrittenCtx + rewrittenCtx.setEnableUnparse(false); + return rr; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java new file mode 100644 index 000000000000..505760e6d969 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.WhereClausePatcher; + +public class DeleteRewriter implements Rewriter { + + protected final SqlGeneratorFactory sqlGeneratorFactory; + private final WhereClausePatcher whereClausePatcher; + + public DeleteRewriter(SqlGeneratorFactory sqlGeneratorFactory, WhereClausePatcher whereClausePatcher) { + this.sqlGeneratorFactory = sqlGeneratorFactory; + this.whereClausePatcher = whereClausePatcher; + } + + @Override + public ParseUtils.ReparseResult rewrite(Context context, DeleteStatement deleteBlock) + throws SemanticException { + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + + sqlGenerator.append("insert into table "); + sqlGenerator.append(sqlGenerator.getTargetTableFullName()); + sqlGenerator.appendPartitionColsOfTarget(); + + sqlGenerator.append(" select "); + sqlGenerator.appendAcidSelectColumns(Context.Operation.DELETE); + sqlGenerator.removeLastChar(); + sqlGenerator.append(" from "); + sqlGenerator.append(sqlGenerator.getTargetTableFullName()); + + sqlGenerator.appendSortBy(sqlGenerator.getSortKeys()); + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + + ASTNode rewrittenInsert = (ASTNode)rewrittenTree.getChildren().get(1); + rewrittenCtx.setOperation(Context.Operation.DELETE); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE); + + if (deleteBlock.getWhereTree() != null) { + whereClausePatcher.patch(rewrittenInsert, deleteBlock.getWhereTree()); + } + + // Note: this will overwrite this.ctx with rewrittenCtx + rewrittenCtx.setEnableUnparse(false); + return rr; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java new file mode 100644 index 000000000000..7cb7cf990030 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.WhereClausePatcher; + +import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.DELETE_PREFIX; + +public class DeleteRewriterFactory implements RewriterFactory { + protected final HiveConf conf; + + public DeleteRewriterFactory(HiveConf conf) { + this.conf = conf; + } + + public Rewriter createRewriter(Table table, String targetTableFullName, String subQueryAlias) { + boolean copyOnWriteMode = false; + HiveStorageHandler storageHandler = table.getStorageHandler(); + if (storageHandler != null) { + copyOnWriteMode = storageHandler.shouldOverwrite(table, Context.Operation.DELETE); + } + + SqlGeneratorFactory sqlGeneratorFactory = new SqlGeneratorFactory( + table, targetTableFullName, conf, null, DELETE_PREFIX); + + if (copyOnWriteMode) { + return new CopyOnWriteDeleteRewriter(conf, sqlGeneratorFactory, new COWWithClauseBuilder()); + } else { + return new DeleteRewriter(sqlGeneratorFactory, new WhereClausePatcher()); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteStatement.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteStatement.java new file mode 100644 index 000000000000..097514738970 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteStatement.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.ASTNode; + +public class DeleteStatement { + private final Table targetTable; + private final ASTNode whereTree; + + public DeleteStatement(Table targetTable, ASTNode whereTree) { + this.targetTable = targetTable; + this.whereTree = whereTree; + } + + public Table getTargetTable() { + return targetTable; + } + + public ASTNode getWhereTree() { + return whereTree; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java new file mode 100644 index 000000000000..ed2c853a4092 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.StorageFormat; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; +import org.apache.hadoop.hive.ql.session.SessionState; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.apache.commons.lang3.StringUtils.isNotBlank; + +public class MergeRewriter implements Rewriter, MergeStatement.DestClausePrefixSetter { + + private final Hive db; + protected final HiveConf conf; + private final SqlGeneratorFactory sqlGeneratorFactory; + + public MergeRewriter(Hive db, HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { + this.db = db; + this.conf = conf; + this.sqlGeneratorFactory = sqlGeneratorFactory; + } + + @Override + public ParseUtils.ReparseResult rewrite(Context ctx, MergeStatement mergeStatement) throws SemanticException { + + setOperation(ctx); + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + handleSource(mergeStatement.hasWhenNotMatchedInsertClause(), mergeStatement.getSourceAlias(), + mergeStatement.getOnClauseAsText(), sqlGenerator); + + MergeStatement.MergeSqlGenerator mergeSqlGenerator = createMergeSqlGenerator(mergeStatement, sqlGenerator); + for (MergeStatement.WhenClause whenClause : mergeStatement.getWhenClauses()) { + whenClause.toSql(mergeSqlGenerator); + } + + boolean validateCardinalityViolation = mergeStatement.shouldValidateCardinalityViolation(conf); + if (validateCardinalityViolation) { + handleCardinalityViolation(mergeStatement.getTargetAlias(), mergeStatement.getOnClauseAsText(), sqlGenerator); + } + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + setOperation(rewrittenCtx); + + //set dest name mapping on new context; 1st child is TOK_FROM + int insClauseIdx = 1; + for (MergeStatement.WhenClause whenClause : mergeStatement.getWhenClauses()) { + insClauseIdx += whenClause.addDestNamePrefixOfInsert(this, insClauseIdx, rewrittenCtx); + } + + if (validateCardinalityViolation) { + //here means the last branch of the multi-insert is Cardinality Validation + rewrittenCtx.addDestNamePrefix(rewrittenTree.getChildCount() - 1, Context.DestClausePrefix.INSERT); + } + + return rr; + } + + protected MergeWhenClauseSqlGenerator createMergeSqlGenerator( + MergeStatement mergeStatement, MultiInsertSqlGenerator sqlGenerator) { + return new MergeWhenClauseSqlGenerator(conf, sqlGenerator, mergeStatement); + } + + private void handleSource(boolean hasWhenNotMatchedClause, String sourceAlias, String onClauseAsText, + MultiInsertSqlGenerator sqlGenerator) { + sqlGenerator.append("FROM\n"); + sqlGenerator.append("(SELECT "); + sqlGenerator.appendAcidSelectColumns(Context.Operation.MERGE); + sqlGenerator.appendAllColsOfTargetTable(); + sqlGenerator.append(" FROM ").appendTargetTableName().append(") "); + sqlGenerator.appendSubQueryAlias(); + sqlGenerator.append('\n'); + sqlGenerator.indent().append(hasWhenNotMatchedClause ? "RIGHT OUTER JOIN" : "INNER JOIN").append("\n"); + sqlGenerator.indent().append(sourceAlias); + sqlGenerator.append('\n'); + sqlGenerator.indent().append("ON ").append(onClauseAsText).append('\n'); + } + + private void handleCardinalityViolation( + String targetAlias, String onClauseAsString, MultiInsertSqlGenerator sqlGenerator) + throws SemanticException { + //this is a tmp table and thus Session scoped and acid requires SQL statement to be serial in a + // given session, i.e. the name can be fixed across all invocations + String tableName = "merge_tmp_table"; + List sortKeys = sqlGenerator.getSortKeys(); + sqlGenerator.append("INSERT INTO ").append(tableName) + .append("\n SELECT cardinality_violation(") + .append(StringUtils.join(sortKeys, ",")); + sqlGenerator.appendPartColsOfTargetTableWithComma(targetAlias); + + sqlGenerator.append(")\n WHERE ").append(onClauseAsString) + .append(" GROUP BY ").append(StringUtils.join(sortKeys, ",")); + + sqlGenerator.appendPartColsOfTargetTableWithComma(targetAlias); + + sqlGenerator.append(" HAVING count(*) > 1"); + //say table T has partition p, we are generating + //select cardinality_violation(ROW_ID, p) WHERE ... GROUP BY ROW__ID, p + //the Group By args are passed to cardinality_violation to add the violating value to the error msg + try { + if (null == db.getTable(tableName, false)) { + StorageFormat format = new StorageFormat(conf); + format.processStorageFormat("TextFile"); + Table table = db.newTable(tableName); + table.setSerializationLib(format.getSerde()); + List fields = new ArrayList<>(); + fields.add(new FieldSchema("val", "int", null)); + table.setFields(fields); + table.setDataLocation(Warehouse.getDnsPath(new Path(SessionState.get().getTempTableSpace(), + tableName), conf)); + table.getTTable().setTemporary(true); + table.setStoredAsSubDirectories(false); + table.setInputFormatClass(format.getInputFormat()); + table.setOutputFormatClass(format.getOutputFormat()); + db.createTable(table, true); + } + } catch (HiveException | MetaException e) { + throw new SemanticException(e.getMessage(), e); + } + } + + protected void setOperation(Context context) { + context.setOperation(Context.Operation.MERGE); + } + + protected static class MergeWhenClauseSqlGenerator implements MergeStatement.MergeSqlGenerator { + + private final HiveConf conf; + protected final MultiInsertSqlGenerator sqlGenerator; + protected final MergeStatement mergeStatement; + protected String hintStr; + + MergeWhenClauseSqlGenerator(HiveConf conf, MultiInsertSqlGenerator sqlGenerator, MergeStatement mergeStatement) { + this.conf = conf; + this.sqlGenerator = sqlGenerator; + this.mergeStatement = mergeStatement; + this.hintStr = mergeStatement.getHintStr(); + } + + @Override + public void appendWhenNotMatchedInsertClause(MergeStatement.InsertClause insertClause) { + sqlGenerator.append("INSERT INTO ").append(mergeStatement.getTargetName()); + if (insertClause.getColumnListText() != null) { + sqlGenerator.append(' ').append(insertClause.getColumnListText()); + } + + sqlGenerator.append(" -- insert clause\n SELECT "); + if (isNotBlank(hintStr)) { + sqlGenerator.append(hintStr); + hintStr = null; + } + + sqlGenerator.append(insertClause.getValuesClause()).append("\n WHERE ").append(insertClause.getPredicate()); + + if (insertClause.getExtraPredicate() != null) { + //we have WHEN NOT MATCHED AND THEN INSERT + sqlGenerator.append(" AND ").append(insertClause.getExtraPredicate()); + } + sqlGenerator.append('\n'); + } + + + @Override + public void appendWhenMatchedUpdateClause(MergeStatement.UpdateClause updateClause) { + Table targetTable = mergeStatement.getTargetTable(); + String targetAlias = mergeStatement.getTargetAlias(); + String onClauseAsString = mergeStatement.getOnClauseAsText(); + + sqlGenerator.append(" -- update clause").append("\n"); + List valuesAndAcidSortKeys = new ArrayList<>( + targetTable.getCols().size() + targetTable.getPartCols().size() + 1); + valuesAndAcidSortKeys.addAll(sqlGenerator.getSortKeys()); + addValues(targetTable, targetAlias, updateClause.getNewValuesMap(), valuesAndAcidSortKeys); + sqlGenerator.appendInsertBranch(hintStr, valuesAndAcidSortKeys); + hintStr = null; + + addWhereClauseOfUpdate( + onClauseAsString, updateClause.getExtraPredicate(), updateClause.getDeleteExtraPredicate(), sqlGenerator); + + sqlGenerator.appendSortBy(sqlGenerator.getSortKeys()); + } + + protected void addValues(Table targetTable, String targetAlias, Map newValues, + List values) { + for (FieldSchema fieldSchema : targetTable.getCols()) { + if (newValues.containsKey(fieldSchema.getName())) { + values.add(newValues.get(fieldSchema.getName())); + } else { + values.add( + String.format("%s.%s", targetAlias, HiveUtils.unparseIdentifier(fieldSchema.getName(), conf))); + } + } + + targetTable.getPartCols().forEach(fieldSchema -> + values.add( + String.format("%s.%s", targetAlias, HiveUtils.unparseIdentifier(fieldSchema.getName(), conf)))); + } + + protected void addWhereClauseOfUpdate(String onClauseAsString, String extraPredicate, String deleteExtraPredicate, + MultiInsertSqlGenerator sqlGenerator) { + sqlGenerator.indent().append("WHERE ").append(onClauseAsString); + if (extraPredicate != null) { + //we have WHEN MATCHED AND THEN DELETE + sqlGenerator.append(" AND ").append(extraPredicate); + } + if (deleteExtraPredicate != null) { + sqlGenerator.append(" AND NOT(").append(deleteExtraPredicate).append(")"); + } + } + + @Override + public void appendWhenMatchedDeleteClause(MergeStatement.DeleteClause deleteClause) { + handleWhenMatchedDelete(mergeStatement.getOnClauseAsText(), + deleteClause.getExtraPredicate(), deleteClause.getUpdateExtraPredicate(), hintStr, sqlGenerator); + hintStr = null; + } + + protected void handleWhenMatchedDelete(String onClauseAsString, String extraPredicate, String updateExtraPredicate, + String hintStr, MultiInsertSqlGenerator sqlGenerator) { + sqlGenerator.appendDeleteBranch(hintStr); + + sqlGenerator.indent().append("WHERE ").append(onClauseAsString); + if (extraPredicate != null) { + //we have WHEN MATCHED AND THEN DELETE + sqlGenerator.append(" AND ").append(extraPredicate); + } + if (updateExtraPredicate != null) { + sqlGenerator.append(" AND NOT(").append(updateExtraPredicate).append(")"); + } + sqlGenerator.append("\n").indent(); + sqlGenerator.appendSortKeys(); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java new file mode 100644 index 000000000000..145639bbdf54 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; + +public class MergeRewriterFactory implements RewriterFactory { + private final Hive db; + private final HiveConf conf; + + public MergeRewriterFactory(HiveConf conf) throws SemanticException { + try { + this.db = Hive.get(conf); + } catch (HiveException e) { + throw new SemanticException(e); + } + this.conf = conf; + } + + @Override + public Rewriter createRewriter(Table table, String targetTableFullName, String subQueryAlias) + throws SemanticException { + boolean splitUpdate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE); + boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); + if (nonNativeAcid && !splitUpdate) { + throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); + } + + SqlGeneratorFactory sqlGeneratorFactory = new SqlGeneratorFactory( + table, + targetTableFullName, + conf, + subQueryAlias, + StringUtils.EMPTY); + + if (splitUpdate) { + return new SplitMergeRewriter(db, conf, sqlGeneratorFactory); + } + return new MergeRewriter(db, conf, sqlGeneratorFactory); + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java new file mode 100644 index 000000000000..64d9c384161c --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class MergeStatement { + protected static final Logger LOG = LoggerFactory.getLogger(MergeStatement.class); + + public static MergeStatementBuilder withTarget(Table targetTable, String targetName, String targetAlias) { + return new MergeStatementBuilder(targetTable, targetName, targetAlias); + } + + public static class MergeStatementBuilder { + private final Table targetTable; + private final String targetName; + private final String targetAlias; + private String sourceName; + private String sourceAlias; + private String onClauseAsText; + private String hintStr; + private final List whenClauses; + + private MergeStatementBuilder(Table targetTable, String targetName, String targetAlias) { + this.targetTable = targetTable; + this.targetName = targetName; + this.targetAlias = targetAlias; + whenClauses = new ArrayList<>(3); + } + + public MergeStatementBuilder sourceName(String sourceName) { + this.sourceName = sourceName; + return this; + } + public MergeStatementBuilder sourceAlias(String sourceAlias) { + this.sourceAlias = sourceAlias; + return this; + } + public MergeStatementBuilder onClauseAsText(String onClauseAsText) { + this.onClauseAsText = onClauseAsText; + return this; + } + public MergeStatementBuilder hintStr(String hintStr) { + this.hintStr = hintStr; + return this; + } + + public MergeStatementBuilder addWhenClause(WhenClause whenClause) { + whenClauses.add(whenClause); + return this; + } + + public MergeStatement build() { + return new MergeStatement(targetTable, targetName, targetAlias, sourceName, sourceAlias, onClauseAsText, hintStr, + Collections.unmodifiableList(whenClauses)); + } + } + + private final Table targetTable; + private final String targetName; + private final String targetAlias; + private final String sourceName; + private final String sourceAlias; + private final String onClauseAsText; + private final String hintStr; + private final List whenClauses; + + private MergeStatement(Table targetTable, String targetName, String targetAlias, String sourceName, String sourceAlias, + String onClauseAsText, String hintStr, List whenClauses) { + this.targetTable = targetTable; + this.targetName = targetName; + this.targetAlias = targetAlias; + this.sourceName = sourceName; + this.sourceAlias = sourceAlias; + this.onClauseAsText = onClauseAsText; + this.hintStr = hintStr; + this.whenClauses = whenClauses; + } + + public Table getTargetTable() { + return targetTable; + } + + public String getTargetName() { + return targetName; + } + + public String getTargetAlias() { + return targetAlias; + } + + public String getSourceName() { + return sourceName; + } + + public String getSourceAlias() { + return sourceAlias; + } + + public String getOnClauseAsText() { + return onClauseAsText; + } + + public String getHintStr() { + return hintStr; + } + + public List getWhenClauses() { + return whenClauses; + } + + public boolean hasWhenNotMatchedInsertClause() { + for (WhenClause whenClause : whenClauses) { + if (whenClause instanceof InsertClause) { + return true; + } + } + + return false; + } + + /** + * Per SQL Spec ISO/IEC 9075-2:2011(E) Section 14.2 under "General Rules" Item 6/Subitem a/Subitem 2/Subitem B, + * an error should be raised if > 1 row of "source" matches the same row in "target". + * This should not affect the runtime of the query as it's running in parallel with other + * branches of the multi-insert. It won't actually write any data to merge_tmp_table since the + * cardinality_violation() UDF throws an error whenever it's called killing the query + * @return true if another Insert clause was added + */ + public boolean shouldValidateCardinalityViolation(HiveConf conf) { + if (!conf.getBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK)) { + LOG.info("Merge statement cardinality violation check is disabled: {}", + HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK.varname); + return false; + } + //if no update or delete in Merge, there is no need to do cardinality check + boolean onlyHaveWhenNotMatchedClause = whenClauses.size() == 1 && whenClauses.get(0) instanceof InsertClause; + return !onlyHaveWhenNotMatchedClause; + } + + protected abstract static class WhenClause { + private final String extraPredicate; + + public WhenClause(String extraPredicate) { + this.extraPredicate = extraPredicate; + } + + public String getExtraPredicate() { + return extraPredicate; + } + + public abstract void toSql(MergeSqlGenerator sqlGenerator); + public abstract int addDestNamePrefixOfInsert( + DestClausePrefixSetter destClausePrefixSetter, int pos, Context context); + } + + public static class InsertClause extends WhenClause { + private final String columnListText; + private final String valuesClause; + private final String predicate; + + + public InsertClause(String columnListText, String valuesClause, String predicate, String extraPredicate) { + super(extraPredicate); + this.predicate = predicate; + this.columnListText = columnListText; + this.valuesClause = valuesClause; + } + + public String getColumnListText() { + return columnListText; + } + + public String getValuesClause() { + return valuesClause; + } + + public String getPredicate() { + return predicate; + } + + @Override + public void toSql(MergeSqlGenerator sqlGenerator) { + sqlGenerator.appendWhenNotMatchedInsertClause(this); + } + + @Override + public int addDestNamePrefixOfInsert(DestClausePrefixSetter setter, int pos, Context context) { + return setter.addDestNamePrefixOfInsert(pos, context); + } + } + + public static class UpdateClause extends WhenClause { + private final Map newValuesMap; + private final String deleteExtraPredicate; + + public UpdateClause(String extraPredicate, String deleteExtraPredicate, Map newValuesMap) { + super(extraPredicate); + this.newValuesMap = newValuesMap; + this.deleteExtraPredicate = deleteExtraPredicate; + } + + public Map getNewValuesMap() { + return newValuesMap; + } + + public String getDeleteExtraPredicate() { + return deleteExtraPredicate; + } + + @Override + public void toSql(MergeSqlGenerator sqlGenerator) { + sqlGenerator.appendWhenMatchedUpdateClause(this); + } + + @Override + public int addDestNamePrefixOfInsert(DestClausePrefixSetter setter, int pos, Context context) { + return setter.addDestNamePrefixOfUpdate(pos, context); + } + } + + public static class DeleteClause extends WhenClause { + private final String updateExtraPredicate; + + public DeleteClause(String extraPredicate, String updateExtraPredicate) { + super(extraPredicate); + this.updateExtraPredicate = updateExtraPredicate; + } + + public String getUpdateExtraPredicate() { + return updateExtraPredicate; + } + + @Override + public void toSql(MergeSqlGenerator sqlGenerator) { + sqlGenerator.appendWhenMatchedDeleteClause(this); + } + + @Override + public int addDestNamePrefixOfInsert(DestClausePrefixSetter setter, int pos, Context context) { + return setter.addDestNamePrefixOfDelete(pos, context); + } + } + + public interface MergeSqlGenerator { + void appendWhenNotMatchedInsertClause(InsertClause insertClause); + void appendWhenMatchedUpdateClause(UpdateClause updateClause); + void appendWhenMatchedDeleteClause(DeleteClause deleteClause); + } + + public interface DestClausePrefixSetter { + default int addDestNamePrefixOfInsert(int pos, Context context) { + context.addDestNamePrefix(pos, Context.DestClausePrefix.INSERT); + return 1; + } + + default int addDestNamePrefixOfUpdate(int pos, Context context) { + context.addDestNamePrefix(pos, Context.DestClausePrefix.UPDATE); + return 1; + } + + default int addDestNamePrefixOfDelete(int pos, Context context) { + context.addDestNamePrefix(pos, Context.DestClausePrefix.DELETE); + return 1; + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/Rewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/Rewriter.java new file mode 100644 index 000000000000..9fd2a7234c1a --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/Rewriter.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; + +public interface Rewriter { + ParseUtils.ReparseResult rewrite(Context context, T rewriteData) throws SemanticException; +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/RewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/RewriterFactory.java new file mode 100644 index 000000000000..14c2faf0d294 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/RewriterFactory.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.SemanticException; + +public interface RewriterFactory { + Rewriter createRewriter(Table table, String targetTableFullName, String subQueryAlias) throws SemanticException; +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitMergeRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitMergeRewriter.java new file mode 100644 index 000000000000..5ba8bd0193e9 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitMergeRewriter.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; + +import java.util.ArrayList; +import java.util.List; + +import static java.util.Arrays.asList; + +public class SplitMergeRewriter extends MergeRewriter { + + public SplitMergeRewriter(Hive db, HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { + super(db, conf, sqlGeneratorFactory); + } + + @Override + protected MergeWhenClauseSqlGenerator createMergeSqlGenerator( + MergeStatement mergeStatement, MultiInsertSqlGenerator sqlGenerator) { + return new SplitMergeWhenClauseSqlGenerator(conf, sqlGenerator, mergeStatement); + } + + static class SplitMergeWhenClauseSqlGenerator extends MergeWhenClauseSqlGenerator { + + SplitMergeWhenClauseSqlGenerator( + HiveConf conf, MultiInsertSqlGenerator sqlGenerator, MergeStatement mergeStatement) { + super(conf, sqlGenerator, mergeStatement); + } + + @Override + public void appendWhenMatchedUpdateClause(MergeStatement.UpdateClause updateClause) { + Table targetTable = mergeStatement.getTargetTable(); + String targetAlias = mergeStatement.getTargetAlias(); + String onClauseAsString = mergeStatement.getOnClauseAsText(); + + sqlGenerator.append(" -- update clause (insert part)\n"); + List values = new ArrayList<>(targetTable.getCols().size() + targetTable.getPartCols().size()); + addValues(targetTable, targetAlias, updateClause.getNewValuesMap(), values); + sqlGenerator.appendInsertBranch(hintStr, values); + hintStr = null; + + addWhereClauseOfUpdate( + onClauseAsString, updateClause.getExtraPredicate(), updateClause.getDeleteExtraPredicate(), sqlGenerator); + + sqlGenerator.append("\n"); + + sqlGenerator.append(" -- update clause (delete part)\n"); + handleWhenMatchedDelete(onClauseAsString, + updateClause.getExtraPredicate(), updateClause.getDeleteExtraPredicate(), hintStr, sqlGenerator); + } + } + + @Override + public void setOperation(Context context) { + context.setOperation(Context.Operation.MERGE, true); + } + + @Override + public int addDestNamePrefixOfUpdate(int pos, Context context) { + context.addDestNamePrefix(pos, Context.DestClausePrefix.INSERT); + context.addDeleteOfUpdateDestNamePrefix(pos + 1, Context.DestClausePrefix.DELETE); + return 2; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java new file mode 100644 index 000000000000..a79ec57fdb25 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.CalcitePlanner; +import org.apache.hadoop.hive.ql.parse.HiveParser; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SetClausePatcher; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SplitUpdateRewriter implements Rewriter { + + private static final Context.Operation OPERATION = Context.Operation.UPDATE; + + private final HiveConf conf; + protected final SqlGeneratorFactory sqlGeneratorFactory; + private final SetClausePatcher setClausePatcher; + + public SplitUpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, SetClausePatcher setClausePatcher) { + this.conf = conf; + this.sqlGeneratorFactory = sqlGeneratorFactory; + this.setClausePatcher = setClausePatcher; + } + + @Override + public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateBlock) + throws SemanticException { + Map setColExprs = new HashMap<>(updateBlock.getSetClauseTree().getChildCount()); + + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + + sqlGenerator.append("FROM\n"); + sqlGenerator.append("(SELECT "); + + sqlGenerator.appendAcidSelectColumns(OPERATION); + List deleteValues = sqlGenerator.getDeleteValues(OPERATION); + int columnOffset = deleteValues.size(); + + List insertValues = new ArrayList<>(updateBlock.getTargetTable().getCols().size()); + boolean first = true; + + List nonPartCols = updateBlock.getTargetTable().getCols(); + for (int i = 0; i < nonPartCols.size(); i++) { + if (first) { + first = false; + } else { + sqlGenerator.append(","); + } + + String name = nonPartCols.get(i).getName(); + ASTNode setCol = updateBlock.getSetCols().get(name); + String identifier = HiveUtils.unparseIdentifier(name, this.conf); + + if (setCol != null) { + if (setCol.getType() == HiveParser.TOK_TABLE_OR_COL && + setCol.getChildCount() == 1 && setCol.getChild(0).getType() == HiveParser.TOK_DEFAULT_VALUE) { + sqlGenerator.append(updateBlock.getColNameToDefaultConstraint().get(name)); + } else { + sqlGenerator.append(identifier); + // This is one of the columns we're setting, record it's position so we can come back + // later and patch it up. 0th is ROW_ID + setColExprs.put(i + columnOffset, setCol); + } + } else { + sqlGenerator.append(identifier); + } + sqlGenerator.append(" AS "); + sqlGenerator.append(identifier); + + insertValues.add(sqlGenerator.qualify(identifier)); + } + if (updateBlock.getTargetTable().getPartCols() != null) { + updateBlock.getTargetTable().getPartCols().forEach( + fieldSchema -> insertValues.add(sqlGenerator.qualify(HiveUtils.unparseIdentifier(fieldSchema.getName(), conf)))); + } + + sqlGenerator.append(" FROM ").append(sqlGenerator.getTargetTableFullName()).append(") "); + sqlGenerator.appendSubQueryAlias().append("\n"); + + sqlGenerator.appendInsertBranch(null, insertValues); + sqlGenerator.appendInsertBranch(null, deleteValues); + + List sortKeys = sqlGenerator.getSortKeys(); + sqlGenerator.appendSortBy(sortKeys); + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + + ASTNode rewrittenInsert = new CalcitePlanner.ASTSearcher().simpleBreadthFirstSearch( + rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_INSERT); + + rewrittenCtx.setOperation(Context.Operation.UPDATE, true); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.INSERT); + rewrittenCtx.addDeleteOfUpdateDestNamePrefix(2, Context.DestClausePrefix.DELETE); + + if (updateBlock.getWhereTree() != null) { + rewrittenInsert.addChild(updateBlock.getWhereTree()); + } + + setClausePatcher.patchProjectionForUpdate(rewrittenInsert, setColExprs); + + // Note: this will overwrite this.ctx with rewrittenCtx + rewrittenCtx.setEnableUnparse(false); + + return rr; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java new file mode 100644 index 000000000000..1e7a5ec7c326 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SetClausePatcher; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.WhereClausePatcher; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class UpdateRewriter implements Rewriter { + + public static final Context.Operation OPERATION = Context.Operation.UPDATE; + + protected final HiveConf conf; + protected final SqlGeneratorFactory sqlGeneratorFactory; + private final WhereClausePatcher whereClausePatcher; + private final SetClausePatcher setClausePatcher; + + public UpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, + WhereClausePatcher whereClausePatcher, SetClausePatcher setClausePatcher) { + this.conf = conf; + this.sqlGeneratorFactory = sqlGeneratorFactory; + this.whereClausePatcher = whereClausePatcher; + this.setClausePatcher = setClausePatcher; + } + + @Override + public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateBlock) + throws SemanticException { + + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + + sqlGenerator.append("insert into table "); + sqlGenerator.appendTargetTableName(); + sqlGenerator.appendPartitionColsOfTarget(); + + int columnOffset = sqlGenerator.getDeleteValues(OPERATION).size(); + sqlGenerator.append(" select "); + sqlGenerator.appendAcidSelectColumns(OPERATION); + sqlGenerator.removeLastChar(); + + Map setColExprs = new HashMap<>(updateBlock.getSetCols().size()); + List nonPartCols = updateBlock.getTargetTable().getCols(); + for (int i = 0; i < nonPartCols.size(); i++) { + sqlGenerator.append(","); + String name = nonPartCols.get(i).getName(); + ASTNode setCol = updateBlock.getSetCols().get(name); + sqlGenerator.append(HiveUtils.unparseIdentifier(name, this.conf)); + if (setCol != null) { + // This is one of the columns we're setting, record it's position so we can come back + // later and patch it up. + // Add one to the index because the select has the ROW__ID as the first column. + setColExprs.put(columnOffset + i, setCol); + } + } + + sqlGenerator.append(" from "); + sqlGenerator.appendTargetTableName(); + + // Add a sort by clause so that the row ids come out in the correct order + sqlGenerator.appendSortBy(sqlGenerator.getSortKeys()); + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + ASTNode rewrittenTree = rr.rewrittenTree; + + ASTNode rewrittenInsert = (ASTNode) rewrittenTree.getChildren().get(1); + rewrittenCtx.setOperation(OPERATION); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); + + if (updateBlock.getWhereTree() != null) { + whereClausePatcher.patch(rewrittenInsert, updateBlock.getWhereTree()); + } + + setClausePatcher.patchProjectionForUpdate(rewrittenInsert, setColExprs); + + // Note: this will overwrite this.ctx with rewrittenCtx + rewrittenCtx.setEnableUnparse(false); + return rr; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java new file mode 100644 index 000000000000..d17d5e97b3fe --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SetClausePatcher; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.WhereClausePatcher; + +import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.DELETE_PREFIX; +import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.SUB_QUERY_ALIAS; + +public class UpdateRewriterFactory implements RewriterFactory { + protected final HiveConf conf; + + public UpdateRewriterFactory(HiveConf conf) { + this.conf = conf; + } + + public Rewriter createRewriter(Table table, String targetTableFullName, String subQueryAlias) + throws SemanticException { + boolean splitUpdate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE); + boolean copyOnWriteMode = false; + HiveStorageHandler storageHandler = table.getStorageHandler(); + if (storageHandler != null) { + copyOnWriteMode = storageHandler.shouldOverwrite(table, Context.Operation.UPDATE); + } + + SqlGeneratorFactory sqlGeneratorFactory = new SqlGeneratorFactory( + table, targetTableFullName, conf, splitUpdate && !copyOnWriteMode ? SUB_QUERY_ALIAS : null, DELETE_PREFIX); + + if (copyOnWriteMode) { + return new CopyOnWriteUpdateRewriter( + conf, sqlGeneratorFactory, new COWWithClauseBuilder(), new SetClausePatcher()); + } else if (splitUpdate) { + return new SplitUpdateRewriter(conf, sqlGeneratorFactory, new SetClausePatcher()); + } else { + if (AcidUtils.isNonNativeAcidTable(table, true)) { + throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); + } + return new UpdateRewriter(conf, sqlGeneratorFactory, new WhereClausePatcher(), new SetClausePatcher()); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateStatement.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateStatement.java new file mode 100644 index 000000000000..d7aeb883a86e --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateStatement.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite; + +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.ASTNode; + +import java.util.Map; + +public class UpdateStatement extends DeleteStatement { + + private final ASTNode setClauseTree; + private final Map setCols; + private final Map colNameToDefaultConstraint; + + public UpdateStatement(Table targetTable, ASTNode whereTree, ASTNode setClauseTree, Map setCols, + Map colNameToDefaultConstraint) { + super(targetTable, whereTree); + this.setClauseTree = setClauseTree; + this.setCols = setCols; + this.colNameToDefaultConstraint = colNameToDefaultConstraint; + } + + public ASTNode getSetClauseTree() { + return setClauseTree; + } + + public Map getSetCols() { + return setCols; + } + + public Map getColNameToDefaultConstraint() { + return colNameToDefaultConstraint; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java new file mode 100644 index 000000000000..0c4e5ac6ebea --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.hadoop.hive.ql.Context; + +public class COWWithClauseBuilder { + + public void appendWith(MultiInsertSqlGenerator sqlGenerator, String filePathCol, String whereClause) { + sqlGenerator.append("WITH t AS ("); + sqlGenerator.append("\n"); + sqlGenerator.append("select "); + sqlGenerator.appendAcidSelectColumnsForDeletedRecords(Context.Operation.DELETE); + sqlGenerator.removeLastChar(); + sqlGenerator.append(" from ("); + sqlGenerator.append("\n"); + sqlGenerator.append("select "); + sqlGenerator.appendAcidSelectColumnsForDeletedRecords(Context.Operation.DELETE); + sqlGenerator.append(" row_number() OVER (partition by ").append(filePathCol).append(") rn"); + sqlGenerator.append(" from "); + sqlGenerator.append(sqlGenerator.getTargetTableFullName()); + sqlGenerator.append("\n"); + sqlGenerator.append("where ").append(whereClause); + sqlGenerator.append("\n"); + sqlGenerator.append(") q"); + sqlGenerator.append("\n"); + sqlGenerator.append("where rn=1\n)\n"); + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java new file mode 100644 index 000000000000..2e568a6f51c4 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.Table; + +import java.util.List; + +import static org.apache.commons.lang3.StringUtils.isBlank; +import static org.apache.commons.lang3.StringUtils.isNotBlank; + +public abstract class MultiInsertSqlGenerator { + private static final String INDENT = " "; + + protected final Table targetTable; + protected final String targetTableFullName; + protected final HiveConf conf; + protected final String subQueryAlias; + protected final StringBuilder queryStr; + + protected MultiInsertSqlGenerator( + Table targetTable, String targetTableFullName, HiveConf conf, String subQueryAlias) { + this.targetTable = targetTable; + this.targetTableFullName = targetTableFullName; + this.conf = conf; + this.subQueryAlias = subQueryAlias; + this.queryStr = new StringBuilder(); + } + + public Table getTargetTable() { + return targetTable; + } + + public String getTargetTableFullName() { + return targetTableFullName; + } + + public abstract void appendAcidSelectColumns(Context.Operation operation); + + public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation) { + throw new UnsupportedOperationException(); + } + + public abstract List getDeleteValues(Context.Operation operation); + public abstract List getSortKeys(); + + public String qualify(String columnName) { + if (isBlank(subQueryAlias)) { + return columnName; + } + return String.format("%s.%s", subQueryAlias, columnName); + } + + public void appendInsertBranch(String hintStr, List values) { + queryStr.append("INSERT INTO ").append(targetTableFullName); + appendPartitionCols(targetTable); + queryStr.append("\n"); + + queryStr.append(INDENT); + queryStr.append("SELECT "); + if (isNotBlank(hintStr)) { + queryStr.append(hintStr); + } + + queryStr.append(StringUtils.join(values, ",")); + queryStr.append("\n"); + } + + public void appendDeleteBranch(String hintStr) { + List deleteValues = getDeleteValues(Context.Operation.DELETE); + appendInsertBranch(hintStr, deleteValues); + } + + public void appendPartitionColsOfTarget() { + appendPartitionCols(targetTable); + } + + /** + * Append list of partition columns to Insert statement. If user specified partition spec, then + * use it to get/set the value for partition column else use dynamic partition mode with no value. + * Static partition mode: + * INSERT INTO T PARTITION(partCol1=val1,partCol2...) SELECT col1, ... partCol1,partCol2... + * Dynamic partition mode: + * INSERT INTO T PARTITION(partCol1,partCol2...) SELECT col1, ... partCol1,partCol2... + */ + public void appendPartitionCols(Table table) { + // If the table is partitioned we have to put the partition() clause in + List partCols = table.getPartCols(); + if (partCols == null || partCols.isEmpty()) { + return; + } + queryStr.append(" partition ("); + appendCols(partCols); + queryStr.append(")"); + } + + public void appendSortBy(List keys) { + if (keys.isEmpty()) { + return; + } + queryStr.append(INDENT).append("SORT BY "); + queryStr.append(StringUtils.join(keys, ",")); + queryStr.append("\n"); + } + + public void appendSortKeys() { + appendSortBy(getSortKeys()); + } + + public MultiInsertSqlGenerator append(String sqlTextFragment) { + queryStr.append(sqlTextFragment); + return this; + } + + @Override + public String toString() { + return queryStr.toString(); + } + + public void removeLastChar() { + queryStr.setLength(queryStr.length() - 1); + } + + public void appendPartColsOfTargetTableWithComma(String alias) { + if (targetTable.getPartCols() == null || targetTable.getPartCols().isEmpty()) { + return; + } + queryStr.append(','); + appendCols(targetTable.getPartCols(), alias); + } + + public void appendAllColsOfTargetTable() { + appendCols(targetTable.getAllCols()); + } + + public void appendCols(List columns) { + appendCols(columns, null); + } + + public void appendCols(List columns, String alias) { + if (columns == null) { + return; + } + + String quotedAlias = null; + if (isNotBlank(alias)) { + quotedAlias = HiveUtils.unparseIdentifier(alias, this.conf); + } + + boolean first = true; + for (FieldSchema fschema : columns) { + if (first) { + first = false; + } else { + queryStr.append(", "); + } + + if (quotedAlias != null) { + queryStr.append(quotedAlias).append('.'); + } + queryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf)); + } + } + + public MultiInsertSqlGenerator appendTargetTableName() { + queryStr.append(targetTableFullName); + return this; + } + + public MultiInsertSqlGenerator append(char c) { + queryStr.append(c); + return this; + } + + public MultiInsertSqlGenerator indent() { + queryStr.append(INDENT); + return this; + } + + public MultiInsertSqlGenerator appendSubQueryAlias() { + queryStr.append(subQueryAlias); + return this; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java new file mode 100644 index 000000000000..729038fac6fa --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.Table; + +import java.util.ArrayList; +import java.util.List; + +import static java.util.Collections.singletonList; + +public class NativeAcidMultiInsertSqlGenerator extends MultiInsertSqlGenerator { + public NativeAcidMultiInsertSqlGenerator(Table table, String targetTableFullName, HiveConf conf, String subQueryAlias) { + super(table, targetTableFullName, conf, subQueryAlias); + } + + @Override + public void appendAcidSelectColumns(Context.Operation operation) { + queryStr.append("ROW__ID,"); + for (FieldSchema fieldSchema : targetTable.getPartCols()) { + String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); + queryStr.append(identifier); + queryStr.append(","); + } + } + + @Override + public List getDeleteValues(Context.Operation operation) { + List deleteValues = new ArrayList<>(1 + targetTable.getPartCols().size()); + deleteValues.add(qualify("ROW__ID")); + for (FieldSchema fieldSchema : targetTable.getPartCols()) { + deleteValues.add(qualify(HiveUtils.unparseIdentifier(fieldSchema.getName(), conf))); + } + return deleteValues; + } + + @Override + public List getSortKeys() { + return singletonList(qualify("ROW__ID")); + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java new file mode 100644 index 000000000000..cbd4014ab1f5 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.Table; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +public class NonNativeAcidMultiInsertSqlGenerator extends MultiInsertSqlGenerator { + private final String deletePrefix; + + public NonNativeAcidMultiInsertSqlGenerator( + Table table, String targetTableFullName, HiveConf conf, String subQueryAlias, String deletePrefix) { + super(table, targetTableFullName, conf, subQueryAlias); + this.deletePrefix = deletePrefix; + } + + @Override + public void appendAcidSelectColumns(Context.Operation operation) { + appendAcidSelectColumns(operation, false); + } + + @Override + public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation) { + appendAcidSelectColumns(operation, true); + } + + private void appendAcidSelectColumns(Context.Operation operation, boolean markRowIdAsDeleted) { + List acidSelectColumns = targetTable.getStorageHandler().acidSelectColumns(targetTable, operation); + for (FieldSchema fieldSchema : acidSelectColumns) { + String identifier = markRowIdAsDeleted && fieldSchema.equals(targetTable.getStorageHandler().getRowId()) ? + "-1" : HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); + queryStr.append(identifier); + + if (StringUtils.isNotEmpty(deletePrefix) && !markRowIdAsDeleted) { + queryStr.append(" AS "); + String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); + queryStr.append(prefixedIdentifier); + } + queryStr.append(","); + } + } + + @Override + public List getDeleteValues(Context.Operation operation) { + List acidSelectColumns = targetTable.getStorageHandler().acidSelectColumns(targetTable, operation); + List deleteValues = new ArrayList<>(acidSelectColumns.size()); + for (FieldSchema fieldSchema : acidSelectColumns) { + String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); + deleteValues.add(qualify(prefixedIdentifier)); + } + return deleteValues; + } + + @Override + public List getSortKeys() { + return targetTable.getStorageHandler().acidSortColumns(targetTable, Context.Operation.DELETE).stream() + .map(fieldSchema -> qualify( + HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf))) + .collect(Collectors.toList()); + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SetClausePatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SetClausePatcher.java new file mode 100644 index 000000000000..ef1ba27e42d6 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SetClausePatcher.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.HiveParser; + +import java.util.Map; + +public class SetClausePatcher { + /** + * Patch up the projection list for updates, putting back the original set expressions. + * Walk through the projection list and replace the column names with the + * expressions from the original update. Under the TOK_SELECT (see above) the structure + * looks like: + *
+   * TOK_SELECT -> TOK_SELEXPR -> expr
+   *           \-> TOK_SELEXPR -> expr ...
+   * 
+ */ + public void patchProjectionForUpdate(ASTNode insertBranch, Map setColExprs) { + ASTNode rewrittenSelect = (ASTNode) insertBranch.getChildren().get(1); + assert rewrittenSelect.getToken().getType() == HiveParser.TOK_SELECT : + "Expected TOK_SELECT as second child of TOK_INSERT but found " + rewrittenSelect.getName(); + for (Map.Entry entry : setColExprs.entrySet()) { + ASTNode selExpr = (ASTNode) rewrittenSelect.getChildren().get(entry.getKey()); + assert selExpr.getToken().getType() == HiveParser.TOK_SELEXPR : + "Expected child of TOK_SELECT to be TOK_SELEXPR but was " + selExpr.getName(); + // Now, change it's child + selExpr.setChild(0, entry.getValue()); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java new file mode 100644 index 000000000000..6c3e48a954c8 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.metadata.Table; + +public class SqlGeneratorFactory { + public static final String DELETE_PREFIX = "__d__"; + public static final String SUB_QUERY_ALIAS = "s"; + + private final Table targetTable; + private final String targetTableFullName; + private final HiveConf conf; + private final String subQueryAlias; + private final String deletePrefix; + + public SqlGeneratorFactory( + Table targetTable, String targetTableFullName, HiveConf conf, String subQueryAlias, String deletePrefix) { + this.targetTable = targetTable; + this.targetTableFullName = targetTableFullName; + this.conf = conf; + this.subQueryAlias = subQueryAlias; + this.deletePrefix = deletePrefix; + } + + public MultiInsertSqlGenerator createSqlGenerator() { + boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable, true); + return nonNativeAcid ? new NonNativeAcidMultiInsertSqlGenerator(targetTable, targetTableFullName, conf, subQueryAlias, deletePrefix) : + new NativeAcidMultiInsertSqlGenerator(targetTable, targetTableFullName, conf, subQueryAlias); + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/WhereClausePatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/WhereClausePatcher.java new file mode 100644 index 000000000000..45c6cb9802d4 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/WhereClausePatcher.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse.rewrite.sql; + +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.HiveParser; +import org.apache.hadoop.hive.ql.parse.SemanticException; + +public class WhereClausePatcher { + public void patch(ASTNode rewrittenInsert, ASTNode whereTree) throws SemanticException { + if (rewrittenInsert.getToken().getType() != HiveParser.TOK_INSERT) { + throw new SemanticException( + "Expected TOK_INSERT as second child of TOK_QUERY but found " + rewrittenInsert.getName()); + } + // The structure of the AST for the rewritten insert statement is: + // TOK_QUERY -> TOK_FROM + // \-> TOK_INSERT -> TOK_INSERT_INTO + // \-> TOK_SELECT + // \-> TOK_SORTBY + // Or + // TOK_QUERY -> TOK_FROM + // \-> TOK_INSERT -> TOK_INSERT_INTO + // \-> TOK_SELECT + // + // The following adds the TOK_WHERE and its subtree from the original query as a child of + // TOK_INSERT, which is where it would have landed if it had been there originally in the + // string. We do it this way because it's easy then turning the original AST back into a + // string and reparsing it. + if (rewrittenInsert.getChildren().size() == 3) { + // We have to move the SORT_BY over one, so grab it and then push it to the second slot, + // and put the where in the first slot + ASTNode sortBy = (ASTNode) rewrittenInsert.getChildren().get(2); + assert sortBy.getToken().getType() == HiveParser.TOK_SORTBY : + "Expected TOK_SORTBY to be third child of TOK_INSERT, but found " + sortBy.getName(); + rewrittenInsert.addChild(sortBy); + rewrittenInsert.setChild(2, whereTree); + } else { + ASTNode select = (ASTNode) rewrittenInsert.getChildren().get(1); + assert select.getToken().getType() == HiveParser.TOK_SELECT : + "Expected TOK_SELECT to be second child of TOK_INSERT, but found " + select.getName(); + rewrittenInsert.addChild(whereTree); + } + } +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java similarity index 99% rename from ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java rename to ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java index fef25e949ecc..ac5795295158 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java @@ -48,9 +48,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TestUpdateDeleteSemanticAnalyzer { +public class TestDMLSemanticAnalyzer { - static final private Logger LOG = LoggerFactory.getLogger(TestUpdateDeleteSemanticAnalyzer.class.getName()); + static final private Logger LOG = LoggerFactory.getLogger(TestDMLSemanticAnalyzer.class.getName()); private QueryState queryState; private HiveConf conf; diff --git a/ql/src/test/queries/clientpositive/sort_acid.q b/ql/src/test/queries/clientpositive/sort_acid.q index ee49143ce23b..4754852dac65 100644 --- a/ql/src/test/queries/clientpositive/sort_acid.q +++ b/ql/src/test/queries/clientpositive/sort_acid.q @@ -3,11 +3,11 @@ set hive.support.concurrency=true; set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; -create table acidtlb(a int, b int) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true'); -create table othertlb(c int, d int) stored as orc TBLPROPERTIES ('transactional'='true'); +create table acidtlb(a int, b int, e string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true'); +create table othertlb(c int, d int, f string) stored as orc TBLPROPERTIES ('transactional'='true'); -insert into acidtlb values(10,200),(30,500); -insert into othertlb values(10, 21),(30, 22),(60, 23),(70, 24),(80, 25); +insert into acidtlb values(10,200,'one'),(30,500,'two'); +insert into othertlb values(10, 21,'one'),(30, 22,'three'),(60, 23,'one'),(70, 24,'three'),(80, 25,'three'); explain cbo @@ -54,12 +54,12 @@ select * from acidtlb order by a; explain cbo merge into acidtlb as t using othertlb as s on t.a = s.c when matched and s.c < 60 then delete -when matched and s.c = 60 then update set b = 1000 -when not matched then insert values (s.c, 2000 + s.d); +when matched and s.c = 60 then update set b = 1000, e=NULL +when not matched then insert values (s.c, 2000 + s.d, NULL); merge into acidtlb as t using othertlb as s on t.a = s.c when matched and s.c < 30 then delete -when matched and s.c = 30 then update set b = 1000 -when not matched then insert values (s.c, 2000 + s.d); +when matched and s.c = 30 then update set b = 1000, e=NULL +when not matched then insert values (s.c, 2000 + s.d, NULL); select * from acidtlb; diff --git a/ql/src/test/queries/clientpositive/sqlmerge.q b/ql/src/test/queries/clientpositive/sqlmerge.q index 2babea34e7cf..412099d820a2 100644 --- a/ql/src/test/queries/clientpositive/sqlmerge.q +++ b/ql/src/test/queries/clientpositive/sqlmerge.q @@ -8,7 +8,7 @@ create table acidTbl_n0(a int, b int) clustered by (a) into 2 buckets stored as create table nonAcidOrcTbl_n0(a int, b int) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='false'); --expect a cardinality check because there is update and hive.merge.cardinality.check=true by default -explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a +explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a WHEN MATCHED AND s.a > 8 THEN DELETE WHEN MATCHED THEN UPDATE SET b = 7 WHEN NOT MATCHED THEN INSERT VALUES(s.a, s.b); @@ -18,7 +18,6 @@ explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a WHEN NOT MATCHED THEN INSERT VALUES(s.a, s.b); set hive.split.update=false; -set hive.merge.split.update=false; explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a WHEN MATCHED AND s.a > 8 THEN DELETE WHEN MATCHED THEN UPDATE SET b = 7 diff --git a/ql/src/test/results/clientpositive/llap/sort_acid.q.out b/ql/src/test/results/clientpositive/llap/sort_acid.q.out index 285bb249e692..30b5d8736ac3 100644 --- a/ql/src/test/results/clientpositive/llap/sort_acid.q.out +++ b/ql/src/test/results/clientpositive/llap/sort_acid.q.out @@ -1,39 +1,41 @@ -PREHOOK: query: create table acidtlb(a int, b int) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: query: create table acidtlb(a int, b int, e string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@acidtlb -POSTHOOK: query: create table acidtlb(a int, b int) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: query: create table acidtlb(a int, b int, e string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@acidtlb -PREHOOK: query: create table othertlb(c int, d int) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: query: create table othertlb(c int, d int, f string) stored as orc TBLPROPERTIES ('transactional'='true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@othertlb -POSTHOOK: query: create table othertlb(c int, d int) stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: query: create table othertlb(c int, d int, f string) stored as orc TBLPROPERTIES ('transactional'='true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@othertlb -PREHOOK: query: insert into acidtlb values(10,200),(30,500) +PREHOOK: query: insert into acidtlb values(10,200,'one'),(30,500,'two') PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@acidtlb -POSTHOOK: query: insert into acidtlb values(10,200),(30,500) +POSTHOOK: query: insert into acidtlb values(10,200,'one'),(30,500,'two') POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@acidtlb POSTHOOK: Lineage: acidtlb.a SCRIPT [] POSTHOOK: Lineage: acidtlb.b SCRIPT [] -PREHOOK: query: insert into othertlb values(10, 21),(30, 22),(60, 23),(70, 24),(80, 25) +POSTHOOK: Lineage: acidtlb.e SCRIPT [] +PREHOOK: query: insert into othertlb values(10, 21,'one'),(30, 22,'three'),(60, 23,'one'),(70, 24,'three'),(80, 25,'three') PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@othertlb -POSTHOOK: query: insert into othertlb values(10, 21),(30, 22),(60, 23),(70, 24),(80, 25) +POSTHOOK: query: insert into othertlb values(10, 21,'one'),(30, 22,'three'),(60, 23,'one'),(70, 24,'three'),(80, 25,'three') POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@othertlb POSTHOOK: Lineage: othertlb.c SCRIPT [] POSTHOOK: Lineage: othertlb.d SCRIPT [] +POSTHOOK: Lineage: othertlb.f SCRIPT [] PREHOOK: query: explain cbo select a, 6 as c, b from acidtlb sort by a, c, b PREHOOK: type: QUERY @@ -72,7 +74,7 @@ POSTHOOK: Input: default@acidtlb POSTHOOK: Output: default@acidtlb POSTHOOK: Output: default@acidtlb CBO PLAN: -HiveProject(row__id=[$4], a=[$0], b=[777]) +HiveProject(row__id=[$5], a=[$0], b=[777], e=[$2]) HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb]) PREHOOK: query: update acidtlb set b=777 @@ -87,6 +89,7 @@ POSTHOOK: Output: default@acidtlb POSTHOOK: Output: default@acidtlb POSTHOOK: Lineage: acidtlb.a SIMPLE [(acidtlb)acidtlb.FieldSchema(name:a, type:int, comment:null), ] POSTHOOK: Lineage: acidtlb.b SIMPLE [] +POSTHOOK: Lineage: acidtlb.e SIMPLE [(acidtlb)acidtlb.FieldSchema(name:e, type:string, comment:null), ] PREHOOK: query: select * from acidtlb PREHOOK: type: QUERY PREHOOK: Input: default@acidtlb @@ -95,8 +98,8 @@ POSTHOOK: query: select * from acidtlb POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb #### A masked pattern was here #### -10 777 -30 777 +10 777 one +30 777 two Warning: Shuffle Join MERGEJOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: explain cbo update acidtlb set b=350 @@ -113,9 +116,9 @@ POSTHOOK: Input: default@acidtlb POSTHOOK: Output: default@acidtlb POSTHOOK: Output: default@acidtlb CBO PLAN: -HiveProject(row__id=[$1], a=[CAST(30):INTEGER], b=[350]) +HiveProject(row__id=[$2], a=[CAST(30):INTEGER], b=[350], e=[$1]) HiveSemiJoin(condition=[true], joinType=[semi]) - HiveProject(a=[CAST(30):INTEGER], ROW__ID=[$4]) + HiveProject(a=[CAST(30):INTEGER], e=[$2], ROW__ID=[$5]) HiveFilter(condition=[=($0, 30)]) HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb]) HiveProject($f0=[$0]) @@ -139,6 +142,7 @@ POSTHOOK: Output: default@acidtlb POSTHOOK: Output: default@acidtlb POSTHOOK: Lineage: acidtlb.a SIMPLE [] POSTHOOK: Lineage: acidtlb.b SIMPLE [] +POSTHOOK: Lineage: acidtlb.e SIMPLE [(acidtlb)acidtlb.FieldSchema(name:e, type:string, comment:null), ] PREHOOK: query: select * from acidtlb PREHOOK: type: QUERY PREHOOK: Input: default@acidtlb @@ -147,8 +151,8 @@ POSTHOOK: query: select * from acidtlb POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb #### A masked pattern was here #### -10 777 -30 350 +10 777 one +30 350 two PREHOOK: query: explain cbo update acidtlb set b=450 where a in (select c from othertlb where c < 65) @@ -166,9 +170,9 @@ POSTHOOK: Input: default@othertlb POSTHOOK: Output: default@acidtlb POSTHOOK: Output: default@acidtlb CBO PLAN: -HiveProject(row__id=[$1], a=[$0], b=[450]) - HiveSemiJoin(condition=[=($0, $2)], joinType=[semi]) - HiveProject(a=[$0], ROW__ID=[$4]) +HiveProject(row__id=[$2], a=[$0], b=[450], e=[$1]) + HiveSemiJoin(condition=[=($0, $3)], joinType=[semi]) + HiveProject(a=[$0], e=[$2], ROW__ID=[$5]) HiveFilter(condition=[<($0, 65)]) HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb]) HiveProject(c=[$0]) @@ -191,6 +195,7 @@ POSTHOOK: Output: default@acidtlb POSTHOOK: Output: default@acidtlb POSTHOOK: Lineage: acidtlb.a SIMPLE [(acidtlb)acidtlb.FieldSchema(name:a, type:int, comment:null), ] POSTHOOK: Lineage: acidtlb.b SIMPLE [] +POSTHOOK: Lineage: acidtlb.e SIMPLE [(acidtlb)acidtlb.FieldSchema(name:e, type:string, comment:null), ] PREHOOK: query: select * from acidtlb PREHOOK: type: QUERY PREHOOK: Input: default@acidtlb @@ -199,8 +204,8 @@ POSTHOOK: query: select * from acidtlb POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb #### A masked pattern was here #### -10 450 -30 450 +10 450 one +30 450 two PREHOOK: query: explain cbo delete from acidtlb where a in ( @@ -225,7 +230,7 @@ CBO PLAN: HiveSortExchange(distribution=[any], collation=[[0]]) HiveProject(row__id=[$1]) HiveSemiJoin(condition=[=($0, $2)], joinType=[semi]) - HiveProject(a=[$0], ROW__ID=[$4]) + HiveProject(a=[$0], ROW__ID=[$5]) HiveFilter(condition=[IS NOT NULL($0)]) HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb]) HiveProject(a=[$0]) @@ -263,12 +268,12 @@ POSTHOOK: query: select * from acidtlb order by a POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb #### A masked pattern was here #### -30 450 +30 450 two PREHOOK: query: explain cbo merge into acidtlb as t using othertlb as s on t.a = s.c when matched and s.c < 60 then delete -when matched and s.c = 60 then update set b = 1000 -when not matched then insert values (s.c, 2000 + s.d) +when matched and s.c = 60 then update set b = 1000, e=NULL +when not matched then insert values (s.c, 2000 + s.d, NULL) PREHOOK: type: QUERY PREHOOK: Input: default@acidtlb PREHOOK: Input: default@othertlb @@ -278,8 +283,8 @@ PREHOOK: Output: default@merge_tmp_table POSTHOOK: query: explain cbo merge into acidtlb as t using othertlb as s on t.a = s.c when matched and s.c < 60 then delete -when matched and s.c = 60 then update set b = 1000 -when not matched then insert values (s.c, 2000 + s.d) +when matched and s.c = 60 then update set b = 1000, e=NULL +when not matched then insert values (s.c, 2000 + s.d, NULL) POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb POSTHOOK: Input: default@othertlb @@ -291,14 +296,14 @@ HiveProject(col0=[$2], col1=[$0], col2=[$1], col3=[$3]) HiveJoin(condition=[=($3, $0)], joinType=[left], algorithm=[none], cost=[not available]) HiveProject(c=[$0], d=[$1]) HiveTableScan(table=[[default, othertlb]], table:alias=[s]) - HiveProject(row__id=[$4], a=[$0]) + HiveProject(row__id=[$5], a=[$0]) HiveFilter(condition=[IS NOT NULL($0)]) HiveTableScan(table=[[default, acidtlb]], table:alias=[acidtlb]) PREHOOK: query: merge into acidtlb as t using othertlb as s on t.a = s.c when matched and s.c < 30 then delete -when matched and s.c = 30 then update set b = 1000 -when not matched then insert values (s.c, 2000 + s.d) +when matched and s.c = 30 then update set b = 1000, e=NULL +when not matched then insert values (s.c, 2000 + s.d, NULL) PREHOOK: type: QUERY PREHOOK: Input: default@acidtlb PREHOOK: Input: default@othertlb @@ -307,8 +312,8 @@ PREHOOK: Output: default@acidtlb PREHOOK: Output: default@merge_tmp_table POSTHOOK: query: merge into acidtlb as t using othertlb as s on t.a = s.c when matched and s.c < 30 then delete -when matched and s.c = 30 then update set b = 1000 -when not matched then insert values (s.c, 2000 + s.d) +when matched and s.c = 30 then update set b = 1000, e=NULL +when not matched then insert values (s.c, 2000 + s.d, NULL) POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb POSTHOOK: Input: default@othertlb @@ -319,6 +324,8 @@ POSTHOOK: Lineage: acidtlb.a SIMPLE [(othertlb)s.FieldSchema(name:c, type:int, c POSTHOOK: Lineage: acidtlb.a SIMPLE [(othertlb)s.FieldSchema(name:c, type:int, comment:null), ] POSTHOOK: Lineage: acidtlb.b EXPRESSION [(othertlb)s.FieldSchema(name:d, type:int, comment:null), ] POSTHOOK: Lineage: acidtlb.b EXPRESSION [(othertlb)s.FieldSchema(name:d, type:int, comment:null), ] +POSTHOOK: Lineage: acidtlb.e EXPRESSION [] +POSTHOOK: Lineage: acidtlb.e EXPRESSION [] POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION [(acidtlb)acidtlb.FieldSchema(name:ROW__ID, type:struct, comment:), ] PREHOOK: query: select * from acidtlb PREHOOK: type: QUERY @@ -328,8 +335,8 @@ POSTHOOK: query: select * from acidtlb POSTHOOK: type: QUERY POSTHOOK: Input: default@acidtlb #### A masked pattern was here #### -10 2021 -30 1000 -60 2023 -70 2024 -80 2025 +10 2021 NULL +30 1000 NULL +60 2023 NULL +70 2024 NULL +80 2025 NULL diff --git a/ql/src/test/results/clientpositive/llap/sqlmerge.q.out b/ql/src/test/results/clientpositive/llap/sqlmerge.q.out index ff8c773ccf2c..807948e470f4 100644 --- a/ql/src/test/results/clientpositive/llap/sqlmerge.q.out +++ b/ql/src/test/results/clientpositive/llap/sqlmerge.q.out @@ -14,7 +14,7 @@ POSTHOOK: query: create table nonAcidOrcTbl_n0(a int, b int) clustered by (a) in POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@nonAcidOrcTbl_n0 -PREHOOK: query: explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a +PREHOOK: query: explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a WHEN MATCHED AND s.a > 8 THEN DELETE WHEN MATCHED THEN UPDATE SET b = 7 WHEN NOT MATCHED THEN INSERT VALUES(s.a, s.b) @@ -24,7 +24,7 @@ PREHOOK: Input: default@nonacidorctbl_n0 PREHOOK: Output: default@acidtbl_n0 PREHOOK: Output: default@acidtbl_n0 PREHOOK: Output: default@merge_tmp_table -POSTHOOK: query: explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a +POSTHOOK: query: explain merge into acidTbl_n0 as t using nonAcidOrcTbl_n0 s ON t.a = s.a WHEN MATCHED AND s.a > 8 THEN DELETE WHEN MATCHED THEN UPDATE SET b = 7 WHEN NOT MATCHED THEN INSERT VALUES(s.a, s.b) From b5b72115fb73f62084e88db1572a80e0b1cf6e9f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Nov 2023 16:49:27 +0530 Subject: [PATCH 052/179] HIVE-27877: Bump org.apache.avro:avro from 1.11.1 to 1.11.3 (#4764). (dependabot, Reviewed by Ayush Saxena) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a2c4078318c0..ffcc61b47cc5 100644 --- a/pom.xml +++ b/pom.xml @@ -111,7 +111,7 @@ 12.0.0 1.12.0 - 1.11.1 + 1.11.3 1.68 1.25.0 5.2.8 From eb6f0b0c57dd55335927b7dde08cd47f4d00e74d Mon Sep 17 00:00:00 2001 From: dengzh Date: Fri, 17 Nov 2023 19:43:16 +0800 Subject: [PATCH 053/179] HIVE-27862: Map propertyContent to a wrong column in package.jdo (#4873) (Zhihua Deng, reviewed by Attila Turoczy, Ayush Saxena) --- .../metastore-server/src/main/resources/package.jdo | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/standalone-metastore/metastore-server/src/main/resources/package.jdo b/standalone-metastore/metastore-server/src/main/resources/package.jdo index c7493d3a7902..982703b2783c 100644 --- a/standalone-metastore/metastore-server/src/main/resources/package.jdo +++ b/standalone-metastore/metastore-server/src/main/resources/package.jdo @@ -1107,7 +1107,7 @@ - + From aa0237d62099d23bcfadb1ff4c4171a15de25447 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sat, 18 Nov 2023 20:48:52 +0800 Subject: [PATCH 054/179] HIVE-27880: Iceberg: Support creating a branch on an empty table (#4875). (zhangbutao, reviewed by Ayush Saxena) --- .../mr/hive/HiveIcebergStorageHandler.java | 7 ++++--- .../iceberg/mr/hive/IcebergBranchExec.java | 12 ++++++++--- .../alter_table_create_branch_negative.q | 3 --- .../positive/alter_table_create_branch.q | 5 ++++- .../alter_table_create_branch_negative.q.out | 12 ----------- .../positive/alter_table_create_branch.q.out | 21 +++++++++++++++++++ 6 files changed, 38 insertions(+), 22 deletions(-) delete mode 100644 iceberg/iceberg-handler/src/test/queries/negative/alter_table_create_branch_negative.q delete mode 100644 iceberg/iceberg-handler/src/test/results/negative/alter_table_create_branch_negative.q.out diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 537ffd28d411..1088d6d43025 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -943,9 +943,6 @@ public void alterTableSnapshotRefOperation(org.apache.hadoop.hive.ql.metadata.Ta AlterTableSnapshotRefSpec alterTableSnapshotRefSpec) { TableDesc tableDesc = Utilities.getTableDesc(hmsTable); Table icebergTable = IcebergTableUtil.getTable(conf, tableDesc.getProperties()); - Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> - new UnsupportedOperationException(String.format("Cannot alter %s on iceberg table %s.%s which has no snapshot", - alterTableSnapshotRefSpec.getOperationType().getName(), hmsTable.getDbName(), hmsTable.getTableName()))); switch (alterTableSnapshotRefSpec.getOperationType()) { case CREATE_BRANCH: @@ -954,6 +951,10 @@ public void alterTableSnapshotRefOperation(org.apache.hadoop.hive.ql.metadata.Ta IcebergBranchExec.createBranch(icebergTable, createBranchSpec); break; case CREATE_TAG: + Optional.ofNullable(icebergTable.currentSnapshot()).orElseThrow(() -> new UnsupportedOperationException( + String.format("Cannot alter %s on iceberg table %s.%s which has no snapshot", + alterTableSnapshotRefSpec.getOperationType().getName(), hmsTable.getDbName(), + hmsTable.getTableName()))); AlterTableSnapshotRefSpec.CreateSnapshotRefSpec createTagSpec = (AlterTableSnapshotRefSpec.CreateSnapshotRefSpec) alterTableSnapshotRefSpec.getOperationParams(); IcebergTagExec.createTag(icebergTable, createTagSpec); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergBranchExec.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergBranchExec.java index bd92b577114c..7425ff95c661 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergBranchExec.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergBranchExec.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive; +import java.util.Optional; import org.apache.hadoop.hive.ql.parse.AlterTableSnapshotRefSpec; import org.apache.iceberg.ManageSnapshots; import org.apache.iceberg.SnapshotRef; @@ -55,11 +56,16 @@ public static void createBranch(Table table, AlterTableSnapshotRefSpec.CreateSna throw new IllegalArgumentException(String.format("Tag %s does not exist", tagName)); } } else { - snapshotId = table.currentSnapshot().snapshotId(); + snapshotId = Optional.ofNullable(table.currentSnapshot()).map(snapshot -> snapshot.snapshotId()).orElse(null); } - LOG.info("Creating branch {} on iceberg table {} with snapshotId {}", branchName, table.name(), snapshotId); ManageSnapshots manageSnapshots = table.manageSnapshots(); - manageSnapshots.createBranch(branchName, snapshotId); + if (snapshotId != null) { + LOG.info("Creating a branch {} on an iceberg table {} with snapshotId {}", branchName, table.name(), snapshotId); + manageSnapshots.createBranch(branchName, snapshotId); + } else { + LOG.info("Creating a branch {} on an empty iceberg table {}", branchName, table.name()); + manageSnapshots.createBranch(branchName); + } if (createBranchSpec.getMaxRefAgeMs() != null) { manageSnapshots.setMaxRefAgeMs(branchName, createBranchSpec.getMaxRefAgeMs()); } diff --git a/iceberg/iceberg-handler/src/test/queries/negative/alter_table_create_branch_negative.q b/iceberg/iceberg-handler/src/test/queries/negative/alter_table_create_branch_negative.q deleted file mode 100644 index 45078a252b38..000000000000 --- a/iceberg/iceberg-handler/src/test/queries/negative/alter_table_create_branch_negative.q +++ /dev/null @@ -1,3 +0,0 @@ -create table ice_tbl (id int, name string) Stored by Iceberg; - -alter table ice_tbl create branch test_branch_1; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/alter_table_create_branch.q b/iceberg/iceberg-handler/src/test/queries/positive/alter_table_create_branch.q index 4b525d71be6c..6e83bef5a4ce 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/alter_table_create_branch.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/alter_table_create_branch.q @@ -3,7 +3,10 @@ set hive.explain.user=false; create table iceTbl (id int, name string) Stored by Iceberg; --- creating branch requires table to have current snapshot. here insert some values to generate current snapshot +-- create a branch on an empty table +explain alter table iceTbl create branch test_branch_0; +alter table iceTbl create branch test_branch_0; + insert into iceTbl values(1, 'jack'); -- create s branch test_branch_1 with default values based on the current snapshotId diff --git a/iceberg/iceberg-handler/src/test/results/negative/alter_table_create_branch_negative.q.out b/iceberg/iceberg-handler/src/test/results/negative/alter_table_create_branch_negative.q.out deleted file mode 100644 index 75c7f0936c6e..000000000000 --- a/iceberg/iceberg-handler/src/test/results/negative/alter_table_create_branch_negative.q.out +++ /dev/null @@ -1,12 +0,0 @@ -PREHOOK: query: create table ice_tbl (id int, name string) Stored by Iceberg -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@ice_tbl -POSTHOOK: query: create table ice_tbl (id int, name string) Stored by Iceberg -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@ice_tbl -PREHOOK: query: alter table ice_tbl create branch test_branch_1 -PREHOOK: type: ALTERTABLE_CREATEBRANCH -PREHOOK: Input: default@ice_tbl -FAILED: Execution Error, return code 40000 from org.apache.hadoop.hive.ql.ddl.DDLTask. java.lang.UnsupportedOperationException: Cannot alter create branch on iceberg table default.ice_tbl which has no snapshot diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_table_create_branch.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_table_create_branch.q.out index 5e86f7e7c9a9..f86670d182a7 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/alter_table_create_branch.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/alter_table_create_branch.q.out @@ -6,6 +6,27 @@ POSTHOOK: query: create table iceTbl (id int, name string) Stored by Iceberg POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@iceTbl +PREHOOK: query: explain alter table iceTbl create branch test_branch_0 +PREHOOK: type: ALTERTABLE_CREATEBRANCH +PREHOOK: Input: default@icetbl +POSTHOOK: query: explain alter table iceTbl create branch test_branch_0 +POSTHOOK: type: ALTERTABLE_CREATEBRANCH +POSTHOOK: Input: default@icetbl +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + SnapshotRef Operation + table name: default.iceTbl + spec: AlterTableSnapshotRefSpec{operationType=CREATE_BRANCH, operationParams=CreateSnapshotRefSpec{refName=test_branch_0}} + +PREHOOK: query: alter table iceTbl create branch test_branch_0 +PREHOOK: type: ALTERTABLE_CREATEBRANCH +PREHOOK: Input: default@icetbl +POSTHOOK: query: alter table iceTbl create branch test_branch_0 +POSTHOOK: type: ALTERTABLE_CREATEBRANCH +POSTHOOK: Input: default@icetbl PREHOOK: query: insert into iceTbl values(1, 'jack') PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table From ab7373c3b14bf55d625c65c290e5d3b2e09e447a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Nov 2023 13:34:55 +0530 Subject: [PATCH 055/179] HIVE-27889: Bump org.apache.santuario:xmlsec from 2.3.0 to 2.3.4 (#4818). (dependabot, Reviewed by Ayush Saxena) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ffcc61b47cc5..4edce4eb3a4f 100644 --- a/pom.xml +++ b/pom.xml @@ -206,7 +206,7 @@ 1.4 2.3 2.12.2 - 2.3.0 + 2.3.4 3.7.2 1.1 2.4.0 From feda35389dc28c8c9bf3c8a3d39de53ba90e41c0 Mon Sep 17 00:00:00 2001 From: Dayakar M <59791497+mdayakar@users.noreply.github.com> Date: Mon, 20 Nov 2023 10:22:43 +0530 Subject: [PATCH 056/179] HIVE-27491: HPL/SQL does not allow variables in update statements (Dayakar M, reviewed by Krisztian Kasa) --- .../java/org/apache/hive/hplsql/Exec.java | 46 ++++++++-- .../java/org/apache/hive/hplsql/Stmt.java | 91 ++++++++++++++----- hplsql/src/test/queries/offline/update.sql | 15 ++- .../src/test/results/offline/update.out.txt | 40 +++++--- 4 files changed, 141 insertions(+), 51 deletions(-) diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java index 126901263744..8d5cab92e136 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java @@ -206,7 +206,7 @@ public Var setVariable(String name, Var value) { else { var = new Var(value); var.setName(name); - if(exec.currentScope != null) { + if(exec.currentScope != null && !exec.buildSql) { exec.currentScope.addVariable(var); } } @@ -216,7 +216,7 @@ public Var setVariable(String name, Var value) { public Var setVariable(String name) { return setVariable(name, Var.Empty); } - + public Var setVariable(String name, String value) { return setVariable(name, new Var(value)); } @@ -1630,6 +1630,18 @@ public Integer visitAssignment_stmt_single_item(HplsqlParser.Assignment_stmt_sin String name = ctx.ident().getText(); visit(ctx.expr()); Var var = setVariable(name); + StringBuilder assignments = new StringBuilder(); + String previousAssignment = stackPop().toString(); + if (previousAssignment != null) { + assignments.append(previousAssignment); + assignments.append(", "); + } + assignments.append(name); + assignments.append(" = "); + assignments.append(var.toString()); + if (exec.buildSql) { + stackPush(assignments); + } if (trace) { trace(ctx, "SET " + name + " = " + var.toSqlString()); } @@ -1642,17 +1654,30 @@ public Integer visitAssignment_stmt_single_item(HplsqlParser.Assignment_stmt_sin @Override public Integer visitAssignment_stmt_multiple_item(HplsqlParser.Assignment_stmt_multiple_itemContext ctx) { int cnt = ctx.ident().size(); - int ecnt = ctx.expr().size(); + int ecnt = ctx.expr().size(); + StringBuilder identifiers = new StringBuilder("("); + StringBuilder expressions = new StringBuilder("("); for (int i = 0; i < cnt; i++) { String name = ctx.ident(i).getText(); if (i < ecnt) { visit(ctx.expr(i)); - Var var = setVariable(name); + Var var = setVariable(name); + if (i > 0) { + identifiers.append(", "); + expressions.append(", "); + } + identifiers.append(name); + expressions.append(var.toString()); if (trace) { trace(ctx, "SET " + name + " = " + var.toString()); } } - } + } + identifiers.append(")"); + expressions.append(")"); + if (exec.buildSql) { + stackPush(identifiers.toString() + " = " + expressions.toString()); + } return 0; } @@ -2775,9 +2800,16 @@ String evalPop(HplsqlParser.DtypeContext type, HplsqlParser.Dtype_lenContext len /** * Get formatted text between 2 tokens */ - public static String getFormattedText(ParserRuleContext ctx) { + public static String getFormattedText(ParserRuleContext ctx, int startIndex, int endIndex) { return ctx.start.getInputStream().getText( - new org.antlr.v4.runtime.misc.Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex())); + new org.antlr.v4.runtime.misc.Interval(startIndex, endIndex)); + } + + /** + * Get formatted text between 2 tokens + */ + public static String getFormattedText(ParserRuleContext ctx) { + return getFormattedText(ctx, ctx.start.getStartIndex(), ctx.stop.getStopIndex()); } /** diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java index 17bf0d08928b..eac3a6d0d2de 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java @@ -35,6 +35,7 @@ import org.apache.hive.hplsql.executor.QueryExecutor; import org.apache.hive.hplsql.executor.QueryResult; import org.apache.hive.hplsql.objects.Table; +import org.jetbrains.annotations.NotNull; /** * HPL/SQL statements execution @@ -665,7 +666,16 @@ public Integer ifBteq(HplsqlParser.If_bteq_stmtContext ctx) { /** * Assignment from SELECT statement */ - public Integer assignFromSelect(HplsqlParser.Assignment_stmt_select_itemContext ctx) { + public Integer assignFromSelect(HplsqlParser.Assignment_stmt_select_itemContext ctx) { + if (exec.buildSql) { + StringBuilder sb = new StringBuilder(); + sb.append(Exec.getFormattedText(ctx, ctx.start.getStartIndex(), ctx.select_stmt().getStart().getStartIndex()-1)); + sb.append(evalPop(ctx.select_stmt()).toString()); + sb.append(")"); + exec.stackPush(sb); + return 0; + } + String sql = evalPop(ctx.select_stmt()).toString(); if (trace) { trace(ctx, sql); @@ -677,27 +687,7 @@ public Integer assignFromSelect(HplsqlParser.Assignment_stmt_select_itemContext } exec.setSqlSuccess(); try { - int cnt = ctx.ident().size(); - if (query.next()) { - for (int i = 0; i < cnt; i++) { - Var var = exec.findVariable(ctx.ident(i).getText()); - if (var != null) { - var.setValue(query, i); - if (trace) { - trace(ctx, "COLUMN: " + query.metadata().columnName(i) + ", " + query.metadata().columnTypeName(i)); - trace(ctx, "SET " + var.getName() + " = " + var.toString()); - } - } - else if(trace) { - trace(ctx, "Variable not found: " + ctx.ident(i).getText()); - } - } - exec.incRowCount(); - exec.setSqlSuccess(); - } else { - exec.setSqlCode(SqlCodes.NO_DATA_FOUND); - exec.signal(Signal.Type.NOTFOUND); - } + processQueryResult(ctx, query); } catch (QueryException e) { exec.signal(query); return 1; @@ -706,7 +696,31 @@ else if(trace) { } return 0; } - + + private void processQueryResult(HplsqlParser.Assignment_stmt_select_itemContext ctx, QueryResult query) { + int cnt = ctx.ident().size(); + if (query.next()) { + for (int i = 0; i < cnt; i++) { + Var var = exec.findVariable(ctx.ident(i).getText()); + if (var != null) { + var.setValue(query, i); + if (trace) { + trace(ctx, "COLUMN: " + query.metadata().columnName(i) + ", " + query.metadata().columnTypeName(i)); + trace(ctx, "SET " + var.getName() + " = " + var.toString()); + } + } + else if(trace) { + trace(ctx, "Variable not found: " + ctx.ident(i).getText()); + } + } + exec.incRowCount(); + exec.setSqlSuccess(); + } else { + exec.setSqlCode(SqlCodes.NO_DATA_FOUND); + exec.signal(Signal.Type.NOTFOUND); + } + } + /** * SQL INSERT statement */ @@ -1152,7 +1166,14 @@ public void leaveLoop(String value) { */ public Integer update(HplsqlParser.Update_stmtContext ctx) { trace(ctx, "UPDATE"); - String sql = exec.getFormattedText(ctx); + boolean oldBuildSql = exec.buildSql; + String sql = null; + try { + exec.buildSql = true; + sql = generateUpdateQuery(ctx); + } finally { + exec.buildSql = oldBuildSql; + } trace(ctx, sql); QueryResult query = queryExecutor.executeQuery(sql, ctx); if (query.error()) { @@ -1163,7 +1184,27 @@ public Integer update(HplsqlParser.Update_stmtContext ctx) { query.close(); return 0; } - + + @NotNull + private String generateUpdateQuery(HplsqlParser.Update_stmtContext ctx) { + HplsqlParser.Update_assignmentContext updateAssignmentContext = ctx.update_assignment(); + StringBuilder sql = new StringBuilder( + Exec.getFormattedText(ctx, ctx.start.getStartIndex(), (updateAssignmentContext.start.getStartIndex() - 1))); + sql.append(evalPop(updateAssignmentContext).toString()); + Token last = updateAssignmentContext.getStop(); + HplsqlParser.Where_clauseContext whereClauseContext = ctx.where_clause(); + if (whereClauseContext != null) { + exec.append(sql, evalPop(whereClauseContext).toString(), last, whereClauseContext.getStart()); + last = whereClauseContext.getStop(); + } + HplsqlParser.Update_upsertContext updateUpsertContext = ctx.update_upsert(); + if (updateUpsertContext != null) { + exec.append(sql, Exec.getFormattedText(updateUpsertContext, updateUpsertContext.start.getStartIndex(), + updateUpsertContext.stop.getStopIndex()), last, updateUpsertContext.getStart()); + } + return sql.toString(); + } + /** * DELETE statement */ diff --git a/hplsql/src/test/queries/offline/update.sql b/hplsql/src/test/queries/offline/update.sql index f40fb0fe2476..0c72c1875ffe 100644 --- a/hplsql/src/test/queries/offline/update.sql +++ b/hplsql/src/test/queries/offline/update.sql @@ -1,15 +1,15 @@ UPDATE tab T SET (c1) = TRIM(c1) WHERE T.c2 = 'A'; -UPDATE tab T - SET c1 = TRIM(c1) +UPDATE tab T + SET c1 = TRIM(c1) WHERE T.c2 = 'A'; - + UPDATE tab SET c1 = '0011' WHERE c1 = '0021'; UPDATE tab T SET c1 = TRIM(c1), c3 = TRIM(c3) WHERE T.col2 = 'A'; -UPDATE tab T - SET (c1, c3) = (TRIM(c1), TRIM(c3)) +UPDATE tab T + SET (c1, c3) = (TRIM(c1), TRIM(c3)) WHERE T.col2 = 'A'; UPDATE tab T @@ -31,3 +31,8 @@ UPDATE tab T SET c1 = (SELECT c1 FROM tab2 C WHERE C.c1 = T.c1) WHERE T.c2 = 'a'; + +DECLARE var1 STRING; +var1 := 'var1_value'; +UPDATE tab SET c2 = 'update_value' WHERE c1 = var1; +UPDATE tab SET c2 = var1 WHERE c1 = 'var1_value'; diff --git a/hplsql/src/test/results/offline/update.out.txt b/hplsql/src/test/results/offline/update.out.txt index fadaac043abd..3f053b0dd9f8 100644 --- a/hplsql/src/test/results/offline/update.out.txt +++ b/hplsql/src/test/results/offline/update.out.txt @@ -1,34 +1,46 @@ Ln:1 UPDATE -Ln:1 UPDATE tab T SET (c1) = TRIM(c1) WHERE T.c2 = 'A' +Ln:1 SET c1 = 'TRIM(c1)' +Ln:1 UPDATE tab T SET c1 = TRIM(c1) WHERE T.c2 = 'A' Ln:3 UPDATE -Ln:3 UPDATE tab T - SET c1 = TRIM(c1) +Ln:4 SET c1 = 'TRIM(c1)' +Ln:3 UPDATE tab T + SET c1 = TRIM(c1) WHERE T.c2 = 'A' Ln:7 UPDATE +Ln:7 SET c1 = '''0011''' Ln:7 UPDATE tab SET c1 = '0011' WHERE c1 = '0021' Ln:9 UPDATE +Ln:9 SET c1 = 'TRIM(c1)' +Ln:9 SET c3 = 'TRIM(c3)' Ln:9 UPDATE tab T SET c1 = TRIM(c1), c3 = TRIM(c3) WHERE T.col2 = 'A' Ln:11 UPDATE -Ln:11 UPDATE tab T - SET (c1, c3) = (TRIM(c1), TRIM(c3)) +Ln:12 SET c1 = TRIM(c1) +Ln:12 SET c3 = TRIM(c3) +Ln:11 UPDATE tab T + SET (c1, c3) = (TRIM(c1), TRIM(c3)) WHERE T.col2 = 'A' Ln:15 UPDATE Ln:15 UPDATE tab T SET (c1, c2, c3, c4) = - (SELECT c1, - c2, - TRIM(c3), - c4 + (SELECT c1, c2, TRIM(c3), c4 FROM tab2 C WHERE C.c1 = T.c1) WHERE T.c2 = 'A' Ln:25 UPDATE +Ln:26 SET c1 = '(SELECT c1 FROM tab2 C WHERE C.c1 = T.c1)' Ln:25 UPDATE tab T - SET (c1) = - (SELECT c1 FROM tab2 C WHERE C.c1 = T.c1) + SET c1 = (SELECT c1 FROM tab2 C WHERE C.c1 = T.c1) WHERE T.c2 = 'a' Ln:30 UPDATE +Ln:31 SET c1 = '(SELECT c1 FROM tab2 C WHERE C.c1 = T.c1)' Ln:30 UPDATE tab T - SET c1 = - (SELECT c1 FROM tab2 C WHERE C.c1 = T.c1) - WHERE T.c2 = 'a' \ No newline at end of file + SET c1 = (SELECT c1 FROM tab2 C WHERE C.c1 = T.c1) + WHERE T.c2 = 'a' +Ln:35 DECLARE var1 STRING +Ln:36 SET var1 = 'var1_value' +Ln:37 UPDATE +Ln:37 SET c2 = '''update_value''' +Ln:37 UPDATE tab SET c2 = 'update_value' WHERE c1 = 'var1_value' +Ln:38 UPDATE +Ln:38 SET c2 = '''var1_value''' +Ln:38 UPDATE tab SET c2 = 'var1_value' WHERE c1 = 'var1_value' \ No newline at end of file From 1b1caf9c308769ac8efaf8dbe8965dad5f2efebe Mon Sep 17 00:00:00 2001 From: Mahesh Raju Somalaraju Date: Tue, 21 Nov 2023 17:31:06 +0530 Subject: [PATCH 057/179] HIVE-27679: Ranger Yarn Queue policies are not applying correctly, rework done for HIVE-26352 (#4683) (Mahesh Raju Somalaraju reviewed by Laszlo Bodor and Laszlo Vegh) --- .../hadoop/hive/ql/exec/tez/YarnQueueHelper.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java index b41a91882608..c7065156c27b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java @@ -93,6 +93,14 @@ public YarnQueueHelper(HiveConf conf) { lastKnownGoodUrl = 0; } + /** + * Checks yarn queue access of a given user for a given queue, and throws + * HiveException in case the user doesn't have access to the queue. + * @param queueName the yarn queue name + * @param userName the username + * @throws IOException when doAs throws an exception, we simply throw it + * InterruptedException when doAs throws an exception, we simply throw it + */ public void checkQueueAccess( String queueName, String userName) throws IOException { UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); @@ -101,8 +109,9 @@ public void checkQueueAccess( checkQueueAccessInternal(queueName, userName); return null; }); - } catch (Exception exception) { + } catch (InterruptedException exception) { LOG.error("Cannot check queue access against UGI: " + ugi, exception); + throw new IOException(exception); } } From 44ceefea86931782844a97a9af562dffe55ecb86 Mon Sep 17 00:00:00 2001 From: Mahesh Raju Somalaraju Date: Tue, 21 Nov 2023 19:32:19 +0530 Subject: [PATCH 058/179] HIVE-27871: Fix formatting problems is YarnQueueHelper (#4874) (Mahesh Raju Somalaraju reviewed by Laszlo Bodor, Ayush Saxena) --- .../hive/ql/exec/tez/YarnQueueHelper.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java index c7065156c27b..45889ec750f6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/YarnQueueHelper.java @@ -51,29 +51,25 @@ public class YarnQueueHelper { private int lastKnownGoodUrl; private boolean sslForYarn; private boolean isHA; - private static String webapp_conf_key = YarnConfiguration.RM_WEBAPP_ADDRESS; - private static String webapp_ssl_conf_key = YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS; - private static String yarn_HA_enabled = YarnConfiguration.RM_HA_ENABLED; - private static String yarn_HA_rmids = YarnConfiguration.RM_HA_IDS; public YarnQueueHelper(HiveConf conf) { ArrayList nodeList = new ArrayList<>(); sslForYarn = YarnConfiguration.useHttps(conf); - isHA = conf.getBoolean(yarn_HA_enabled, false); - LOG.info(String.format("Yarn is using SSL: %s", sslForYarn)); - LOG.info(String.format("Yarn HA is enabled: %s", isHA)); + isHA = conf.getBoolean(YarnConfiguration.RM_HA_ENABLED, false); + LOG.info("Yarn is using SSL: {}, HA enabled: {}", sslForYarn, isHA); if (isHA) { - String[] rmids = conf.getStrings(yarn_HA_rmids); + String[] rmids = conf.getStrings(YarnConfiguration.RM_HA_IDS); if (sslForYarn == true) { for (String rmid : rmids) { - nodeList.addAll(Arrays.asList(conf.getTrimmedStrings(webapp_ssl_conf_key + "."+rmid))); + nodeList.addAll(Arrays.asList(conf + .getTrimmedStrings(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + "." + rmid))); } Preconditions.checkArgument(nodeList.size() > 0, "yarn.resourcemanager.ha.rm-ids must be set to enable queue access checks in Yarn HA mode"); }else{ for (String rmid : rmids) { - nodeList.addAll(Arrays.asList(conf.getTrimmedStrings(webapp_conf_key + "."+rmid))); + nodeList.addAll(Arrays.asList(conf.getTrimmedStrings(YarnConfiguration.RM_WEBAPP_ADDRESS + "."+rmid))); Preconditions.checkArgument(nodeList.size() > 0, "yarn.resourcemanager.ha.rm-ids must be set to enable queue access checks in Yarn HA mode"); } @@ -81,11 +77,11 @@ public YarnQueueHelper(HiveConf conf) { rmNodes = nodeList.toArray(new String[nodeList.size()]); }else { if (sslForYarn == true) { - rmNodes = conf.getTrimmedStrings(webapp_ssl_conf_key); + rmNodes = conf.getTrimmedStrings(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS); Preconditions.checkArgument((rmNodes != null && rmNodes.length > 0), "yarn.resourcemanager.webapp.https.address must be set to enable queue access checks using TLS"); } else { - rmNodes = conf.getTrimmedStrings(webapp_conf_key); + rmNodes = conf.getTrimmedStrings(YarnConfiguration.RM_WEBAPP_ADDRESS); Preconditions.checkArgument((rmNodes != null && rmNodes.length > 0), "yarn.resourcemanager.webapp.address must be set to enable queue access checks"); } From d411e500bfff023d0f97c796432341805ab0ddb7 Mon Sep 17 00:00:00 2001 From: dengzh Date: Wed, 22 Nov 2023 00:07:48 +0800 Subject: [PATCH 059/179] HIVE-27865: HMS in http mode drops down silently with no errors (#4866) (Zhihua Deng, reviewed by Ayush Saxena, Attila Turoczy) --- .../hadoop/hive/metastore/HiveMetaStore.java | 25 ++++++++----------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index ecf9904af9d3..986bb2e4e848 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -490,13 +490,11 @@ public void start() throws Throwable { HMSHandler.LOG.debug("Starting HTTPServer for HMS"); server.setStopAtShutdown(true); server.start(); - HMSHandler.LOG.info("Started the new HTTPServer for metastore on port [" + port - + "]..."); - HMSHandler.LOG.info("Options.minWorkerThreads = " - + minWorkerThreads); - HMSHandler.LOG.info("Options.maxWorkerThreads = " - + maxWorkerThreads); - HMSHandler.LOG.info("Enable SSL = " + (sslContextFactory != null)); + HMSHandler.LOG.info("Started the new HTTPServer for metastore on port [{}]...", port); + HMSHandler.LOG.info("Options.minWorkerThreads = {}", minWorkerThreads); + HMSHandler.LOG.info("Options.maxWorkerThreads = {}", maxWorkerThreads); + HMSHandler.LOG.info("Enable SSL = {}", (sslContextFactory != null)); + server.join(); } @Override @@ -642,15 +640,12 @@ public void processContext(ServerContext serverContext, TTransport tTransport, T return new ThriftServer() { @Override public void start() throws Throwable { + HMSHandler.LOG.info("Started the new metaserver on port [{}]...", port); + HMSHandler.LOG.info("Options.minWorkerThreads = {}", minWorkerThreads); + HMSHandler.LOG.info("Options.maxWorkerThreads = {}", maxWorkerThreads); + HMSHandler.LOG.info("TCP keepalive = {}", tcpKeepAlive); + HMSHandler.LOG.info("Enable SSL = {}", useSSL); tServer.serve(); - HMSHandler.LOG.info("Started the new metaserver on port [" + port - + "]..."); - HMSHandler.LOG.info("Options.minWorkerThreads = " - + minWorkerThreads); - HMSHandler.LOG.info("Options.maxWorkerThreads = " - + maxWorkerThreads); - HMSHandler.LOG.info("TCP keepalive = " + tcpKeepAlive); - HMSHandler.LOG.info("Enable SSL = " + useSSL); } @Override From ae64e6de113d8eb7b7a350d1b4430f16c824244d Mon Sep 17 00:00:00 2001 From: harshal-16 <109334642+harshal-16@users.noreply.github.com> Date: Wed, 22 Nov 2023 06:44:27 +0530 Subject: [PATCH 060/179] HIVE-27833: Hive Acid Replication Support for Dell Powerscale (#4841) (Harshal Patel, reviewed by Teddy Choi) --- .../apache/hadoop/hive/common/FileUtils.java | 24 +++++++++++++++---- .../org/apache/hadoop/hive/conf/HiveConf.java | 2 ++ .../hadoop/hive/common/TestFileUtils.java | 15 ++++++++++++ .../hadoop/hive/ql/parse/repl/CopyUtils.java | 2 +- .../hadoop/hive/shims/Hadoop23Shims.java | 18 ++++++++++---- .../org/apache/hadoop/hive/shims/Utils.java | 8 +++++-- 6 files changed, 56 insertions(+), 13 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java index 18efe167a632..be994461f318 100644 --- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java +++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.common; +import static org.apache.hadoop.hive.shims.Utils.RAW_RESERVED_VIRTUAL_PATH; + import java.io.EOFException; import java.io.File; import java.io.FileNotFoundException; @@ -61,11 +63,13 @@ import org.apache.hadoop.fs.PathExistsException; import org.apache.hadoop.fs.PathIsDirectoryException; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.HadoopShims; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; +import com.google.common.base.Preconditions; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.functional.RemoteIterators; import org.apache.hive.common.util.ShutdownHookManager; @@ -767,7 +771,7 @@ static boolean copy(FileSystem srcFS, Path src, // is tried and it fails. We depend upon that behaviour in cases like replication, // wherein if distcp fails, there is good reason to not plod along with a trivial // implementation, and fail instead. - copied = doIOUtilsCopyBytes(srcFS, srcFS.getFileStatus(src), dstFS, dst, deleteSource, overwrite, shouldPreserveXAttrs(conf, srcFS, dstFS), conf, copyStatistics); + copied = doIOUtilsCopyBytes(srcFS, srcFS.getFileStatus(src), dstFS, dst, deleteSource, overwrite, shouldPreserveXAttrs(conf, srcFS, dstFS, src), conf, copyStatistics); } return copied; } @@ -895,11 +899,21 @@ private static void checkDependencies(FileSystem srcFS, Path src, FileSystem dst } } - public static boolean shouldPreserveXAttrs(HiveConf conf, FileSystem srcFS, FileSystem dstFS) throws IOException { - if (!Utils.checkFileSystemXAttrSupport(srcFS) || !Utils.checkFileSystemXAttrSupport(dstFS)){ - return false; + public static boolean shouldPreserveXAttrs(HiveConf conf, FileSystem srcFS, FileSystem dstFS, Path path) throws IOException { + Preconditions.checkNotNull(path); + if (conf.getBoolVar(ConfVars.DFS_XATTR_ONLY_SUPPORTED_ON_RESERVED_NAMESPACE)) { + + if (!(path.toUri().getPath().startsWith(RAW_RESERVED_VIRTUAL_PATH) + && Utils.checkFileSystemXAttrSupport(srcFS, new Path(RAW_RESERVED_VIRTUAL_PATH)) + && Utils.checkFileSystemXAttrSupport(dstFS, new Path(RAW_RESERVED_VIRTUAL_PATH)))) { + return false; + } + } else { + if (!Utils.checkFileSystemXAttrSupport(srcFS) || !Utils.checkFileSystemXAttrSupport(dstFS)) { + return false; + } } - for (Map.Entry entry : conf.getPropsWithPrefix(Utils.DISTCP_OPTIONS_PREFIX).entrySet()) { + for (Map.Entry entry : conf.getPropsWithPrefix(Utils.DISTCP_OPTIONS_PREFIX).entrySet()) { String distCpOption = entry.getKey(); if (distCpOption.startsWith("p")) { return distCpOption.contains("x"); diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index ec463178912f..027bab6eb53d 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -487,6 +487,8 @@ public static enum ConfVars { MSC_CACHE_RECORD_STATS("hive.metastore.client.cache.v2.recordStats", false, "This property enables recording metastore client cache stats in DEBUG logs"), // QL execution stuff + DFS_XATTR_ONLY_SUPPORTED_ON_RESERVED_NAMESPACE("dfs.xattr.supported.only.on.reserved.namespace", false, + "DFS supports xattr only on Reserved Name space (/.reserved/raw)"), SCRIPTWRAPPER("hive.exec.script.wrapper", null, ""), PLAN("hive.exec.plan", "", ""), STAGINGDIR("hive.exec.stagingdir", ".hive-staging", diff --git a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/common/TestFileUtils.java b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/common/TestFileUtils.java index 732180158a23..807d4694fc08 100644 --- a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/common/TestFileUtils.java +++ b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/common/TestFileUtils.java @@ -133,6 +133,21 @@ public void testXAttrsPreserved() throws Exception { verifyXAttrsPreserved(src, new Path(dst, src.getName())); } + @Test + public void testShouldPreserveXAttrs() throws Exception { + conf.setBoolean(HiveConf.ConfVars.DFS_XATTR_ONLY_SUPPORTED_ON_RESERVED_NAMESPACE.varname, true); + Path filePath = new Path(basePath, "src.txt"); + fs.create(filePath).close(); + Assert.assertFalse(FileUtils.shouldPreserveXAttrs(conf, fs, fs, filePath)); + Path reservedRawPath = new Path("/.reserved/raw/", "src1.txt"); + fs.create(reservedRawPath).close(); + Assert.assertTrue(FileUtils.shouldPreserveXAttrs(conf, fs, fs, reservedRawPath)); + + conf.setBoolean(HiveConf.ConfVars.DFS_XATTR_ONLY_SUPPORTED_ON_RESERVED_NAMESPACE.varname, false); + Assert.assertTrue(FileUtils.shouldPreserveXAttrs(conf, fs, fs, filePath)); + Assert.assertTrue(FileUtils.shouldPreserveXAttrs(conf, fs, fs, reservedRawPath)); + } + private void verifyXAttrsPreserved(Path src, Path dst) throws Exception { FileStatus srcStatus = fs.getFileStatus(src); FileStatus dstStatus = fs.getFileStatus(dst); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java index 4dc640ca73fb..eb55b549fb89 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java @@ -127,7 +127,7 @@ void copyFilesBetweenFS(FileSystem srcFS, Path[] paths, FileSystem dstFS, Path dst, boolean deleteSource, boolean overwrite, DataCopyStatistics copyStatistics) throws IOException { retryableFxn(() -> { - boolean preserveXAttrs = FileUtils.shouldPreserveXAttrs(hiveConf, srcFS, dstFS); + boolean preserveXAttrs = FileUtils.shouldPreserveXAttrs(hiveConf, srcFS, dstFS, paths[0]); FileUtils.copy(srcFS, paths, dstFS, dst, deleteSource, overwrite, preserveXAttrs, hiveConf, copyStatistics); return null; diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java index 52e9d1f4503d..0437417e4b47 100644 --- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java +++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java @@ -108,6 +108,7 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.test.MiniTezCluster; +import static org.apache.hadoop.hive.shims.Utils.RAW_RESERVED_VIRTUAL_PATH; import static org.apache.hadoop.tools.DistCpConstants.CONF_LABEL_DISTCP_JOB_ID; /** @@ -1047,7 +1048,7 @@ public HadoopShims.StoragePolicyShim getStoragePolicyShim(FileSystem fs) { List constructDistCpParams(List srcPaths, Path dst, Configuration conf) throws IOException { // -update and -delete are mandatory options for directory copy to work. List params = constructDistCpDefaultParams(conf, dst.getFileSystem(conf), - srcPaths.get(0).getFileSystem(conf)); + srcPaths.get(0).getFileSystem(conf), srcPaths); if (!params.contains("-delete")) { params.add("-delete"); } @@ -1059,7 +1060,7 @@ List constructDistCpParams(List srcPaths, Path dst, Configuration } private List constructDistCpDefaultParams(Configuration conf, FileSystem dstFs, - FileSystem sourceFs) throws IOException { + FileSystem sourceFs, List srcPaths) throws IOException { List params = new ArrayList(); boolean needToAddPreserveOption = true; for (Map.Entry entry : conf.getPropsWithPrefix(Utils.DISTCP_OPTIONS_PREFIX).entrySet()){ @@ -1074,8 +1075,15 @@ private List constructDistCpDefaultParams(Configuration conf, FileSystem } } if (needToAddPreserveOption) { - params.add((Utils.checkFileSystemXAttrSupport(dstFs) - && Utils.checkFileSystemXAttrSupport(sourceFs)) ? "-pbx" : "-pb"); + if (conf.getBoolean("dfs.xattr.supported.only.on.reserved.namespace", false)) { + boolean shouldCopyXAttrs = srcPaths.get(0).toUri().getPath().startsWith(RAW_RESERVED_VIRTUAL_PATH) + && Utils.checkFileSystemXAttrSupport(sourceFs, new Path(RAW_RESERVED_VIRTUAL_PATH)) + && Utils.checkFileSystemXAttrSupport(dstFs, new Path(RAW_RESERVED_VIRTUAL_PATH)); + params.add(shouldCopyXAttrs ? "-pbx" : "-pb"); + } else { + params.add((Utils.checkFileSystemXAttrSupport(dstFs) + && Utils.checkFileSystemXAttrSupport(sourceFs)) ? "-pbx" : "-pb"); + } } if (!params.contains("-update")) { params.add("-update"); @@ -1097,7 +1105,7 @@ List constructDistCpWithSnapshotParams(List srcPaths, Path dst, St Configuration conf, String diff) throws IOException { // Get the default distcp params List params = constructDistCpDefaultParams(conf, dst.getFileSystem(conf), - srcPaths.get(0).getFileSystem(conf)); + srcPaths.get(0).getFileSystem(conf), srcPaths); if (params.contains("-delete")) { params.remove("-delete"); } diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/Utils.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/Utils.java index 339f0b5e9c9a..12566d8d93b0 100644 --- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/Utils.java +++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/Utils.java @@ -49,7 +49,7 @@ public class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); - + public static final String RAW_RESERVED_VIRTUAL_PATH = "/.reserved/raw/"; private static final boolean IBM_JAVA = System.getProperty("java.vendor") .contains("IBM"); @@ -165,8 +165,12 @@ public static Filter getXSRFFilter() { } public static boolean checkFileSystemXAttrSupport(FileSystem fs) throws IOException { + return checkFileSystemXAttrSupport(fs, new Path(Path.SEPARATOR)); + } + + public static boolean checkFileSystemXAttrSupport(FileSystem fs, Path path) throws IOException { try { - fs.getXAttrs(new Path(Path.SEPARATOR)); + fs.getXAttrs(path); return true; } catch (UnsupportedOperationException e) { LOG.warn("XAttr won't be preserved since it is not supported for file system: " + fs.getUri()); From 4d6d0ed140901fbcd1b3a37699b838e2ba6be689 Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Wed, 22 Nov 2023 05:03:41 +0100 Subject: [PATCH 061/179] HIVE-27788: Exception when join has 2 Group By operators in the same branch in the same reducer (Krisztian Kasa, reviewed by Stamatis Zampetakis) --- .../hadoop/hive/ql/exec/OperatorUtils.java | 41 ++++ .../hive/ql/optimizer/ConvertJoinMapJoin.java | 17 ++ .../hive/ql/exec/TestOperatorUtils.java | 168 +++++++++++++++ .../clientpositive/auto_sortmerge_join_17.q | 20 ++ .../llap/auto_sortmerge_join_17.q.out | 195 ++++++++++++++++++ .../llap/subquery_in_having.q.out | 182 ++++++++-------- 6 files changed, 541 insertions(+), 82 deletions(-) create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorUtils.java create mode 100644 ql/src/test/queries/clientpositive/auto_sortmerge_join_17.q create mode 100644 ql/src/test/results/clientpositive/llap/auto_sortmerge_join_17.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java index 3924164166b8..eb10e9aa6cdd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java @@ -176,6 +176,47 @@ public static Set findOperatorsUpstreamJoinAccounted(Operator start, C return found; } + /** + * Check whether there are more operators in the specified operator tree branch than the given limit + * until a ReduceSinkOperator is reached. + * The method traverses the parent operators of the specified root operator in dept first manner. + * @param start root of the operator tree to check + * @param opClazz type of operator to track + * @param limit maximum allowed number of operator in a branch of the tree + * @return true if limit is exceeded false otherwise + */ + public static boolean hasMoreOperatorsThan( + Operator start, Class opClazz, int limit) { + return hasMoreOperatorsThan(start, opClazz, limit, new HashSet<>()); + } + + private static boolean hasMoreOperatorsThan( + Operator start, Class opClazz, int limit, Set> visited) { + if (!visited.add(start)) { + return limit < 0; + } + + if (limit < 0) { + return false; + } + + if (start instanceof ReduceSinkOperator) { + return false; + } + + if (opClazz.isInstance(start)) { + limit--; + } + + if (start.getParentOperators() != null) { + for (Operator parent : start.getParentOperators()) { + if (hasMoreOperatorsThan(parent, opClazz, limit, visited)) { + return true; + } + } + } + return limit < 0; + } public static void setChildrenCollector(List> childOperators, OutputCollector out) { if (childOperators == null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java index 57299f95699f..98a80ab603ee 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java @@ -83,6 +83,8 @@ import com.google.common.base.Preconditions; import com.google.common.math.DoubleMath; +import static org.apache.hadoop.hive.ql.exec.OperatorUtils.hasMoreOperatorsThan; + /** * ConvertJoinMapJoin is an optimization that replaces a common join * (aka shuffle join) with a map join (aka broadcast or fragment replicate @@ -755,6 +757,21 @@ private boolean checkConvertJoinSMBJoin(JoinOperator joinOp, OptimizeTezProcCont LOG.debug("External table {} found in join and also could not provide statistics - disabling SMB join.", sb); return false; } + // GBY operators buffers one record. These are processed when ReduceRecordSources flushes the operator tree + // when end of record stream reached. If the tree has more than two GBY operators CommonMergeJoinOperator can + // not process all buffered records. + // HIVE-27788 + if (parentOp.getParentOperators() != null) { + // Parent operator is RS and hasMoreOperatorsThan traverses until the next RS, so we start from grandparent + for (Operator grandParent : parentOp.getParentOperators()) { + if (hasMoreOperatorsThan(grandParent, GroupByOperator.class, 1)) { + LOG.info("We cannot convert to SMB join " + + "because one of the join branches has more than one Group by operators in the same reducer."); + return false; + } + } + } + // each side better have 0 or more RS. if either side is unbalanced, cannot convert. // This is a workaround for now. Right fix would be to refactor code in the // MapRecordProcessor and ReduceRecordProcessor with respect to the sources. diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorUtils.java new file mode 100644 index 000000000000..6eec95a23adb --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorUtils.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.hadoop.hive.ql.CompilationOpContext; +import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc; +import org.apache.hadoop.hive.ql.plan.GroupByDesc; +import org.apache.hadoop.hive.ql.plan.LimitDesc; +import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.junit.jupiter.api.Test; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.*; + +class TestOperatorUtils { + @Test + void testHasMoreGBYsReturnsFalseWhenLimitIs0() { + // RS-SEL-LIM-FIL + CompilationOpContext context = new CompilationOpContext(); + Operator filter = OperatorFactory.get(context, FilterDesc.class); + Operator limit = OperatorFactory.get(context, LimitDesc.class); + filter.setParentOperators(singletonList(limit)); + Operator select = OperatorFactory.get(context, SelectDesc.class); + limit.setParentOperators(singletonList(select)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + select.setParentOperators(singletonList(rs)); + + assertFalse(OperatorUtils.hasMoreOperatorsThan(filter, GroupByOperator.class, 0)); + } + + @Test + void testHasMoreGBYsReturnsFalseWhenNoGBYInBranchAndLimitIsMoreThan0() { + // RS-SEL-LIM-FIL + CompilationOpContext context = new CompilationOpContext(); + Operator filter = OperatorFactory.get(context, FilterDesc.class); + Operator limit = OperatorFactory.get(context, LimitDesc.class); + filter.setParentOperators(singletonList(limit)); + Operator select = OperatorFactory.get(context, SelectDesc.class); + limit.setParentOperators(singletonList(select)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + select.setParentOperators(singletonList(rs)); + + assertFalse(OperatorUtils.hasMoreOperatorsThan(filter, GroupByOperator.class, 1)); + } + + @Test + void testHasMoreGBYsReturnsFalseWhenNumberOfGBYIsLessThanLimit() { + // RS-GBY-SEL-LIM-FIL + CompilationOpContext context = new CompilationOpContext(); + Operator filter = OperatorFactory.get(context, FilterDesc.class); + Operator limit = OperatorFactory.get(context, LimitDesc.class); + filter.setParentOperators(singletonList(limit)); + Operator select = OperatorFactory.get(context, SelectDesc.class); + limit.setParentOperators(singletonList(select)); + Operator gby = OperatorFactory.get(context, GroupByDesc.class); + select.setParentOperators(singletonList(gby)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + gby.setParentOperators(singletonList(rs)); + + assertFalse(OperatorUtils.hasMoreOperatorsThan(filter, GroupByOperator.class, 1)); + } + + @Test + void testHasMoreGBYsReturnsTrueWhenNumberOfGBYIsEqualsWithLimit() { + // RS-GBY-FIL-SEL-GBY + CompilationOpContext context = new CompilationOpContext(); + Operator gby1 = OperatorFactory.get(context, GroupByDesc.class); + Operator select = OperatorFactory.get(context, SelectDesc.class); + gby1.setParentOperators(singletonList(select)); + Operator filter = OperatorFactory.get(context, FilterDesc.class); + select.setParentOperators(singletonList(filter)); + Operator gby2 = OperatorFactory.get(context, GroupByDesc.class); + filter.setParentOperators(singletonList(gby2)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + gby2.setParentOperators(singletonList(rs)); + + assertTrue(OperatorUtils.hasMoreOperatorsThan(gby1, GroupByOperator.class, 1)); + } + + @Test + void testHasMoreGBYsReturnsFalseWhenNumberOfGBYIsEqualsWithLimitButHasAnRSInTheMiddle() { + // TS-GBY-RS-SEL-GBY + CompilationOpContext context = new CompilationOpContext(); + Operator gby1 = OperatorFactory.get(context, GroupByDesc.class); + Operator select = OperatorFactory.get(context, SelectDesc.class); + gby1.setParentOperators(singletonList(select)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + select.setParentOperators(singletonList(rs)); + Operator gby2 = OperatorFactory.get(context, GroupByDesc.class); + rs.setParentOperators(singletonList(gby2)); + Operator ts = OperatorFactory.get(context, TableScanDesc.class); + gby2.setParentOperators(singletonList(ts)); + + assertFalse(OperatorUtils.hasMoreOperatorsThan(gby1, GroupByOperator.class, 1)); + } + + @Test + void testHasMoreGBYsReturnsTrueWhenBranchHasJoinAndNumberOfGBYIsEqualsWithLimit() { + // RS-GBY-FIL--JOIN-GBY + // RS-SEL-/ + CompilationOpContext context = new CompilationOpContext(); + Operator gby1 = OperatorFactory.get(context, GroupByDesc.class); + Operator join = OperatorFactory.get(context, CommonMergeJoinDesc.class); + gby1.setParentOperators(singletonList(join)); + + // Join branch #1 has the second GBY + Operator filter = OperatorFactory.get(context, FilterDesc.class); + Operator gby2 = OperatorFactory.get(context, GroupByDesc.class); + filter.setParentOperators(singletonList(gby2)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + gby2.setParentOperators(singletonList(rs)); + + // Join branch #2 + Operator select = OperatorFactory.get(context, SelectDesc.class); + Operator rs2 = OperatorFactory.get(context, ReduceSinkDesc.class); + select.setParentOperators(singletonList(rs2)); + + join.setParentOperators(asList(filter, select)); + + assertTrue(OperatorUtils.hasMoreOperatorsThan(gby1, GroupByOperator.class, 1)); + } + + @Test + void testHasMoreGBYsReturnsFalseWhenBranchHasJoinAndBothJoinBranchesHasLessGBYThanLimit() { + // RS-GBY-SEL--JOIN + // RS-GBY-FIL-/ + CompilationOpContext context = new CompilationOpContext(); + Operator join = OperatorFactory.get(context, CommonMergeJoinDesc.class); + + // Join branch #1 + Operator filter = OperatorFactory.get(context, FilterDesc.class); + Operator gby1 = OperatorFactory.get(context, GroupByDesc.class); + filter.setParentOperators(singletonList(gby1)); + Operator rs = OperatorFactory.get(context, ReduceSinkDesc.class); + gby1.setParentOperators(singletonList(rs)); + + // Join branch #2 + Operator select = OperatorFactory.get(context, SelectDesc.class); + Operator gby2 = OperatorFactory.get(context, GroupByDesc.class); + select.setParentOperators(singletonList(gby2)); + Operator rs2 = OperatorFactory.get(context, ReduceSinkDesc.class); + gby2.setParentOperators(singletonList(rs2)); + + join.setParentOperators(asList(filter, select)); + + assertFalse(OperatorUtils.hasMoreOperatorsThan(join, GroupByOperator.class, 1)); + } +} \ No newline at end of file diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_17.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_17.q new file mode 100644 index 000000000000..0d6a5deadb07 --- /dev/null +++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_17.q @@ -0,0 +1,20 @@ +CREATE TABLE tbl1_n5(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +insert into tbl1_n5(key, value) +values +(0, 'val_0'), +(2, 'val_2'), +(9, 'val_9'); + +explain +SELECT t1.key from +(SELECT key , row_number() over(partition by key order by value desc) as rk from tbl1_n5) t1 +join +( SELECT key,count(distinct value) as cp_count from tbl1_n5 group by key) t2 +on t1.key = t2.key where rk = 1; + +SELECT t1.key from +(SELECT key , row_number() over(partition by key order by value desc) as rk from tbl1_n5) t1 +join +( SELECT key,count(distinct value) as cp_count from tbl1_n5 group by key) t2 +on t1.key = t2.key where rk = 1; \ No newline at end of file diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_17.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_17.q.out new file mode 100644 index 000000000000..80527eb62402 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_17.q.out @@ -0,0 +1,195 @@ +PREHOOK: query: CREATE TABLE tbl1_n5(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl1_n5 +POSTHOOK: query: CREATE TABLE tbl1_n5(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl1_n5 +PREHOOK: query: insert into tbl1_n5(key, value) +values +(0, 'val_0'), +(2, 'val_2'), +(9, 'val_9') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl1_n5 +POSTHOOK: query: insert into tbl1_n5(key, value) +values +(0, 'val_0'), +(2, 'val_2'), +(9, 'val_9') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl1_n5 +POSTHOOK: Lineage: tbl1_n5.key SCRIPT [] +POSTHOOK: Lineage: tbl1_n5.value SCRIPT [] +PREHOOK: query: explain +SELECT t1.key from +(SELECT key , row_number() over(partition by key order by value desc) as rk from tbl1_n5) t1 +join +( SELECT key,count(distinct value) as cp_count from tbl1_n5 group by key) t2 +on t1.key = t2.key where rk = 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1_n5 +#### A masked pattern was here #### +POSTHOOK: query: explain +SELECT t1.key from +(SELECT key , row_number() over(partition by key order by value desc) as rk from tbl1_n5) t1 +join +( SELECT key,count(distinct value) as cp_count from tbl1_n5 group by key) t2 +on t1.key = t2.key where rk = 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1_n5 +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 4 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl1_n5 + filterExpr: key is not null (type: boolean) + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: key (type: int), value (type: string) + null sort order: aa + sort order: +- + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: key (type: int), value (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: string) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: string) + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: int, _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 DESC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT t1.key from +(SELECT key , row_number() over(partition by key order by value desc) as rk from tbl1_n5) t1 +join +( SELECT key,count(distinct value) as cp_count from tbl1_n5 group by key) t2 +on t1.key = t2.key where rk = 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1_n5 +#### A masked pattern was here #### +POSTHOOK: query: SELECT t1.key from +(SELECT key , row_number() over(partition by key order by value desc) as rk from tbl1_n5) t1 +join +( SELECT key,count(distinct value) as cp_count from tbl1_n5 group by key) t2 +on t1.key = t2.key where rk = 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1_n5 +#### A masked pattern was here #### +0 +2 +9 diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out index cd5989f5cd9a..bf88614de222 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out @@ -282,7 +282,9 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 4 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -308,17 +310,6 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: bigint) - Execution mode: vectorized, llap - LLAP IO: all inputs - Map 3 - Map Operator Tree: - TableScan - alias: part_subq - filterExpr: p_mfgr is not null (type: boolean) - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_mfgr is not null (type: boolean) - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: max(p_size), min(p_size) keys: p_mfgr (type: string) @@ -336,6 +327,45 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: all inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), (UDFToDouble(_col1) / _col2) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: double) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator aggregations: max(VALUE._col0), min(VALUE._col1) @@ -352,37 +382,16 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - mode: final + minReductionHashAggr: 0.99 + mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Dummy Store - Execution mode: llap - Reduce Operator Tree: - Group By Operator - aggregations: sum(VALUE._col0), count(VALUE._col1) - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), (UDFToDouble(_col1) / _col2) (type: double) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 Fetch Operator @@ -425,7 +434,9 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 4 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -451,17 +462,6 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: bigint) - Execution mode: vectorized, llap - LLAP IO: all inputs - Map 3 - Map Operator Tree: - TableScan - alias: part_subq - filterExpr: p_mfgr is not null (type: boolean) - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: p_mfgr is not null (type: boolean) - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: max(p_size), min(p_size) keys: p_mfgr (type: string) @@ -479,6 +479,45 @@ STAGE PLANS: Execution mode: vectorized, llap LLAP IO: all inputs Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), (UDFToDouble(_col1) / _col2) (type: double) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: double) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator aggregations: max(VALUE._col0), min(VALUE._col1) @@ -495,37 +534,16 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - mode: final + minReductionHashAggr: 0.99 + mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Dummy Store - Execution mode: llap - Reduce Operator Tree: - Group By Operator - aggregations: sum(VALUE._col0), count(VALUE._col1) - keys: KEY._col0 (type: string) - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), (UDFToDouble(_col1) / _col2) (type: double) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 Fetch Operator From a79097c246d77224d8c25f9450a0fb5f842c864e Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Wed, 22 Nov 2023 17:22:45 +0200 Subject: [PATCH 062/179] HIVE-26950: Iceberg: Addendum: CTLT sets incorrect 'TRANSLATED_TO_EXTERNAL' value (Denys Kuzmenko, reviewed by Ayush Saxena) Closes #4890 --- .../apache/hadoop/hive/conf/Constants.java | 7 ++--- .../mr/hive/HiveIcebergStorageHandler.java | 19 ++++++------- .../src/test/queries/positive/ctlt_iceberg.q | 5 ++-- .../test/results/positive/ctlt_iceberg.q.out | 3 +++ .../hive/ql/parse/SemanticAnalyzer.java | 5 ++-- .../repl/dump/io/PartitionSerializer.java | 6 ----- .../hadoop/hive/metastore/HiveMetaHook.java | 27 ++++++++++--------- .../hive/metastore/utils/MetaStoreUtils.java | 9 +++++-- .../hive/metastore/HiveAlterHandler.java | 4 +-- .../MetastoreDefaultTransformer.java | 8 +++--- 10 files changed, 48 insertions(+), 45 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/Constants.java b/common/src/java/org/apache/hadoop/hive/conf/Constants.java index 5d2550cbbd6a..7bd736c130be 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/Constants.java +++ b/common/src/java/org/apache/hadoop/hive/conf/Constants.java @@ -98,16 +98,13 @@ public class Constants { public static final String ORC_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; public static final String ORC_OUTPUT_FORMAT = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; - - + public static final Pattern COMPACTION_POOLS_PATTERN = Pattern.compile("hive\\.compactor\\.worker\\.(.*)\\.threads"); public static final String HIVE_COMPACTOR_WORKER_POOL = "hive.compactor.worker.pool"; - public static final String HIVE_COMPACTOR_REBALANCE_ORDERBY = "hive.compactor.rebalance.orderby"; public static final String HTTP_HEADER_REQUEST_TRACK = "X-Request-ID"; public static final String TIME_POSTFIX_REQUEST_TRACK = "_TIME"; - - public static final String ICEBERG = "iceberg"; + public static final String ICEBERG_PARTITION_TABLE_SCHEMA = "partition,spec_id,record_count,file_count," + "position_delete_record_count,position_delete_file_count,equality_delete_record_count," + "equality_delete_file_count,last_updated_at,total_data_file_size_in_bytes,last_updated_snapshot_id"; diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 1088d6d43025..604f078f79c5 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -63,6 +63,7 @@ import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.ErrorMsg; @@ -441,7 +442,7 @@ public Map getBasicStatistics(Partish partish) { // For write queries where rows got modified, don't fetch from cache as values could have changed. Table table = getTable(hmsTable); Map stats = Maps.newHashMap(); - if (getStatsSource().equals(Constants.ICEBERG)) { + if (getStatsSource().equals(HiveMetaHook.ICEBERG)) { if (table.currentSnapshot() != null) { Map summary = table.currentSnapshot().summary(); if (summary != null) { @@ -492,7 +493,7 @@ private Table getTable(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { @Override public boolean canSetColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); - return table.currentSnapshot() != null && getStatsSource().equals(Constants.ICEBERG); + return table.currentSnapshot() != null && getStatsSource().equals(HiveMetaHook.ICEBERG); } @Override @@ -568,7 +569,7 @@ private ColumnStatistics readColStats(Table table, Path statsPath) { @Override public boolean canComputeQueryUsingStats(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { - if (getStatsSource().equals(Constants.ICEBERG)) { + if (getStatsSource().equals(HiveMetaHook.ICEBERG)) { Table table = getTable(hmsTable); if (table.currentSnapshot() != null) { Map summary = table.currentSnapshot().summary(); @@ -585,7 +586,8 @@ public boolean canComputeQueryUsingStats(org.apache.hadoop.hive.ql.metadata.Tabl } private String getStatsSource() { - return HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_ICEBERG_STATS_SOURCE, Constants.ICEBERG).toLowerCase(); + return HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_ICEBERG_STATS_SOURCE, HiveMetaHook.ICEBERG) + .toUpperCase(); } private Path getColStatsPath(Table table) { @@ -1581,9 +1583,8 @@ public void prepareAlterTableEnvironmentContext(AbstractAlterTableDesc alterTabl public void setTableParametersForCTLT(org.apache.hadoop.hive.ql.metadata.Table tbl, CreateTableLikeDesc desc, Map origParams) { // Preserve the format-version of the iceberg table and filter out rest. - String formatVersion = origParams.get(TableProperties.FORMAT_VERSION); - if ("2".equals(formatVersion)) { - tbl.getParameters().put(TableProperties.FORMAT_VERSION, formatVersion); + if (IcebergTableUtil.isV2Table(origParams)) { + tbl.getParameters().put(TableProperties.FORMAT_VERSION, "2"); tbl.getParameters().put(TableProperties.DELETE_MODE, MERGE_ON_READ); tbl.getParameters().put(TableProperties.UPDATE_MODE, MERGE_ON_READ); tbl.getParameters().put(TableProperties.MERGE_MODE, MERGE_ON_READ); @@ -1591,12 +1592,12 @@ public void setTableParametersForCTLT(org.apache.hadoop.hive.ql.metadata.Table t // check if the table is being created as managed table, in that case we translate it to external if (!desc.isExternal()) { - tbl.getParameters().put("TRANSLATED_TO_EXTERNAL", "TRUE"); + tbl.getParameters().put(HiveMetaHook.TRANSLATED_TO_EXTERNAL, "TRUE"); desc.setIsExternal(true); } // If source is Iceberg table set the schema and the partition spec - if ("ICEBERG".equalsIgnoreCase(origParams.get("table_type"))) { + if (MetaStoreUtils.isIcebergTable(origParams)) { tbl.getParameters() .put(InputFormatConfig.TABLE_SCHEMA, origParams.get(InputFormatConfig.TABLE_SCHEMA)); tbl.getParameters() diff --git a/iceberg/iceberg-handler/src/test/queries/positive/ctlt_iceberg.q b/iceberg/iceberg-handler/src/test/queries/positive/ctlt_iceberg.q index 895f51a4ff29..13d97f1ca486 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/ctlt_iceberg.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/ctlt_iceberg.q @@ -3,8 +3,6 @@ -- Mask random uuid --! qt:replace:/(\s+'uuid'=')\S+('\s*)/$1#Masked#$2/ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; set hive.explain.user=false; create table source(a int) stored by iceberg tblproperties ('format-version'='2') ; @@ -47,6 +45,9 @@ create table emp_like2 like emp stored by iceberg; -- Partition column should be there show create table emp_like2; +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + -- create a managed table create managed table man_table (id int) Stored as orc TBLPROPERTIES ('transactional'='true'); diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out index d25fde849cdc..7fe78cad78a2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctlt_iceberg.q.out @@ -38,6 +38,7 @@ STORED BY LOCATION 'hdfs://### HDFS PATH ###' TBLPROPERTIES ( + 'TRANSLATED_TO_EXTERNAL'='TRUE', 'bucketing_version'='2', 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"a","required":false,"type":"int"}]}', @@ -162,6 +163,7 @@ STORED BY LOCATION 'hdfs://### HDFS PATH ###' TBLPROPERTIES ( + 'TRANSLATED_TO_EXTERNAL'='TRUE', 'bucketing_version'='2', 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', @@ -233,6 +235,7 @@ STORED BY LOCATION 'hdfs://### HDFS PATH ###' TBLPROPERTIES ( + 'TRANSLATED_TO_EXTERNAL'='TRUE', 'bucketing_version'='2', 'created_with_ctlt'='true', 'current-schema'='{"type":"struct","schema-id":0,"fields":[{"id":1,"name":"id","required":false,"type":"int"},{"id":2,"name":"company","required":false,"type":"string"}]}', diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index ca0cc1798760..ba1fd1b49b2e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -14263,14 +14263,13 @@ ASTNode analyzeCreateTable( isTransactional, isManaged, new String[]{qualifiedTabName.getDb(), qualifiedTabName.getTable()}, isDefaultTableTypeChanged); tblProps.put(hive_metastoreConstants.TABLE_IS_CTLT, "true"); - isExt = isIcebergTable(tblProps) || - isExternalTableChanged(tblProps, isTransactional, isExt, isDefaultTableTypeChanged); + isExt = isExternalTableChanged(tblProps, isTransactional, isExt, isDefaultTableTypeChanged); addDbAndTabToOutputs(new String[] {qualifiedTabName.getDb(), qualifiedTabName.getTable()}, TableType.MANAGED_TABLE, isTemporary, tblProps, storageFormat); Table likeTable = getTable(likeTableName, false); if (likeTable != null) { - if (isTemporary || isExt) { + if (isTemporary || isExt || isIcebergTable(tblProps)) { updateDefaultTblProps(likeTable.getParameters(), tblProps, new ArrayList<>(Arrays.asList(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES))); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/PartitionSerializer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/PartitionSerializer.java index 281bb37d32c2..68a9202433ce 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/PartitionSerializer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/PartitionSerializer.java @@ -67,10 +67,4 @@ public void writeTo(JsonWriter writer, ReplicationSpec additionalPropertiesProvi throw new SemanticException(ErrorMsg.ERROR_SERIALIZE_METASTORE.getMsg(), e); } } - - private boolean isPartitionExternal() { - Map params = partition.getParameters(); - return params.containsKey("EXTERNAL") - && params.get("EXTERNAL").equalsIgnoreCase("TRUE"); - } } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java index 8a0970a474b3..695a3282838c 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaHook.java @@ -42,18 +42,21 @@ @InterfaceStability.Stable public interface HiveMetaHook { - public String ALTER_TABLE_OPERATION_TYPE = "alterTableOpType"; + String ALTER_TABLE_OPERATION_TYPE = "alterTableOpType"; // These should remain in sync with AlterTableType enum - public List allowedAlterTypes = ImmutableList.of("ADDPROPS", "DROPPROPS"); + List allowedAlterTypes = ImmutableList.of("ADDPROPS", "DROPPROPS"); String ALTERLOCATION = "ALTERLOCATION"; String ALLOW_PARTITION_KEY_CHANGE = "allow_partition_key_change"; String SET_PROPERTIES = "set_properties"; String UNSET_PROPERTIES = "unset_properties"; - String TABLE_TYPE = "table_type"; + String TRANSLATED_TO_EXTERNAL = "TRANSLATED_TO_EXTERNAL"; + String TABLE_TYPE = "table_type"; + String EXTERNAL = "EXTERNAL"; String ICEBERG = "ICEBERG"; + String PROPERTIES_SEPARATOR = "'"; String MIGRATE_HIVE_TO_ICEBERG = "migrate_hive_to_iceberg"; String INITIALIZE_ROLLBACK_MIGRATION = "initialize_rollback_migration"; @@ -70,7 +73,7 @@ public interface HiveMetaHook { * * @param table new table definition */ - public void preCreateTable(Table table) + void preCreateTable(Table table) throws MetaException; /** @@ -79,7 +82,7 @@ public void preCreateTable(Table table) * * @param table new table definition */ - public void rollbackCreateTable(Table table) + void rollbackCreateTable(Table table) throws MetaException; /** @@ -88,7 +91,7 @@ public void rollbackCreateTable(Table table) * * @param table new table definition */ - public void commitCreateTable(Table table) + void commitCreateTable(Table table) throws MetaException; /** @@ -97,7 +100,7 @@ public void commitCreateTable(Table table) * * @param table table definition */ - public void preDropTable(Table table) + void preDropTable(Table table) throws MetaException; /** @@ -118,7 +121,7 @@ default void preDropTable(Table table, boolean deleteData) throws MetaException * * @param table table definition */ - public void rollbackDropTable(Table table) + void rollbackDropTable(Table table) throws MetaException; /** @@ -130,7 +133,7 @@ public void rollbackDropTable(Table table) * @param deleteData whether to delete data as well; this should typically * be ignored in the case of an external table */ - public void commitDropTable(Table table, boolean deleteData) + void commitDropTable(Table table, boolean deleteData) throws MetaException; /** @@ -139,7 +142,7 @@ public void commitDropTable(Table table, boolean deleteData) * * @param table new table definition */ - public default void preAlterTable(Table table, EnvironmentContext context) throws MetaException { + default void preAlterTable(Table table, EnvironmentContext context) throws MetaException { String alterOpType = (context == null || context.getProperties() == null) ? null : context.getProperties().get(ALTER_TABLE_OPERATION_TYPE); // By default allow only ADDPROPS and DROPPROPS. @@ -175,11 +178,11 @@ default void rollbackAlterTable(Table table, EnvironmentContext context) throws * @param context context of the truncate operation * @throws MetaException */ - public default void preTruncateTable(Table table, EnvironmentContext context) throws MetaException { + default void preTruncateTable(Table table, EnvironmentContext context) throws MetaException { preTruncateTable(table, context, null); } - public default void preTruncateTable(Table table, EnvironmentContext context, List partNames) throws MetaException { + default void preTruncateTable(Table table, EnvironmentContext context, List partNames) throws MetaException { // Do nothing } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java index 1491b45a3537..1aa98d9f1d18 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.common.repl.ReplConst; import org.apache.hadoop.hive.metastore.ColumnType; +import org.apache.hadoop.hive.metastore.HiveMetaHook; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -286,10 +287,14 @@ public static boolean isExternalTable(Table table) { return isExternal(params); } + public static boolean isIcebergTable(Map params) { + return HiveMetaHook.ICEBERG.equalsIgnoreCase(params.get(HiveMetaHook.TABLE_TYPE)); + } + public static boolean isTranslatedToExternalTable(Table table) { Map params = table.getParameters(); - return params != null && MetaStoreUtils.isPropertyTrue(params, "EXTERNAL") - && MetaStoreUtils.isPropertyTrue(params, "TRANSLATED_TO_EXTERNAL") && table.getSd() != null + return params != null && MetaStoreUtils.isPropertyTrue(params, HiveMetaHook.EXTERNAL) + && MetaStoreUtils.isPropertyTrue(params, HiveMetaHook.TRANSLATED_TO_EXTERNAL) && table.getSd() != null && table.getSd().isSetLocation(); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java index 9bce99ad6801..1c18631e1cc3 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java @@ -254,9 +254,9 @@ public void alterTable(RawStore msdb, Warehouse wh, String catName, String dbnam boolean renamedTranslatedToExternalTable = rename && MetaStoreUtils.isTranslatedToExternalTable(oldt) && MetaStoreUtils.isTranslatedToExternalTable(newt); boolean renamedExternalTable = rename && MetaStoreUtils.isExternalTable(oldt) - && !MetaStoreUtils.isPropertyTrue(oldt.getParameters(), "TRANSLATED_TO_EXTERNAL"); + && !MetaStoreUtils.isPropertyTrue(oldt.getParameters(), HiveMetaHook.TRANSLATED_TO_EXTERNAL); boolean isRenameIcebergTable = - rename && HiveMetaHook.ICEBERG.equalsIgnoreCase(newt.getParameters().get(HiveMetaHook.TABLE_TYPE)); + rename && MetaStoreUtils.isIcebergTable(newt.getParameters()); List columnStatistics = getColumnStats(msdb, oldt); columnStatistics = deleteTableColumnStats(msdb, oldt, newt, columnStatistics); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java index 481743005167..ff92ab86d420 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java @@ -652,9 +652,9 @@ public Table transformCreateTable(Table table, List processorCapabilitie newTable.setTableType(TableType.EXTERNAL_TABLE.toString()); params.remove(TABLE_IS_TRANSACTIONAL); params.remove(TABLE_TRANSACTIONAL_PROPERTIES); - params.put("EXTERNAL", "TRUE"); + params.put(HiveMetaHook.EXTERNAL, "TRUE"); params.put(EXTERNAL_TABLE_PURGE, "TRUE"); - params.put("TRANSLATED_TO_EXTERNAL", "TRUE"); + params.put(HiveMetaHook.TRANSLATED_TO_EXTERNAL, "TRUE"); newTable.setParameters(params); LOG.info("Modified table params are:" + params.toString()); @@ -782,8 +782,8 @@ private boolean isTableRename(Table oldTable, Table newTable) { private boolean isTranslatedToExternalTable(Table table) { Map p = table.getParameters(); - return p != null && MetaStoreUtils.isPropertyTrue(p, "EXTERNAL") - && MetaStoreUtils.isPropertyTrue(p, "TRANSLATED_TO_EXTERNAL") && table.getSd() != null + return p != null && MetaStoreUtils.isPropertyTrue(p, HiveMetaHook.EXTERNAL) + && MetaStoreUtils.isPropertyTrue(p, HiveMetaHook.TRANSLATED_TO_EXTERNAL) && table.getSd() != null && table.getSd().isSetLocation(); } From ae89034eb424b3a51b001ac1df8781c4e507b9e0 Mon Sep 17 00:00:00 2001 From: Naresh P R Date: Wed, 22 Nov 2023 09:11:31 -0800 Subject: [PATCH 063/179] HIVE-27885 : Cast decimal from string with space without digits before dot returns NULL (#4876) --- ql/src/test/queries/clientpositive/cast2.q | 3 +++ ql/src/test/results/clientpositive/llap/cast2.q.out | 9 +++++++++ .../hadoop/hive/common/type/FastHiveDecimalImpl.java | 1 + 3 files changed, 13 insertions(+) diff --git a/ql/src/test/queries/clientpositive/cast2.q b/ql/src/test/queries/clientpositive/cast2.q index 12228befb568..bd4835c62809 100644 --- a/ql/src/test/queries/clientpositive/cast2.q +++ b/ql/src/test/queries/clientpositive/cast2.q @@ -3,3 +3,6 @@ select cast('1' as tinyint), cast('1' as smallint), cast('1' as int), cast('1' a -- Check that leading/trailing space is handled consistently for numeric types select cast(' 1 ' as tinyint), cast(' 1 ' as smallint), cast(' 1 ' as int), cast(' 1 ' as bigint), cast(' 1 ' as float), cast(' 1 ' as double), cast(' 1 ' as decimal(10,2)); + +-- Decimal cast with spaces/without digits before dot & only dot. +select cast(".0000 " as decimal(8,4)), cast(" .0000" as decimal(8,4)), cast(" .0000 " as decimal(8,4)), cast("." as decimal(8,4)), cast(". " as decimal(8,4)), cast(" . " as decimal(8,4)), cast(".00 00 " as decimal(8,4)); diff --git a/ql/src/test/results/clientpositive/llap/cast2.q.out b/ql/src/test/results/clientpositive/llap/cast2.q.out index d5d6bbac2c68..e85f4945488a 100644 --- a/ql/src/test/results/clientpositive/llap/cast2.q.out +++ b/ql/src/test/results/clientpositive/llap/cast2.q.out @@ -16,3 +16,12 @@ POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table #### A masked pattern was here #### 1 1 1 1 1.0 1.0 1.00 +PREHOOK: query: select cast(".0000 " as decimal(8,4)), cast(" .0000" as decimal(8,4)), cast(" .0000 " as decimal(8,4)), cast("." as decimal(8,4)), cast(". " as decimal(8,4)), cast(" . " as decimal(8,4)), cast(".00 00 " as decimal(8,4)) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: select cast(".0000 " as decimal(8,4)), cast(" .0000" as decimal(8,4)), cast(" .0000 " as decimal(8,4)), cast("." as decimal(8,4)), cast(". " as decimal(8,4)), cast(" . " as decimal(8,4)), cast(".00 00 " as decimal(8,4)) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +0.0000 0.0000 0.0000 NULL NULL NULL NULL diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java index f9466d7c9ca9..65429e10b855 100644 --- a/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java +++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java @@ -443,6 +443,7 @@ public static boolean fastSetFromBytes(byte[] bytes, int offset, int length, boo } break; } + haveInteger = true; digitValue = work - BYTE_DIGIT_ZERO; if (digitNum == 0) { From 7638cb1a3b07713cc490aa2909a37037f89e08b4 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Thu, 23 Nov 2023 11:50:50 +0530 Subject: [PATCH 064/179] Refactor Some Code. (#4887). (Ayush Saxena, reviewed by Denys Kuzmenko) --- service/src/java/org/apache/hive/service/CookieSigner.java | 4 +--- .../src/test/org/apache/hive/service/TestCookieSigner.java | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/service/src/java/org/apache/hive/service/CookieSigner.java b/service/src/java/org/apache/hive/service/CookieSigner.java index 49ef2c606a7c..867adc8fa267 100644 --- a/service/src/java/org/apache/hive/service/CookieSigner.java +++ b/service/src/java/org/apache/hive/service/CookieSigner.java @@ -76,10 +76,8 @@ public String verifyAndExtract(String signedStr) { String rawValue = signedStr.substring(0, index); String currentSignature = getSignature(rawValue); - LOG.debug("Signature generated for {} inside verify is {}", rawValue, currentSignature); if (!MessageDigest.isEqual(originalSignature.getBytes(), currentSignature.getBytes())) { - throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + - " current = " + currentSignature); + throw new IllegalArgumentException("Invalid sign= " + originalSignature); } return rawValue; } diff --git a/service/src/test/org/apache/hive/service/TestCookieSigner.java b/service/src/test/org/apache/hive/service/TestCookieSigner.java index 54801d444003..a5a6454194de 100644 --- a/service/src/test/org/apache/hive/service/TestCookieSigner.java +++ b/service/src/test/org/apache/hive/service/TestCookieSigner.java @@ -70,7 +70,7 @@ public void testVerifyAndExtractInvalidSignature() { try { cs.verifyAndExtract(modifedSignedStr); } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().startsWith("Invalid sign, original = ")); + assertTrue(e.getMessage().startsWith("Invalid sign= ")); return; } fail("Expected IllegalArgumentException checking signature"); From 5861b169c2b3a7aa889d204e53d4858173022c73 Mon Sep 17 00:00:00 2001 From: seonggon Date: Thu, 23 Nov 2023 20:41:30 +0900 Subject: [PATCH 065/179] HIVE-27006: Fix ParallelEdgeFixer (Seonggon Namgung, reviewed by Zoltan Haindrich, Denys Kuzmenko) Closes #4043 --- .../positive/dynamic_semijoin_reduction.q.out | 8 +- .../DynamicPartitionPruningOptimization.java | 84 ++--- .../hive/ql/optimizer/ParallelEdgeFixer.java | 14 +- .../clientpositive/sharedwork_semi_2.q | 46 +++ .../llap/annotate_stats_groupby.q.out | 8 +- .../llap/constant_prop_join_rs.q.out | 8 +- .../llap/dynamic_semijoin_reduction.q.out | 82 ++--- .../llap/dynamic_semijoin_reduction_2.q.out | 24 +- .../llap/dynamic_semijoin_reduction_3.q.out | 16 +- .../llap/dynamic_semijoin_reduction_4.q.out | 24 +- .../dynamic_semijoin_reduction_multicol.q.out | 8 +- .../llap/dynamic_semijoin_reduction_sw.q.out | 18 +- .../llap/dynamic_semijoin_reduction_sw2.q.out | 10 +- .../llap/dynamic_semijoin_user_level.q.out | 24 +- .../clientpositive/llap/mapjoin_hint.q.out | 8 +- .../clientpositive/llap/reopt_semijoin.q.out | 16 +- .../clientpositive/llap/semijoin_hint.q.out | 136 ++++---- .../llap/semijoin_removal_udf.q.out | 8 +- .../clientpositive/llap/sharedwork_semi.q.out | 32 +- .../llap/sharedwork_semi_2.q.out | 326 ++++++++++++++++++ ...ectorized_dynamic_semijoin_reduction.q.out | 48 +-- ...ctorized_dynamic_semijoin_reduction2.q.out | 64 ++-- .../perf/tpcds30tb/tez/query1.q.out | 8 +- .../perf/tpcds30tb/tez/query10.q.out | 8 +- .../perf/tpcds30tb/tez/query16.q.out | 16 +- .../perf/tpcds30tb/tez/query1b.q.out | 16 +- .../perf/tpcds30tb/tez/query2.q.out | 26 +- .../perf/tpcds30tb/tez/query24.q.out | 8 +- .../perf/tpcds30tb/tez/query32.q.out | 8 +- .../perf/tpcds30tb/tez/query37.q.out | 8 +- .../perf/tpcds30tb/tez/query39.q.out | 8 +- .../perf/tpcds30tb/tez/query40.q.out | 10 +- .../perf/tpcds30tb/tez/query54.q.out | 8 +- .../perf/tpcds30tb/tez/query59.q.out | 26 +- .../perf/tpcds30tb/tez/query6.q.out | 8 +- .../perf/tpcds30tb/tez/query64.q.out | 32 +- .../perf/tpcds30tb/tez/query65.q.out | 8 +- .../perf/tpcds30tb/tez/query69.q.out | 24 +- .../perf/tpcds30tb/tez/query71.q.out | 18 +- .../perf/tpcds30tb/tez/query72.q.out | 24 +- .../perf/tpcds30tb/tez/query78.q.out | 16 +- .../perf/tpcds30tb/tez/query80.q.out | 24 +- .../perf/tpcds30tb/tez/query81.q.out | 8 +- .../perf/tpcds30tb/tez/query82.q.out | 8 +- .../perf/tpcds30tb/tez/query92.q.out | 8 +- .../perf/tpcds30tb/tez/query94.q.out | 16 +- .../perf/tpcds30tb/tez/query95.q.out | 18 +- 47 files changed, 876 insertions(+), 498 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/sharedwork_semi_2.q create mode 100644 ql/src/test/results/clientpositive/llap/sharedwork_semi_2.q.out diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out index 2d2da52de0b2..0ace027030fa 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_semijoin_reduction.q.out @@ -123,11 +123,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized Reducer 2 @@ -172,11 +172,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java index 68c5f1f72767..85a420df8883 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java @@ -476,10 +476,10 @@ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext pars keyExprs.add(key); // group by requires "ArrayList", don't ask. - ArrayList outputNames = new ArrayList(); + List outputNames = new ArrayList(); outputNames.add(HiveConf.getColumnInternalName(0)); - ArrayList selectColInfos = new ArrayList(); + List selectColInfos = new ArrayList(); selectColInfos.add(new ColumnInfo(outputNames.get(0), key.getTypeInfo(), "", false)); // project the relevant key column @@ -503,7 +503,7 @@ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext pars HiveConf.getFloatVar(parseContext.getConf(), ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); - ArrayList groupByExprs = new ArrayList(); + List groupByExprs = new ArrayList(); ExprNodeDesc groupByExpr = new ExprNodeColumnDesc(key.getTypeInfo(), outputNames.get(0), null, false); groupByExprs.add(groupByExpr); @@ -513,7 +513,7 @@ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext pars new ArrayList(), false, groupByMemoryUsage, memoryThreshold, minReductionHashAggr, minReductionHashAggrLowerBound, null, false, -1, true); - ArrayList groupbyColInfos = new ArrayList(); + List groupbyColInfos = new ArrayList(); groupbyColInfos.add(new ColumnInfo(outputNames.get(0), key.getTypeInfo(), "", false)); GroupByOperator groupByOp = (GroupByOperator) OperatorFactory.getAndMakeChild( @@ -583,7 +583,7 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex keyExprs.add(key); // group by requires "ArrayList", don't ask. - ArrayList outputNames = new ArrayList(); + List outputNames = new ArrayList(); // project the relevant key column SelectDesc select = new SelectDesc(keyExprs, outputNames); @@ -592,7 +592,7 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex ColumnInfo columnInfo = parentOfRS.getSchema().getColumnInfo(internalColName); columnInfo = new ColumnInfo(columnInfo); outputNames.add(internalColName); - ArrayList signature = new ArrayList(); + List signature = new ArrayList(); signature.add(columnInfo); RowSchema rowSchema = new RowSchema(signature); @@ -627,12 +627,12 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex // Add min/max and bloom filter aggregations List aggFnOIs = new ArrayList(); aggFnOIs.add(key.getWritableObjectInspector()); - ArrayList params = new ArrayList(); + List params = new ArrayList(); params.add( new ExprNodeColumnDesc(key.getTypeInfo(), outputNames.get(0), "", false)); - ArrayList aggs = new ArrayList(); + List aggs = new ArrayList(); try { AggregationDesc min = new AggregationDesc("min", FunctionRegistry.getGenericUDAFEvaluator("min", aggFnOIs, false, false), @@ -666,7 +666,7 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex } // Create the Group by Operator - ArrayList gbOutputNames = new ArrayList(); + List gbOutputNames = new ArrayList(); gbOutputNames.add(SemanticAnalyzer.getColumnInternalName(0)); gbOutputNames.add(SemanticAnalyzer.getColumnInternalName(1)); gbOutputNames.add(SemanticAnalyzer.getColumnInternalName(2)); @@ -675,10 +675,10 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex groupByMemoryUsage, memoryThreshold, minReductionHashAggr, minReductionHashAggrLowerBound, null, false, -1, false); - ArrayList groupbyColInfos = new ArrayList(); + List groupbyColInfos = new ArrayList(); groupbyColInfos.add(new ColumnInfo(gbOutputNames.get(0), key.getTypeInfo(), "", false)); groupbyColInfos.add(new ColumnInfo(gbOutputNames.get(1), key.getTypeInfo(), "", false)); - groupbyColInfos.add(new ColumnInfo(gbOutputNames.get(2), key.getTypeInfo(), "", false)); + groupbyColInfos.add(new ColumnInfo(gbOutputNames.get(2), TypeInfoFactory.binaryTypeInfo, "", false)); GroupByOperator groupByOp = (GroupByOperator)OperatorFactory.getAndMakeChild( groupBy, new RowSchema(groupbyColInfos), selectOp); @@ -686,24 +686,15 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex groupByOp.setColumnExprMap(new HashMap()); // Get the column names of the aggregations for reduce sink - int colPos = 0; - ArrayList rsValueCols = new ArrayList(); + List rsValueCols = new ArrayList(); Map columnExprMap = new HashMap(); - for (int i = 0; i < aggs.size() - 1; i++) { - ExprNodeColumnDesc colExpr = new ExprNodeColumnDesc(key.getTypeInfo(), - gbOutputNames.get(colPos), "", false); + for (int i = 0; i < aggs.size(); i++) { + ExprNodeColumnDesc colExpr = + new ExprNodeColumnDesc(groupbyColInfos.get(i).getType(), gbOutputNames.get(i), "", false); rsValueCols.add(colExpr); - columnExprMap.put(gbOutputNames.get(colPos), colExpr); - colPos++; + columnExprMap.put(gbOutputNames.get(i), colExpr); } - // Bloom Filter uses binary - ExprNodeColumnDesc colExpr = new ExprNodeColumnDesc(TypeInfoFactory.binaryTypeInfo, - gbOutputNames.get(colPos), "", false); - rsValueCols.add(colExpr); - columnExprMap.put(gbOutputNames.get(colPos), colExpr); - colPos++; - // Create the reduce sink operator ReduceSinkDesc rsDesc = PlanUtils.getReduceSinkDesc( new ArrayList(), rsValueCols, gbOutputNames, false, @@ -715,14 +706,14 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex rsOp.getConf().setReducerTraits(EnumSet.of(ReduceSinkDesc.ReducerTraits.QUICKSTART)); // Create the final Group By Operator - ArrayList aggsFinal = new ArrayList(); + List aggsFinal = new ArrayList(); try { List minFinalFnOIs = new ArrayList(); List maxFinalFnOIs = new ArrayList(); List bloomFilterFinalFnOIs = new ArrayList(); - ArrayList minFinalParams = new ArrayList(); - ArrayList maxFinalParams = new ArrayList(); - ArrayList bloomFilterFinalParams = new ArrayList(); + List minFinalParams = new ArrayList(); + List maxFinalParams = new ArrayList(); + List bloomFilterFinalParams = new ArrayList(); // Use the expressions from Reduce Sink. minFinalFnOIs.add(rsValueCols.get(0).getWritableObjectInspector()); maxFinalFnOIs.add(rsValueCols.get(1).getWritableObjectInspector()); @@ -795,32 +786,29 @@ private void createFinalRsForSemiJoinOp( ParseContext parseContext, TableScanOperator ts, GroupByOperator gb, ExprNodeDesc key, String keyBaseAlias, ExprNodeDesc colExpr, boolean isHint) throws SemanticException { - ArrayList gbOutputNames = new ArrayList<>(); - // One each for min, max and bloom filter - gbOutputNames.add(SemanticAnalyzer.getColumnInternalName(0)); - gbOutputNames.add(SemanticAnalyzer.getColumnInternalName(1)); - gbOutputNames.add(SemanticAnalyzer.getColumnInternalName(2)); - - int colPos = 0; - ArrayList rsValueCols = new ArrayList(); - for (int i = 0; i < gbOutputNames.size() - 1; i++) { - ExprNodeColumnDesc expr = new ExprNodeColumnDesc(key.getTypeInfo(), - gbOutputNames.get(colPos++), "", false); - rsValueCols.add(expr); + List gbOutputNames = new ArrayList<>(); + List rsValueCols = new ArrayList(); + Map columnExprMap = new HashMap(); + List rsColInfos = new ArrayList<>(); + for (ColumnInfo gbyColInfo : gb.getSchema().getSignature()) { + String gbyColName = gbyColInfo.getInternalName(); + gbOutputNames.add(gbyColName); + + TypeInfo typInfo = gbyColInfo.getType(); + ExprNodeColumnDesc rsValExpr = new ExprNodeColumnDesc(typInfo, gbyColName, "", false); + rsValueCols.add(rsValExpr); + + String rsOutputColName = Utilities.ReduceField.VALUE + "." + gbyColName; + columnExprMap.put(rsOutputColName, rsValExpr); + rsColInfos.add(new ColumnInfo(rsOutputColName, typInfo, "", false)); } - // Bloom Filter uses binary - ExprNodeColumnDesc colBFExpr = new ExprNodeColumnDesc(TypeInfoFactory.binaryTypeInfo, - gbOutputNames.get(colPos++), "", false); - rsValueCols.add(colBFExpr); - // Create the final Reduce Sink Operator ReduceSinkDesc rsDescFinal = PlanUtils.getReduceSinkDesc( new ArrayList(), rsValueCols, gbOutputNames, false, -1, 0, 1, Operation.NOT_ACID, NullOrdering.defaultNullOrder(parseContext.getConf())); ReduceSinkOperator rsOpFinal = (ReduceSinkOperator)OperatorFactory.getAndMakeChild( - rsDescFinal, new RowSchema(gb.getSchema()), gb); - Map columnExprMap = new HashMap<>(); + rsDescFinal, new RowSchema(rsColInfos), gb); rsOpFinal.setColumnExprMap(columnExprMap); LOG.debug("DynamicSemiJoinPushdown: Saving RS to TS mapping: " + rsOpFinal + ": " + ts); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ParallelEdgeFixer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ParallelEdgeFixer.java index 350c3f0cc41e..76e4aae9fbcd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ParallelEdgeFixer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ParallelEdgeFixer.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hive.ql.optimizer.graph.OperatorGraph; import org.apache.hadoop.hive.ql.optimizer.graph.OperatorGraph.Cluster; import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.RuntimeValuesInfo; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.parse.SemiJoinBranchInfo; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; @@ -155,6 +156,11 @@ private void removeSJEdges() throws SemanticException { rs.getChildOperators().clear(); ts.getParentOperators().remove(rs); rs2sj.put((ReduceSinkOperator) rs, sji); + + if (pctx.getRsToRuntimeValuesInfoMap().containsKey(e.getKey())) { + RuntimeValuesInfo rvi = pctx.getRsToRuntimeValuesInfoMap().remove(e.getKey()); + pctx.getRsToRuntimeValuesInfoMap().put((ReduceSinkOperator) rs, rvi); + } } pctx.setRsToSemiJoinBranchInfo(rs2sj); } @@ -325,6 +331,12 @@ public static Optional> colMappingInverseKeys(ReduceSinkOperator rs) Map ret = new HashMap(); Map exprMap = rs.getColumnExprMap(); Set neededColumns = new HashSet(); + + if (!rs.getSchema().getColumnNames().stream().allMatch(exprMap::containsKey)) { + // Cannot invert RS because exprMap does not contain all of RS's input columns. + return Optional.empty(); + } + try { for (Entry e : exprMap.entrySet()) { String columnName = extractColumnName(e.getValue()); @@ -344,6 +356,6 @@ public static Optional> colMappingInverseKeys(ReduceSinkOperator rs) } catch (SemanticException e) { return Optional.empty(); } - } } + diff --git a/ql/src/test/queries/clientpositive/sharedwork_semi_2.q b/ql/src/test/queries/clientpositive/sharedwork_semi_2.q new file mode 100644 index 000000000000..3ccbecd0a27d --- /dev/null +++ b/ql/src/test/queries/clientpositive/sharedwork_semi_2.q @@ -0,0 +1,46 @@ +set hive.auto.convert.join=true; +set hive.optimize.shared.work.dppunion.merge.eventops=true; +set hive.optimize.shared.work.dppunion=true; +set hive.optimize.shared.work.extended=true; +set hive.optimize.shared.work.parallel.edge.support=true; +set hive.optimize.shared.work=true; +set hive.support.concurrency=true; +set hive.tez.bigtable.minsize.semijoin.reduction=1; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop table if exists x1_store_sales; +drop table if exists x1_date_dim; + +create table x1_store_sales (ss_sold_date_sk int, ss_item_sk int) stored as orc tblproperties('transactional'='true', 'transactional_properties'='default'); +create table x1_date_dim (d_date_sk int, d_month_seq int, d_year int, d_moy int) stored as orc tblproperties('transactional'='true', 'transactional_properties'='default'); + +insert into x1_date_dim values (1,1,2000,1), (2,2,2001,2), (3,2,2001,3), (4,2,2001,4), (5,2,2001,5), (6,2,2001,6), (7,2,2001,7), (8,2,2001,8); +insert into x1_store_sales values (1,1),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9),(10,10),(11,11); + +alter table x1_store_sales update statistics set('numRows'='123456', 'rawDataSize'='1234567'); +alter table x1_date_dim update statistics set('numRows'='28', 'rawDataSize'='81449'); + +explain +select ss_item_sk +from ( + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + ) as tmp, + x1_date_dim +where ss_sold_date_sk = d_date_sk and d_moy=1; + +select ss_item_sk +from ( + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + ) as tmp, + x1_date_dim +where ss_sold_date_sk = d_date_sk and d_moy=1; + + diff --git a/ql/src/test/results/clientpositive/llap/annotate_stats_groupby.q.out b/ql/src/test/results/clientpositive/llap/annotate_stats_groupby.q.out index 6124ef99cfc1..404dd16893c5 100644 --- a/ql/src/test/results/clientpositive/llap/annotate_stats_groupby.q.out +++ b/ql/src/test/results/clientpositive/llap/annotate_stats_groupby.q.out @@ -1712,11 +1712,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1793,11 +1793,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/constant_prop_join_rs.q.out b/ql/src/test/results/clientpositive/llap/constant_prop_join_rs.q.out index c159483d6910..8ea74f5359cd 100644 --- a/ql/src/test/results/clientpositive/llap/constant_prop_join_rs.q.out +++ b/ql/src/test/results/clientpositive/llap/constant_prop_join_rs.q.out @@ -224,11 +224,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 164 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 164 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap Map 8 @@ -308,11 +308,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1187500) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 164 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 164 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out index b66ad9a8b07d..f60d2aeb13e2 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction.q.out @@ -365,11 +365,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -416,11 +416,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -860,11 +860,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -949,16 +949,16 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -1555,11 +1555,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1610,11 +1610,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 3 Execution mode: llap @@ -1659,11 +1659,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=2200) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap @@ -1672,11 +1672,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -1879,13 +1879,13 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator bucketingVersion: 2 null sort order: numBuckets: -1 sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) auto parallelism: false @@ -2032,13 +2032,13 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator bucketingVersion: 2 null sort order: numBuckets: -1 sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) auto parallelism: false @@ -2402,11 +2402,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2432,11 +2432,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -2869,11 +2869,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2905,11 +2905,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2956,11 +2956,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=2200) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 512 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 4 Execution mode: vectorized, llap @@ -2969,11 +2969,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 6 Execution mode: vectorized, llap @@ -3698,11 +3698,11 @@ STAGE PLANS: minReductionHashAggr: 0.9 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -3745,11 +3745,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=10) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 5 Execution mode: vectorized, llap @@ -3905,11 +3905,11 @@ STAGE PLANS: minReductionHashAggr: 0.9 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -3952,11 +3952,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=10) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 5 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out index 2266110ba87c..42e695b3321d 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out @@ -140,11 +140,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 480 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 480 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: decimal(26,12)), _col1 (type: decimal(26,12)), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -213,11 +213,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 480 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 448 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 480 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: decimal(26,12)), _col1 (type: decimal(26,12)), _col2 (type: binary) Reducer 2 Execution mode: llap @@ -321,11 +321,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) Reducer 9 Execution mode: vectorized, llap @@ -334,11 +334,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) Stage: Stage-0 @@ -638,11 +638,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 3 Execution mode: llap @@ -687,11 +687,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=153) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out index 153d584065dc..ad674bd3d93c 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out @@ -88,11 +88,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -302,11 +302,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-6 @@ -462,11 +462,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -552,11 +552,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-2 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_4.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_4.q.out index 21eb5af3b37a..77e957d7cdee 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_4.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_4.q.out @@ -368,11 +368,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -419,11 +419,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -540,11 +540,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -608,11 +608,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -943,11 +943,11 @@ STAGE PLANS: minReductionHashAggr: 0.98 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1032,11 +1032,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=49) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_multicol.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_multicol.q.out index 2c3df77f84b4..c53fb23f10e2 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_multicol.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_multicol.q.out @@ -251,11 +251,11 @@ STAGE PLANS: minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -300,11 +300,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out index ade3c003f8c9..693bdd233491 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw.q.out @@ -311,11 +311,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Select Operator expressions: _col0 (type: string) @@ -326,11 +326,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -428,16 +428,16 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 9 Execution mode: vectorized, llap @@ -446,11 +446,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 736 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 696 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw2.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw2.q.out index 1494cf300c59..f88d9fa96e17 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw2.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_sw2.q.out @@ -274,11 +274,11 @@ STAGE PLANS: minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -380,16 +380,16 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out index c72226012b66..69c915317684 100644 --- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_user_level.q.out @@ -208,11 +208,11 @@ Stage-0 default@srcpart_date_n9,srcpart_date_n9,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] <-Reducer 5 [BROADCAST_EDGE] llap BROADCAST [RS_23] - Group By Operator [GBY_22] (rows=1 width=639) + Group By Operator [GBY_22] (rows=1 width=599) Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, 1, expectedEntries=20)"] <-Map 4 [CUSTOM_SIMPLE_EDGE] llap PARTITION_ONLY_SHUFFLE [RS_21] - Group By Operator [GBY_20] (rows=1 width=639) + Group By Operator [GBY_20] (rows=1 width=599) Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=20)"] Select Operator [SEL_19] (rows=20 width=87) Output:["_col0"] @@ -467,11 +467,11 @@ Stage-0 default@alltypesorc_int_n2,alltypesorc_int_n2,Tbl:COMPLETE,Col:COMPLETE,Output:["cstring"] <-Reducer 6 [BROADCAST_EDGE] llap BROADCAST [RS_50] - Group By Operator [GBY_32] (rows=1 width=639) + Group By Operator [GBY_32] (rows=1 width=599) Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, 1, expectedEntries=20)"] <-Map 5 [CUSTOM_SIMPLE_EDGE] llap PARTITION_ONLY_SHUFFLE [RS_31] - Group By Operator [GBY_30] (rows=1 width=639) + Group By Operator [GBY_30] (rows=1 width=599) Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=20)"] Select Operator [SEL_29] (rows=20 width=87) Output:["_col0"] @@ -854,11 +854,11 @@ Stage-0 default@srcpart_date_n9,srcpart_date_n9,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] <-Reducer 7 [BROADCAST_EDGE] llap BROADCAST [RS_33] - Group By Operator [GBY_32] (rows=1 width=639) + Group By Operator [GBY_32] (rows=1 width=599) Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, 1, expectedEntries=20)"] <-Map 6 [CUSTOM_SIMPLE_EDGE] llap PARTITION_ONLY_SHUFFLE [RS_31] - Group By Operator [GBY_30] (rows=1 width=639) + Group By Operator [GBY_30] (rows=1 width=599) Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=20)"] Select Operator [SEL_29] (rows=20 width=87) Output:["_col0"] @@ -874,11 +874,11 @@ Stage-0 default@alltypesorc_int_n2,alltypesorc_int_n2,Tbl:COMPLETE,Col:COMPLETE,Output:["cstring"] <-Reducer 5 [BROADCAST_EDGE] llap BROADCAST [RS_48] - Group By Operator [GBY_47] (rows=1 width=552) + Group By Operator [GBY_47] (rows=1 width=512) Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, 1, expectedEntries=2200)"] <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap PARTITION_ONLY_SHUFFLE [RS_46] - Group By Operator [GBY_45] (rows=1 width=552) + Group By Operator [GBY_45] (rows=1 width=512) Output:["_col0","_col1","_col2"],aggregations:["min(_col1)","max(_col1)","bloom_filter(_col1, expectedEntries=2200)"] Select Operator [SEL_44] (rows=2200 width=178) Output:["_col1"] @@ -1083,13 +1083,13 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator bucketingVersion: 2 null sort order: numBuckets: -1 sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) auto parallelism: false @@ -1241,13 +1241,13 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator bucketingVersion: 2 null sort order: numBuckets: -1 sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) auto parallelism: false diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out index dbf5490a776d..a51b8f78e2dc 100644 --- a/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out +++ b/ql/src/test/results/clientpositive/llap/mapjoin_hint.q.out @@ -202,11 +202,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -253,11 +253,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out b/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out index 50f683ad5b0c..750e5bab777f 100644 --- a/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out +++ b/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out @@ -201,11 +201,11 @@ STAGE PLANS: minReductionHashAggr: 0.75 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -253,11 +253,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=4) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 @@ -383,11 +383,11 @@ STAGE PLANS: minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -435,11 +435,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out index 229887302a1d..085d3c9ee554 100644 --- a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out +++ b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out @@ -242,11 +242,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -295,11 +295,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=316) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -552,11 +552,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 3 Execution mode: llap @@ -603,11 +603,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -679,11 +679,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -776,11 +776,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=3000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -850,11 +850,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -924,11 +924,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -1124,11 +1124,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1160,11 +1160,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1217,11 +1217,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 2 Execution mode: llap @@ -1266,11 +1266,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 8 Execution mode: llap @@ -1399,11 +1399,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1450,11 +1450,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -1705,11 +1705,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 3 Execution mode: llap @@ -1754,11 +1754,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -1831,11 +1831,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1941,11 +1941,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=3000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -2015,11 +2015,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2087,11 +2087,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -2281,11 +2281,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2313,11 +2313,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2362,11 +2362,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 2 Execution mode: llap @@ -2411,11 +2411,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Reducer 8 Execution mode: llap @@ -2536,11 +2536,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2587,11 +2587,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=316) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 639 Basic stats: PARTIAL Column stats: PARTIAL + Statistics: Num rows: 1 Data size: 599 Basic stats: PARTIAL Column stats: PARTIAL value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -2769,11 +2769,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2871,11 +2871,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -2978,11 +2978,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -3046,11 +3046,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=3000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -3116,11 +3116,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -3184,11 +3184,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=5000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -3413,11 +3413,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -3601,11 +3601,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-6 diff --git a/ql/src/test/results/clientpositive/llap/semijoin_removal_udf.q.out b/ql/src/test/results/clientpositive/llap/semijoin_removal_udf.q.out index fe192603c736..e5e22b53addb 100644 --- a/ql/src/test/results/clientpositive/llap/semijoin_removal_udf.q.out +++ b/ql/src/test/results/clientpositive/llap/semijoin_removal_udf.q.out @@ -164,11 +164,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -201,11 +201,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/sharedwork_semi.q.out b/ql/src/test/results/clientpositive/llap/sharedwork_semi.q.out index 506a22b40256..e0fb405f398c 100644 --- a/ql/src/test/results/clientpositive/llap/sharedwork_semi.q.out +++ b/ql/src/test/results/clientpositive/llap/sharedwork_semi.q.out @@ -255,11 +255,11 @@ STAGE PLANS: minReductionHashAggr: 0.75 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Filter Operator predicate: ((d_moy = 3) and d_date_sk is not null) (type: boolean) @@ -283,11 +283,11 @@ STAGE PLANS: minReductionHashAggr: 0.75 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -298,11 +298,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=4) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 11 Execution mode: vectorized, llap @@ -311,11 +311,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=4) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 2 Execution mode: llap @@ -605,11 +605,11 @@ STAGE PLANS: minReductionHashAggr: 0.75 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Filter Operator predicate: ((d_moy = 3) and d_date_sk is not null) (type: boolean) @@ -633,11 +633,11 @@ STAGE PLANS: minReductionHashAggr: 0.75 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -648,11 +648,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=4) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 2 Execution mode: llap @@ -760,11 +760,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=4) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Union 4 Vertex: Union 4 diff --git a/ql/src/test/results/clientpositive/llap/sharedwork_semi_2.q.out b/ql/src/test/results/clientpositive/llap/sharedwork_semi_2.q.out new file mode 100644 index 000000000000..9607725b9627 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/sharedwork_semi_2.q.out @@ -0,0 +1,326 @@ +PREHOOK: query: drop table if exists x1_store_sales +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists x1_store_sales +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists x1_date_dim +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists x1_date_dim +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create table x1_store_sales (ss_sold_date_sk int, ss_item_sk int) stored as orc tblproperties('transactional'='true', 'transactional_properties'='default') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@x1_store_sales +POSTHOOK: query: create table x1_store_sales (ss_sold_date_sk int, ss_item_sk int) stored as orc tblproperties('transactional'='true', 'transactional_properties'='default') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@x1_store_sales +PREHOOK: query: create table x1_date_dim (d_date_sk int, d_month_seq int, d_year int, d_moy int) stored as orc tblproperties('transactional'='true', 'transactional_properties'='default') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@x1_date_dim +POSTHOOK: query: create table x1_date_dim (d_date_sk int, d_month_seq int, d_year int, d_moy int) stored as orc tblproperties('transactional'='true', 'transactional_properties'='default') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@x1_date_dim +PREHOOK: query: insert into x1_date_dim values (1,1,2000,1), (2,2,2001,2), (3,2,2001,3), (4,2,2001,4), (5,2,2001,5), (6,2,2001,6), (7,2,2001,7), (8,2,2001,8) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@x1_date_dim +POSTHOOK: query: insert into x1_date_dim values (1,1,2000,1), (2,2,2001,2), (3,2,2001,3), (4,2,2001,4), (5,2,2001,5), (6,2,2001,6), (7,2,2001,7), (8,2,2001,8) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@x1_date_dim +POSTHOOK: Lineage: x1_date_dim.d_date_sk SCRIPT [] +POSTHOOK: Lineage: x1_date_dim.d_month_seq SCRIPT [] +POSTHOOK: Lineage: x1_date_dim.d_moy SCRIPT [] +POSTHOOK: Lineage: x1_date_dim.d_year SCRIPT [] +PREHOOK: query: insert into x1_store_sales values (1,1),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9),(10,10),(11,11) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@x1_store_sales +POSTHOOK: query: insert into x1_store_sales values (1,1),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9),(10,10),(11,11) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@x1_store_sales +POSTHOOK: Lineage: x1_store_sales.ss_item_sk SCRIPT [] +POSTHOOK: Lineage: x1_store_sales.ss_sold_date_sk SCRIPT [] +PREHOOK: query: alter table x1_store_sales update statistics set('numRows'='123456', 'rawDataSize'='1234567') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@x1_store_sales +PREHOOK: Output: default@x1_store_sales +POSTHOOK: query: alter table x1_store_sales update statistics set('numRows'='123456', 'rawDataSize'='1234567') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@x1_store_sales +POSTHOOK: Output: default@x1_store_sales +PREHOOK: query: alter table x1_date_dim update statistics set('numRows'='28', 'rawDataSize'='81449') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@x1_date_dim +PREHOOK: Output: default@x1_date_dim +POSTHOOK: query: alter table x1_date_dim update statistics set('numRows'='28', 'rawDataSize'='81449') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@x1_date_dim +POSTHOOK: Output: default@x1_date_dim +PREHOOK: query: explain +select ss_item_sk +from ( + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + ) as tmp, + x1_date_dim +where ss_sold_date_sk = d_date_sk and d_moy=1 +PREHOOK: type: QUERY +PREHOOK: Input: default@x1_date_dim +PREHOOK: Input: default@x1_store_sales +#### A masked pattern was here #### +POSTHOOK: query: explain +select ss_item_sk +from ( + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + ) as tmp, + x1_date_dim +where ss_sold_date_sk = d_date_sk and d_moy=1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@x1_date_dim +POSTHOOK: Input: default@x1_store_sales +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 1 <- Map 5 (BROADCAST_EDGE), Union 2 (CONTAINS) + Map 3 <- Map 5 (BROADCAST_EDGE), Reducer 7 (BROADCAST_EDGE), Union 2 (CONTAINS) + Map 4 <- Map 5 (BROADCAST_EDGE), Reducer 6 (BROADCAST_EDGE), Union 2 (CONTAINS) + Reducer 6 <- Map 5 (CUSTOM_SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: x1_store_sales + filterExpr: ss_sold_date_sk is not null (type: boolean) + probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_39_container, bigKeyColName:ss_sold_date_sk, smallTablePos:1, keyRatio:3.240020736132711E-5 + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ss_sold_date_sk is not null (type: boolean) + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ss_item_sk (type: int), ss_sold_date_sk (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + input vertices: + 1 Map 5 + Statistics: Num rows: 148147 Data size: 592588 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 148147 Data size: 592588 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 3 + Map Operator Tree: + TableScan + alias: x1_store_sales + filterExpr: (ss_sold_date_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_15_x1_date_dim_d_date_sk_min) AND DynamicValue(RS_15_x1_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_15_x1_date_dim_d_date_sk_bloom_filter))) (type: boolean) + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (ss_sold_date_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_15_x1_date_dim_d_date_sk_min) AND DynamicValue(RS_15_x1_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_15_x1_date_dim_d_date_sk_bloom_filter))) (type: boolean) + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ss_item_sk (type: int), ss_sold_date_sk (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + input vertices: + 1 Map 5 + Statistics: Num rows: 148147 Data size: 592588 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 148147 Data size: 592588 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 4 + Map Operator Tree: + TableScan + alias: x1_store_sales + filterExpr: (ss_sold_date_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_15_x1_date_dim_d_date_sk_min) AND DynamicValue(RS_15_x1_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_15_x1_date_dim_d_date_sk_bloom_filter))) (type: boolean) + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (ss_sold_date_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_15_x1_date_dim_d_date_sk_min) AND DynamicValue(RS_15_x1_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_15_x1_date_dim_d_date_sk_bloom_filter))) (type: boolean) + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ss_item_sk (type: int), ss_sold_date_sk (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 123456 Data size: 987648 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + input vertices: + 1 Map 5 + Statistics: Num rows: 148147 Data size: 592588 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 148147 Data size: 592588 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 5 + Map Operator Tree: + TableScan + alias: x1_date_dim + filterExpr: ((d_moy = 1) and d_date_sk is not null) (type: boolean) + Statistics: Num rows: 28 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((d_moy = 1) and d_date_sk is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: d_date_sk (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1000000) + minReductionHashAggr: 0.75 + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: binary) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) + Union 2 + Vertex: Union 2 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ss_item_sk +from ( + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + ) as tmp, + x1_date_dim +where ss_sold_date_sk = d_date_sk and d_moy=1 +PREHOOK: type: QUERY +PREHOOK: Input: default@x1_date_dim +PREHOOK: Input: default@x1_store_sales +#### A masked pattern was here #### +POSTHOOK: query: select ss_item_sk +from ( + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + union all + select ss_item_sk, ss_sold_date_sk from x1_store_sales + ) as tmp, + x1_date_dim +where ss_sold_date_sk = d_date_sk and d_moy=1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@x1_date_dim +POSTHOOK: Input: default@x1_store_sales +#### A masked pattern was here #### +1 +1 +1 diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out index 09d99dca2435..5ec87142259e 100644 --- a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out +++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction.q.out @@ -154,7 +154,7 @@ STAGE PLANS: minReductionHashAggr: 0.98245615 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -162,7 +162,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -250,7 +250,7 @@ STAGE PLANS: projectedOutputColumnNums: [0, 1, 2] mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -258,7 +258,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 @@ -404,7 +404,7 @@ STAGE PLANS: minReductionHashAggr: 0.98245615 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -412,7 +412,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -500,7 +500,7 @@ STAGE PLANS: projectedOutputColumnNums: [0, 1, 2] mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -508,7 +508,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -654,7 +654,7 @@ STAGE PLANS: minReductionHashAggr: 0.98245615 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -662,7 +662,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -750,7 +750,7 @@ STAGE PLANS: projectedOutputColumnNums: [0, 1, 2] mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -758,7 +758,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -905,7 +905,7 @@ STAGE PLANS: minReductionHashAggr: 0.98245615 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -913,7 +913,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reduce Output Operator key expressions: _col0 (type: int) @@ -1031,7 +1031,7 @@ STAGE PLANS: projectedOutputColumnNums: [0, 1, 2] mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -1039,7 +1039,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 @@ -2225,7 +2225,7 @@ STAGE PLANS: minReductionHashAggr: 0.5 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -2233,7 +2233,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2321,7 +2321,7 @@ STAGE PLANS: projectedOutputColumnNums: [0, 1, 2] mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -2329,7 +2329,7 @@ STAGE PLANS: className: VectorReduceSinkEmptyKeyOperator native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 @@ -2441,11 +2441,11 @@ STAGE PLANS: minReductionHashAggr: 0.98245615 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -2492,11 +2492,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=57) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out index 1f3074656945..262c0184faf0 100644 --- a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out +++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out @@ -164,11 +164,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -215,11 +215,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 @@ -375,7 +375,7 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -384,7 +384,7 @@ STAGE PLANS: native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true valueColumns: 0:decimal(10,1), 1:decimal(10,1), 2:binary - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: decimal(10,1)), _col1 (type: decimal(10,1)), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -492,7 +492,7 @@ STAGE PLANS: projectedOutputColumnNums: [0, 1, 2] mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: @@ -501,7 +501,7 @@ STAGE PLANS: native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true valueColumns: 0:decimal(10,1), 1:decimal(10,1), 2:binary - Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: decimal(10,1)), _col1 (type: decimal(10,1)), _col2 (type: binary) Stage: Stage-0 @@ -595,11 +595,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -646,11 +646,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) Stage: Stage-0 @@ -744,11 +744,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -795,11 +795,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 256 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: binary) Stage: Stage-0 @@ -893,11 +893,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -944,11 +944,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) Stage: Stage-0 @@ -1042,11 +1042,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1093,11 +1093,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 512 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) Stage: Stage-0 @@ -1191,11 +1191,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: char(10)), _col1 (type: char(10)), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1242,11 +1242,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: char(10)), _col1 (type: char(10)), _col2 (type: binary) Stage: Stage-0 @@ -1340,11 +1340,11 @@ STAGE PLANS: minReductionHashAggr: 0.95 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: varchar(10)), _col1 (type: varchar(10)), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: all inputs @@ -1391,11 +1391,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=20) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: varchar(10)), _col1 (type: varchar(10)), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1.q.out index db6d6203827a..8ef6ee587a17 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1.q.out @@ -158,11 +158,11 @@ STAGE PLANS: minReductionHashAggr: 0.974359 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -345,11 +345,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query10.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query10.q.out index a93734d2875a..007c1392965a 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query10.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query10.q.out @@ -151,11 +151,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -450,11 +450,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query16.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query16.q.out index 12f93cc0792c..849d3ece302a 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query16.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query16.q.out @@ -78,11 +78,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -239,11 +239,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap @@ -314,11 +314,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1869746) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap @@ -327,11 +327,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1869746) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1b.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1b.q.out index 01f577c9e74c..ceb1f018abe3 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1b.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query1b.q.out @@ -182,11 +182,11 @@ STAGE PLANS: minReductionHashAggr: 0.974359 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -244,11 +244,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap @@ -316,11 +316,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=329849) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 6 Execution mode: vectorized, llap @@ -379,11 +379,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=39) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query2.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query2.q.out index d138e3c66144..60dad2d85c42 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query2.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query2.q.out @@ -11,7 +11,7 @@ STAGE PLANS: Map 11 <- Map 6 (BROADCAST_EDGE), Union 12 (CONTAINS) Map 14 <- Map 6 (BROADCAST_EDGE), Union 12 (CONTAINS) Map 5 <- Map 6 (BROADCAST_EDGE), Union 2 (CONTAINS) - Map 6 <- Reducer 8 (BROADCAST_EDGE), Reducer 9 (BROADCAST_EDGE) + Map 6 <- Reducer 10 (BROADCAST_EDGE), Reducer 8 (BROADCAST_EDGE), Reducer 9 (BROADCAST_EDGE) Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE) Reducer 13 <- Map 7 (BROADCAST_EDGE), Union 12 (SIMPLE_EDGE) Reducer 3 <- Map 7 (BROADCAST_EDGE), Reducer 13 (BROADCAST_EDGE), Union 2 (SIMPLE_EDGE) @@ -322,11 +322,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Filter Operator predicate: ((d_year = 2001) and d_week_seq is not null) (type: boolean) @@ -350,22 +350,24 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) Reducer 10 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: binary) + outputColumnNames: _col0, _col1, _col2 Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 13 Execution mode: vectorized, llap @@ -453,11 +455,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 9 Execution mode: vectorized, llap @@ -466,16 +468,16 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Union 12 Vertex: Union 12 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query24.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query24.q.out index 5d4a5b22d945..dd0ab3975dba 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query24.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query24.q.out @@ -162,11 +162,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Select Operator expressions: i_item_sk (type: bigint), i_current_price (type: decimal(7,2)), i_size (type: char(20)), i_color (type: char(20)), i_units (type: char(10)), i_manager_id (type: int) @@ -272,11 +272,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 2 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query32.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query32.q.out index a712ab95b289..a585554808b8 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query32.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query32.q.out @@ -64,11 +64,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -182,11 +182,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 6 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query37.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query37.q.out index eb5b25c15342..d092f0bdbc50 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query37.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query37.q.out @@ -86,11 +86,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -166,11 +166,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 6 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query39.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query39.q.out index 57f800e3cc3e..6fb410963a91 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query39.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query39.q.out @@ -44,11 +44,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Map Join Operator condition map: @@ -198,11 +198,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query40.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query40.q.out index 5cd26da9a997..4c105dc643fd 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query40.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query40.q.out @@ -88,11 +88,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -259,16 +259,16 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query54.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query54.q.out index 0c013677a995..bc282c8d9b5c 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query54.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query54.q.out @@ -314,11 +314,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator key expressions: _col0 (type: bigint) @@ -607,11 +607,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query59.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query59.q.out index 110793925143..6d0eda9d0ba3 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query59.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query59.q.out @@ -8,7 +8,7 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Map 1 <- Map 2 (BROADCAST_EDGE) - Map 3 <- Reducer 10 (BROADCAST_EDGE), Reducer 11 (BROADCAST_EDGE) + Map 3 <- Reducer 10 (BROADCAST_EDGE), Reducer 11 (BROADCAST_EDGE), Reducer 12 (BROADCAST_EDGE) Map 5 <- Map 3 (BROADCAST_EDGE), Reducer 4 (BROADCAST_EDGE) Reducer 10 <- Map 9 (CUSTOM_SIMPLE_EDGE) Reducer 11 <- Map 9 (CUSTOM_SIMPLE_EDGE) @@ -233,11 +233,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Filter Operator predicate: (d_month_seq BETWEEN 1185 AND 1196 and d_week_seq is not null) (type: boolean) @@ -261,11 +261,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -276,11 +276,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 11 Execution mode: vectorized, llap @@ -289,25 +289,27 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 12 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: binary) + outputColumnNames: _col0, _col1, _col2 Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 4 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query6.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query6.q.out index fb140ba043c7..3aaa8ab05d5d 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query6.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query6.q.out @@ -448,11 +448,11 @@ STAGE PLANS: minReductionHashAggr: 0.96774197 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap @@ -461,11 +461,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query64.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query64.q.out index 81337a284189..2530b3ee01d2 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query64.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query64.q.out @@ -429,11 +429,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Select Operator expressions: i_item_sk (type: bigint) @@ -454,11 +454,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -1038,21 +1038,21 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 31 Execution mode: vectorized, llap @@ -1061,21 +1061,21 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 4 Execution mode: vectorized, llap @@ -1113,11 +1113,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 5 Execution mode: vectorized, llap @@ -1126,11 +1126,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query65.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query65.q.out index 085133790a43..5e56e55ad535 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query65.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query65.q.out @@ -326,11 +326,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap @@ -339,11 +339,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query69.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query69.q.out index c7aa15977a65..7703593c815e 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query69.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query69.q.out @@ -61,11 +61,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -329,11 +329,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap @@ -386,11 +386,11 @@ STAGE PLANS: minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 4 Execution mode: vectorized, llap @@ -469,11 +469,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 8 Execution mode: vectorized, llap @@ -482,11 +482,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=3956347) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 9 Execution mode: vectorized, llap @@ -495,11 +495,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=4369890) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query71.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query71.q.out index 7d1b20280378..3810f5344910 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query71.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query71.q.out @@ -9,7 +9,7 @@ STAGE PLANS: Edges: Map 1 <- Map 10 (BROADCAST_EDGE), Map 13 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE), Union 2 (CONTAINS) Map 5 <- Map 10 (BROADCAST_EDGE), Map 13 (BROADCAST_EDGE), Reducer 11 (BROADCAST_EDGE), Reducer 7 (BROADCAST_EDGE), Union 2 (CONTAINS) - Map 9 <- Map 10 (BROADCAST_EDGE), Map 13 (BROADCAST_EDGE), Reducer 8 (BROADCAST_EDGE), Union 2 (CONTAINS) + Map 9 <- Map 10 (BROADCAST_EDGE), Map 13 (BROADCAST_EDGE), Reducer 12 (BROADCAST_EDGE), Reducer 8 (BROADCAST_EDGE), Union 2 (CONTAINS) Reducer 11 <- Map 10 (CUSTOM_SIMPLE_EDGE) Reducer 12 <- Reducer 11 (CUSTOM_SIMPLE_EDGE) Reducer 3 <- Union 2 (SIMPLE_EDGE) @@ -111,11 +111,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator key expressions: _col0 (type: bigint) @@ -383,25 +383,27 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 12 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Select Operator + expressions: VALUE._col0 (type: bigint), VALUE._col1 (type: bigint), VALUE._col2 (type: binary) + outputColumnNames: _col0, _col1, _col2 Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query72.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query72.q.out index 1259de00d5e8..362a76884717 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query72.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query72.q.out @@ -79,11 +79,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -116,11 +116,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Filter Operator predicate: d_date is not null (type: boolean) @@ -167,11 +167,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -347,11 +347,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) Reducer 13 Execution mode: vectorized, llap @@ -360,11 +360,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 2 Execution mode: vectorized, llap @@ -514,11 +514,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query78.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query78.q.out index 98d928f4ca29..66911ec9a1a2 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query78.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query78.q.out @@ -316,11 +316,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 12 Execution mode: vectorized, llap @@ -329,11 +329,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 15 Execution mode: vectorized, llap @@ -405,11 +405,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 17 Execution mode: vectorized, llap @@ -418,11 +418,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=15782384) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query80.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query80.q.out index f3ce7d7c304a..a0ec8a821da1 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query80.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query80.q.out @@ -146,11 +146,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator key expressions: _col0 (type: bigint) @@ -167,11 +167,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator key expressions: _col0 (type: bigint) @@ -188,11 +188,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -502,11 +502,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 17 Execution mode: vectorized, llap @@ -515,11 +515,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 18 Execution mode: vectorized, llap @@ -528,11 +528,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 2 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query81.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query81.q.out index e2b05bb7a214..817a9878004c 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query81.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query81.q.out @@ -58,11 +58,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -278,11 +278,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1481623) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 4 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query82.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query82.q.out index f2a8dadac004..93313a115625 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query82.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query82.q.out @@ -86,11 +86,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -166,11 +166,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 6 Execution mode: vectorized, llap diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query92.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query92.q.out index 7e1a427778df..abe37997a8ed 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query92.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query92.q.out @@ -105,11 +105,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -241,11 +241,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query94.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query94.q.out index 82e981fbc6ca..4fabcb85b2ed 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query94.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query94.q.out @@ -78,11 +78,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -239,11 +239,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: vectorized, llap @@ -314,11 +314,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap @@ -327,11 +327,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query95.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query95.q.out index bd8f0c2b386f..42521b94f870 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query95.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query95.q.out @@ -83,11 +83,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) @@ -363,11 +363,11 @@ STAGE PLANS: minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 3 Execution mode: llap @@ -435,16 +435,16 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Reducer 7 Execution mode: vectorized, llap @@ -453,11 +453,11 @@ STAGE PLANS: aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, 1, expectedEntries=1000000) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) Stage: Stage-0 From 9546c10a748630ac5cf39d935a90a97446b93be8 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Fri, 24 Nov 2023 10:35:39 +0200 Subject: [PATCH 066/179] HIVE-27794: Iceberg: Implement Copy-On-Write for Merge queries, enable CRUD for V1 tables (Denys Kuzmenko, reviewed by Krisztian Kasa) Closes #4852 --- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 6 +- .../mr/hive/HiveIcebergOutputCommitter.java | 1 + .../iceberg/mr/hive/HiveIcebergSerDe.java | 33 +- .../mr/hive/HiveIcebergStorageHandler.java | 50 +- .../iceberg/mr/hive/IcebergAcidUtil.java | 5 +- .../iceberg/mr/hive/IcebergTableUtil.java | 22 + .../HiveIcebergCopyOnWriteRecordWriter.java | 2 +- .../iceberg/mr/hive/writer/WriterBuilder.java | 40 +- .../org/apache/iceberg/mr/TestHelper.java | 2 + .../HiveIcebergStorageHandlerTestUtils.java | 1 + ...veIcebergStorageHandlerWithEngineBase.java | 32 +- .../hive/TestHiveIcebergBranchOperation.java | 8 + ...cebergV2.java => TestHiveIcebergCRUD.java} | 199 ++-- .../iceberg/mr/hive/TestHiveIcebergCTAS.java | 21 +- .../hive/TestHiveIcebergExpireSnapshots.java | 8 + .../mr/hive/TestHiveIcebergInserts.java | 8 +- .../mr/hive/TestHiveIcebergRollback.java | 7 + .../TestHiveIcebergSetCurrentSnapshot.java | 9 +- .../mr/hive/TestHiveIcebergTagOperation.java | 9 + .../mr/hive/TestHiveIcebergTimeTravel.java | 8 + .../mr/hive/TestHiveIcebergTruncateTable.java | 3 +- .../iceberg/mr/hive/TestOptimisticRetry.java | 26 +- .../queries/negative/merge_split_update_off.q | 2 +- ...table_partition_spec_with_part_evolution.q | 2 +- .../negative/update_split_update_off.q | 2 +- .../merge_iceberg_copy_on_write_partitioned.q | 32 + ...erge_iceberg_copy_on_write_unpartitioned.q | 35 + ...date_iceberg_copy_on_write_unpartitioned.q | 6 + .../negative/merge_split_update_off.q.out | 6 +- ...e_partition_spec_with_part_evolution.q.out | 6 +- .../negative/update_split_update_off.q.out | 6 +- ...te_iceberg_copy_on_write_partitioned.q.out | 396 ++++---- .../llap/vectorized_iceberg_read_mixed.q.out | 224 +++-- .../llap/vectorized_iceberg_read_orc.q.out | 166 ++-- .../vectorized_iceberg_read_parquet.q.out | 128 +-- ...ge_iceberg_copy_on_write_partitioned.q.out | 579 ++++++++++++ ..._iceberg_copy_on_write_unpartitioned.q.out | 848 ++++++++++++++++++ ..._iceberg_copy_on_write_unpartitioned.q.out | 80 ++ .../vectorized_iceberg_read_mixed.q.out | 184 ++-- .../vectorized_iceberg_read_orc.q.out | 138 +-- .../vectorized_iceberg_read_parquet.q.out | 100 ++- .../org/apache/hadoop/hive/ql/Context.java | 11 +- .../AlterMaterializedViewRebuildAnalyzer.java | 4 +- .../apache/hadoop/hive/ql/io/AcidUtils.java | 17 +- .../hive/ql/metadata/HiveStorageHandler.java | 9 +- .../hadoop/hive/ql/parse/CalcitePlanner.java | 2 +- .../hive/ql/parse/MergeSemanticAnalyzer.java | 28 +- .../hive/ql/parse/SemanticAnalyzer.java | 33 +- .../rewrite/CopyOnWriteDeleteRewriter.java | 5 +- .../rewrite/CopyOnWriteMergeRewriter.java | 247 +++++ .../rewrite/CopyOnWriteUpdateRewriter.java | 61 +- .../hive/ql/parse/rewrite/DeleteRewriter.java | 4 +- .../parse/rewrite/DeleteRewriterFactory.java | 6 +- .../hive/ql/parse/rewrite/MergeRewriter.java | 38 +- .../parse/rewrite/MergeRewriterFactory.java | 33 +- .../hive/ql/parse/rewrite/MergeStatement.java | 16 +- .../ql/parse/rewrite/SplitUpdateRewriter.java | 4 +- .../hive/ql/parse/rewrite/UpdateRewriter.java | 7 +- .../parse/rewrite/UpdateRewriterFactory.java | 12 +- .../rewrite/sql/COWWithClauseBuilder.java | 26 +- .../rewrite/sql/MultiInsertSqlGenerator.java | 43 +- .../NonNativeAcidMultiInsertSqlGenerator.java | 24 +- .../rewrite/sql/SqlGeneratorFactory.java | 3 +- 63 files changed, 3055 insertions(+), 1018 deletions(-) rename iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/{TestHiveIcebergV2.java => TestHiveIcebergCRUD.java} (84%) create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_partitioned.q create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out create mode 100644 iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java diff --git a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 5b1f68050c67..0968227fff01 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -483,11 +483,9 @@ public enum ErrorMsg { CBO_IS_REQUIRED(10433, "The following functionality requires CBO (" + HiveConf.ConfVars.HIVE_CBO_ENABLED.varname + "): {0}", true), CTLF_UNSUPPORTED_FORMAT(10434, "CREATE TABLE LIKE FILE is not supported by the ''{0}'' file format", true), - NON_NATIVE_ACID_UPDATE(10435, "Update and Merge into non-native ACID table is only supported when " + - HiveConf.ConfVars.SPLIT_UPDATE.varname + " is true."), + NON_NATIVE_ACID_UPDATE(10435, "Update and Merge to a non-native ACID table in \"merge-on-read\" mode is only supported when \"" + + HiveConf.ConfVars.SPLIT_UPDATE.varname + "\"=\"true\""), READ_ONLY_DATABASE(10436, "Database {0} is read-only", true), - NON_NATIVE_ACID_COW_UPDATE(10437, "Update and Merge into non-native ACID table in copy-on-write mode is only supported when " + - HiveConf.ConfVars.SPLIT_UPDATE.varname + " is false."), //========================== 20000 range starts here ========================// diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index 6816bdf71b00..37941c62761a 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -487,6 +487,7 @@ private void commitWrite(Table table, String branchName, Long snapshotId, long s if (snapshotId != null) { write.validateFromSnapshot(snapshotId); } + write.validateNoConflictingData(); write.commit(); return; } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java index 694e7e9b605e..130b7186221d 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java @@ -41,13 +41,11 @@ import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.hadoop.HadoopFileIO; @@ -160,23 +158,7 @@ public void initialize(@Nullable Configuration configuration, Properties serDePr private static Schema projectedSchema(Configuration configuration, String tableName, Schema tableSchema, Map jobConfs) { Context.Operation operation = HiveCustomStorageHandlerUtils.getWriteOperation(configuration, tableName); - if (operation != null) { - switch (operation) { - case DELETE: - return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); - case UPDATE: - if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( - configuration.get(TableProperties.UPDATE_MODE))) { - return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); - } else { - return IcebergAcidUtil.createSerdeSchemaForUpdate(tableSchema.columns()); - } - case OTHER: - return tableSchema; - default: - throw new IllegalArgumentException("Unsupported operation " + operation); - } - } else { + if (operation == null) { jobConfs.put(InputFormatConfig.CASE_SENSITIVE, "false"); String[] selectedColumns = ColumnProjectionUtils.getReadColumnNames(configuration); // When same table is joined multiple times, it is possible some selected columns are duplicated, @@ -193,6 +175,19 @@ private static Schema projectedSchema(Configuration configuration, String tableN return projectedSchema; } } + if (IcebergTableUtil.isCopyOnWriteMode(operation, configuration::get)) { + return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); + } + switch (operation) { + case DELETE: + return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); + case UPDATE: + return IcebergAcidUtil.createSerdeSchemaForUpdate(tableSchema.columns()); + case OTHER: + return tableSchema; + default: + throw new IllegalArgumentException("Unsupported operation " + operation); + } } @Override diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 604f078f79c5..10a00db534af 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -147,7 +147,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.PartitionsTable; -import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SerializableTable; @@ -672,7 +671,7 @@ public DynamicPartitionCtx createDPContext( throws SemanticException { // delete records are already clustered by partition spec id and the hash of the partition struct // there is no need to do any additional sorting based on partition columns - if (writeOperation == Operation.DELETE) { + if (writeOperation == Operation.DELETE && !shouldOverwrite(hmsTable, writeOperation)) { return null; } @@ -1064,35 +1063,8 @@ public void validateSinkDesc(FileSinkDesc sinkDesc) throws SemanticException { } @Override - public AcidSupportType supportsAcidOperations(org.apache.hadoop.hive.ql.metadata.Table table, - boolean isWriteOperation) { - if (IcebergTableUtil.isV2Table(table.getParameters())) { - if (isWriteOperation) { - checkDMLOperationMode(table); - } - return AcidSupportType.WITHOUT_TRANSACTIONS; - } - - return AcidSupportType.NONE; - } - - // TODO: remove the checks as copy-on-write mode implementation for these DML ops get added - private static void checkDMLOperationMode(org.apache.hadoop.hive.ql.metadata.Table table) { - Map opTypes = ImmutableMap.of( - TableProperties.MERGE_MODE, TableProperties.MERGE_MODE_DEFAULT); - - for (Map.Entry opType : opTypes.entrySet()) { - String mode = table.getParameters().get(opType.getKey()); - RowLevelOperationMode rowLevelOperationMode = RowLevelOperationMode.fromName( - mode != null ? mode : opType.getValue() - ); - if (RowLevelOperationMode.COPY_ON_WRITE.equals(rowLevelOperationMode)) { - throw new UnsupportedOperationException( - String.format("Hive doesn't support copy-on-write mode as %s. Please set '%s'='merge-on-read' on %s " + - "before running ACID operations on it.", opType.getKey(), opType.getKey(), table.getTableName()) - ); - } - } + public AcidSupportType supportsAcidOperations() { + return AcidSupportType.WITHOUT_TRANSACTIONS; } @Override @@ -1623,21 +1595,7 @@ public Map getNativeProperties(org.apache.hadoop.hive.ql.metadat @Override public boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, Context.Operation operation) { - String mode = null; - // As of now only update & delete modes are supported, for all others return false - if (IcebergTableUtil.isV2Table(mTable.getParameters())) { - switch (operation) { - case DELETE: - mode = mTable.getTTable().getParameters().getOrDefault(TableProperties.DELETE_MODE, - TableProperties.DELETE_MODE_DEFAULT); - break; - case UPDATE: - mode = mTable.getTTable().getParameters().getOrDefault(TableProperties.UPDATE_MODE, - TableProperties.UPDATE_MODE_DEFAULT); - break; - } - } - return COPY_ON_WRITE.equalsIgnoreCase(mode); + return IcebergTableUtil.isCopyOnWriteMode(operation, mTable.getParameters()::getOrDefault); } @Override diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergAcidUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergAcidUtil.java index f6dae6430340..43195e38846e 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergAcidUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergAcidUtil.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; +import org.apache.commons.lang3.ObjectUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.ql.io.PositionDeleteInfo; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; @@ -110,14 +111,14 @@ public static Schema createSerdeSchemaForDelete(List dataCols public static PositionDelete getPositionDelete(Record rec, Record rowData) { PositionDelete positionDelete = PositionDelete.create(); String filePath = rec.get(SERDE_META_COLS.get(MetadataColumns.FILE_PATH), String.class); - long filePosition = rec.get(SERDE_META_COLS.get(MetadataColumns.ROW_POSITION), Long.class); + Long filePosition = rec.get(SERDE_META_COLS.get(MetadataColumns.ROW_POSITION), Long.class); int dataOffset = SERDE_META_COLS.size(); // position in the rec where the actual row data begins for (int i = dataOffset; i < rec.size(); ++i) { rowData.set(i - dataOffset, rec.get(i)); } - positionDelete.set(filePath, filePosition, rowData); + positionDelete.set(filePath, ObjectUtils.defaultIfNull(filePosition, 0L), rowData); return positionDelete; } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java index e22c5cfcf362..a47f6fbfdef8 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java @@ -23,10 +23,12 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.function.BinaryOperator; import java.util.function.Function; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.metadata.HiveUtils; @@ -38,6 +40,7 @@ import org.apache.iceberg.ManageSnapshots; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.StructLike; @@ -287,6 +290,25 @@ public static boolean isV2Table(Map props) { "2".equals(props.get(TableProperties.FORMAT_VERSION)); } + public static boolean isCopyOnWriteMode(Context.Operation operation, BinaryOperator props) { + String mode = null; + switch (operation) { + case DELETE: + mode = props.apply(TableProperties.DELETE_MODE, + TableProperties.DELETE_MODE_DEFAULT); + break; + case UPDATE: + mode = props.apply(TableProperties.UPDATE_MODE, + TableProperties.UPDATE_MODE_DEFAULT); + break; + case MERGE: + mode = props.apply(TableProperties.MERGE_MODE, + TableProperties.MERGE_MODE_DEFAULT); + break; + } + return RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase(mode); + } + public static void performMetadataDelete(Table icebergTable, String branchName, SearchArgument sarg) { Expression exp = HiveIcebergFilterFactory.generateFilterExpression(sarg); DeleteFiles deleteFiles = icebergTable.newDelete(); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java index a6681a955232..f13f13ec0465 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java @@ -61,10 +61,10 @@ class HiveIcebergCopyOnWriteRecordWriter extends HiveIcebergWriterBase { public void write(Writable row) throws IOException { Record record = ((Container) row).get(); PositionDelete positionDelete = IcebergAcidUtil.getPositionDelete(record, rowDataTemplate); - int specId = IcebergAcidUtil.parseSpecId(record); Record rowData = positionDelete.row(); if (positionDelete.pos() < 0) { + int specId = IcebergAcidUtil.parseSpecId(record); DataFile dataFile = DataFiles.builder(specs.get(specId)) .withPath(positionDelete.path().toString()) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java index 77f022d17109..2f7177b63810 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java @@ -26,12 +26,12 @@ import org.apache.hadoop.mapred.TaskAttemptID; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.mr.hive.IcebergTableUtil; import org.apache.iceberg.util.PropertyUtil; import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; @@ -122,36 +122,24 @@ public HiveIcebergWriter build() { skipRowData ? null : dataSchema); HiveIcebergWriter writer; - switch (operation) { - case DELETE: - if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( - properties.get(TableProperties.DELETE_MODE))) { - writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, - outputFileFactory, io, targetFileSize); - } else { + if (IcebergTableUtil.isCopyOnWriteMode(operation, properties::getOrDefault)) { + writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, + outputFileFactory, io, targetFileSize); + } else { + switch (operation) { + case DELETE: writer = new HiveIcebergDeleteWriter(dataSchema, specs, writerFactory, deleteOutputFileFactory, io, targetFileSize, skipRowData); - } - break; - case UPDATE: - if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase( - properties.get(TableProperties.UPDATE_MODE))) { - writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, - outputFileFactory, io, targetFileSize); - } else { + break; + case OTHER: + writer = new HiveIcebergRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, + io, targetFileSize); + break; + default: // Update and Merge should be splitted to inserts and deletes throw new IllegalArgumentException("Unsupported operation when creating IcebergRecordWriter: " + operation.name()); - } - break; - case OTHER: - writer = new HiveIcebergRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, - io, targetFileSize); - break; - default: - // Update and Merge should be splitted to inserts and deletes - throw new IllegalArgumentException("Unsupported operation when creating IcebergRecordWriter: " + - operation.name()); + } } WriterRegistry.registerWriter(attemptID, tableName, writer); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestHelper.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestHelper.java index 5614281a7312..6605ceb0fa4c 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestHelper.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestHelper.java @@ -29,6 +29,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SortOrder; @@ -93,6 +94,7 @@ public Map properties() { Map props = Maps.newHashMap(tblProps); props.put(TableProperties.DEFAULT_FILE_FORMAT, fileFormat.name()); props.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); + props.put(TableProperties.DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); return props; } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java index 2010b7cac08d..b106295340c3 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java @@ -87,6 +87,7 @@ static TestHiveShell shell(Map configs) { TestHiveShell shell = new TestHiveShell(); shell.setHiveConfValue("hive.notification.event.poll.interval", "-1"); shell.setHiveConfValue("hive.tez.exec.print.summary", "true"); + shell.setHiveConfValue("tez.counters.max", "1024"); configs.forEach((k, v) -> shell.setHiveConfValue(k, v)); // We would like to make sure that ORC reading overrides this config, so reading Iceberg tables could work in // systems (like Hive 3.2 and higher) where this value is set to true explicitly. diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java index 8d2e94e72f7d..0853124b6025 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; @@ -54,6 +55,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.junit.Assume.assumeTrue; import static org.junit.runners.Parameterized.Parameter; import static org.junit.runners.Parameterized.Parameters; @@ -106,7 +108,7 @@ public abstract class HiveIcebergStorageHandlerWithEngineBase { StatsSetupConst.TOTAL_SIZE, SnapshotSummary.TOTAL_FILE_SIZE_PROP ); - @Parameters(name = "fileFormat={0}, engine={1}, catalog={2}, isVectorized={3}") + @Parameters(name = "fileFormat={0}, engine={1}, catalog={2}, isVectorized={3}, formatVersion={4}") public static Collection parameters() { Collection testParams = Lists.newArrayList(); String javaVersion = System.getProperty("java.specification.version"); @@ -114,14 +116,18 @@ public static Collection parameters() { // Run tests with every FileFormat for a single Catalog (HiveCatalog) for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { for (String engine : EXECUTION_ENGINES) { - // include Tez tests only for Java 8 - if (javaVersion.equals("1.8")) { - testParams.add(new Object[] {fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, false}); - // test for vectorization=ON in case of ORC and PARQUET format with Tez engine - if (fileFormat != FileFormat.METADATA && "tez".equals(engine) && HiveVersion.min(HiveVersion.HIVE_3)) { - testParams.add(new Object[] {fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, true}); + IntStream.of(2, 1).forEach(formatVersion -> { + // include Tez tests only for Java 8 + if (javaVersion.equals("1.8")) { + testParams.add(new Object[]{fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, false, + formatVersion}); + // test for vectorization=ON in case of ORC and PARQUET format with Tez engine + if (fileFormat != FileFormat.METADATA && "tez".equals(engine) && HiveVersion.min(HiveVersion.HIVE_3)) { + testParams.add(new Object[]{fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, true, + formatVersion}); + } } - } + }); } } @@ -129,7 +135,7 @@ public static Collection parameters() { // skip HiveCatalog tests as they are added before for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { - testParams.add(new Object[]{FileFormat.PARQUET, "tez", testTableType, false}); + testParams.add(new Object[]{FileFormat.PARQUET, "tez", testTableType, false, 1}); } } @@ -152,6 +158,9 @@ public static Collection parameters() { @Parameter(3) public boolean isVectorized; + @Parameter(4) + public Integer formatVersion; + @Rule public TemporaryFolder temp = new TemporaryFolder(); @@ -170,6 +179,7 @@ public static void afterClass() throws Exception { @Before public void before() throws IOException { + validateTestParams(); testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); @@ -182,6 +192,10 @@ public void before() throws IOException { } } + protected void validateTestParams() { + assumeTrue(formatVersion == 1); + } + @After public void after() throws Exception { HiveIcebergStorageHandlerTestUtils.close(shell); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java index 24816c9abaa6..d892a72b8221 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java @@ -21,16 +21,24 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.assertj.core.api.Assertions; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import static org.apache.iceberg.mr.hive.HiveIcebergTestUtils.timestampAfterSnapshot; public class TestHiveIcebergBranchOperation extends HiveIcebergStorageHandlerWithEngineBase { + @Override + protected void validateTestParams() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test public void testCreateBranchWithDefaultConfig() throws InterruptedException, IOException { Table table = diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergV2.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java similarity index 84% rename from iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergV2.java rename to iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java index 47c5e8251d7a..acfe94126dc6 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergV2.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java @@ -24,7 +24,6 @@ import java.util.concurrent.Executors; import java.util.stream.StreamSupport; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -53,17 +52,21 @@ import static org.apache.iceberg.types.Types.NestedField.required; /** - * Tests Format V2 specific features, such as reading/writing V2 tables, using delete files, etc. + * Tests Format specific features, such as reading/writing tables, using delete files, etc. */ -public class TestHiveIcebergV2 extends HiveIcebergStorageHandlerWithEngineBase { +public class TestHiveIcebergCRUD extends HiveIcebergStorageHandlerWithEngineBase { + + @Override + protected void validateTestParams() { + } @Test public void testReadAndWriteFormatV2UnpartitionedWithEqDelete() throws IOException { - Assume.assumeFalse("Reading V2 tables with delete files are only supported currently in " + - "non-vectorized mode and only Parquet/Avro", isVectorized || fileFormat == FileFormat.ORC); + Assume.assumeTrue("Reading V2 tables with eq delete files are only supported currently in " + + "non-vectorized mode", !isVectorized && formatVersion == 2); Table tbl = testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, formatVersion); // delete one of the rows List toDelete = TestHelper.RecordsBuilder @@ -82,13 +85,13 @@ public void testReadAndWriteFormatV2UnpartitionedWithEqDelete() throws IOExcepti @Test public void testReadAndWriteFormatV2Partitioned_EqDelete_AllColumnsSupplied() throws IOException { - Assume.assumeFalse("Reading V2 tables with delete files are only supported currently in " + - "non-vectorized mode and only Parquet/Avro", isVectorized || fileFormat == FileFormat.ORC); + Assume.assumeTrue("Reading V2 tables with eq delete files are only supported currently in " + + "non-vectorized mode", !isVectorized && formatVersion == 2); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) .identity("customer_id").build(); Table tbl = testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, formatVersion); // add one more row to the same partition shell.executeStatement("insert into customers values (1, 'Bob', 'Hoover')"); @@ -109,13 +112,13 @@ public void testReadAndWriteFormatV2Partitioned_EqDelete_AllColumnsSupplied() th @Test public void testReadAndWriteFormatV2Partitioned_EqDelete_OnlyEqColumnsSupplied() throws IOException { - Assume.assumeFalse("Reading V2 tables with delete files are only supported currently in " + - "non-vectorized mode and only Parquet/Avro", isVectorized || fileFormat == FileFormat.ORC); + Assume.assumeTrue("Reading V2 tables with eq delete files are only supported currently in " + + "non-vectorized mode", !isVectorized && formatVersion == 2); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) .identity("customer_id").build(); Table tbl = testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, formatVersion); // add one more row to the same partition shell.executeStatement("insert into customers values (1, 'Bob', 'Hoover')"); @@ -137,11 +140,10 @@ public void testReadAndWriteFormatV2Partitioned_EqDelete_OnlyEqColumnsSupplied() @Test public void testReadAndWriteFormatV2Unpartitioned_PosDelete() throws IOException { - Assume.assumeFalse("Reading V2 tables with delete files are only supported currently in " + - "non-vectorized mode and only Parquet/Avro", isVectorized || fileFormat == FileFormat.ORC); + Assume.assumeTrue(formatVersion == 2); Table tbl = testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, formatVersion); // delete one of the rows DataFile dataFile = StreamSupport.stream(tbl.currentSnapshot().addedDataFiles(tbl.io()).spliterator(), false) @@ -164,13 +166,12 @@ public void testReadAndWriteFormatV2Unpartitioned_PosDelete() throws IOException @Test public void testReadAndWriteFormatV2Partitioned_PosDelete_RowNotSupplied() throws IOException { - Assume.assumeFalse("Reading V2 tables with delete files are only supported currently in " + - "non-vectorized mode and only Parquet/Avro", isVectorized || fileFormat == FileFormat.ORC); + Assume.assumeTrue(formatVersion == 2); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) .identity("customer_id").build(); Table tbl = testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, formatVersion); // add some more data to the same partition shell.executeStatement("insert into customers values (0, 'Laura', 'Yellow'), (0, 'John', 'Green'), " + @@ -202,13 +203,12 @@ public void testReadAndWriteFormatV2Partitioned_PosDelete_RowNotSupplied() throw @Test public void testReadAndWriteFormatV2Partitioned_PosDelete_RowSupplied() throws IOException { - Assume.assumeFalse("Reading V2 tables with delete files are only supported currently in " + - "non-vectorized mode and only Parquet/Avro", isVectorized || fileFormat == FileFormat.ORC); + Assume.assumeTrue(formatVersion == 2); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) .identity("customer_id").build(); Table tbl = testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, formatVersion); // add some more data to the same partition shell.executeStatement("insert into customers values (0, 'Laura', 'Yellow'), (0, 'John', 'Green'), " + @@ -244,12 +244,15 @@ public void testReadAndWriteFormatV2Partitioned_PosDelete_RowSupplied() throws I public void testDeleteStatementUnpartitioned() throws TException, InterruptedException { // create and insert an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); // verify delete mode set to merge-on-read - Assert.assertEquals(HiveIcebergStorageHandler.MERGE_ON_READ, - shell.metastore().getTable("default", "customers") - .getParameters().get(TableProperties.DELETE_MODE)); + if (formatVersion == 2) { + Assert.assertEquals(HiveIcebergStorageHandler.MERGE_ON_READ, + shell.metastore().getTable("default", "customers") + .getParameters().get(TableProperties.DELETE_MODE)); + } // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, @@ -278,7 +281,7 @@ public void testDeleteStatementPartitioned() { // create and insert an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, formatVersion); // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, TableIdentifier.of("default", "customers"), false)); @@ -306,9 +309,9 @@ public void testDeleteStatementWithOtherTable() { // create a couple of tables, with an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, formatVersion); testTables.createTable(shell, "other", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, formatVersion); shell.executeStatement("DELETE FROM customers WHERE customer_id in (select t1.customer_id from customers t1 join " + "other t2 on t1.customer_id = t2.customer_id) or " + @@ -334,7 +337,7 @@ public void testDeleteStatementWithPartitionAndSchemaEvolution() { // create and insert an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, formatVersion); // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, TableIdentifier.of("default", "customers"), false)); @@ -373,6 +376,8 @@ public void testDeleteStatementWithPartitionAndSchemaEvolution() { @Test public void testDeleteForSupportedTypes() throws IOException { + Assume.assumeTrue(formatVersion == 2); + for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { Type type = SUPPORTED_TYPES.get(i); @@ -395,7 +400,7 @@ public void testDeleteForSupportedTypes() throws IOException { Schema schema = new Schema(required(1, columnName, type)); List records = TestHelper.generateRandomRecords(schema, 1, 0L); Table table = testTables.createTable(shell, tableName, schema, PartitionSpec.unpartitioned(), fileFormat, records, - 2); + formatVersion); shell.executeStatement("DELETE FROM " + tableName); HiveIcebergTestUtils.validateData(table, ImmutableList.of(), 0); @@ -403,10 +408,19 @@ public void testDeleteForSupportedTypes() throws IOException { } @Test - public void testUpdateStatementUnpartitioned() { + public void testUpdateStatementUnpartitioned() throws TException, InterruptedException { // create and insert an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); + + // verify update mode set to merge-on-read + if (formatVersion == 2) { + Assert.assertEquals(HiveIcebergStorageHandler.MERGE_ON_READ, + shell.metastore().getTable("default", "customers") + .getParameters().get(TableProperties.UPDATE_MODE)); + } + // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, TableIdentifier.of("default", "customers"), false)); @@ -441,7 +455,7 @@ public void testUpdateStatementPartitioned() { // create and insert an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, formatVersion); // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, TableIdentifier.of("default", "customers"), false)); @@ -476,9 +490,9 @@ public void testUpdateStatementWithOtherTable() { // create a couple of tables, with an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, formatVersion); testTables.createTable(shell, "other", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, formatVersion); shell.executeStatement("UPDATE customers SET last_name='Changed' WHERE customer_id in " + "(select t1.customer_id from customers t1 join other t2 on t1.customer_id = t2.customer_id) or " + @@ -509,7 +523,7 @@ public void testUpdateStatementWithPartitionAndSchemaEvolution() { // create and insert an initial batch of records testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, formatVersion); // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, TableIdentifier.of("default", "customers"), false)); @@ -556,6 +570,8 @@ public void testUpdateStatementWithPartitionAndSchemaEvolution() { @Test public void testUpdateForSupportedTypes() throws IOException { + Assume.assumeTrue(formatVersion == 2); + for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { Type type = SUPPORTED_TYPES.get(i); @@ -578,7 +594,7 @@ public void testUpdateForSupportedTypes() throws IOException { Schema schema = new Schema(required(1, columnName, type)); List originalRecords = TestHelper.generateRandomRecords(schema, 1, 0L); Table table = testTables.createTable(shell, tableName, schema, PartitionSpec.unpartitioned(), fileFormat, - originalRecords, 2); + originalRecords, formatVersion); List newRecords = TestHelper.generateRandomRecords(schema, 1, 3L); shell.executeStatement(testTables.getUpdateQuery(tableName, newRecords.get(0))); @@ -586,92 +602,34 @@ public void testUpdateForSupportedTypes() throws IOException { } } - @Test - public void testDeleteStatementFormatV1() { - // create and insert an initial batch of records - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2); - // insert one more batch so that we have multiple data files within the same partition - shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, - TableIdentifier.of("default", "customers"), false)); - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Attempt to do update or delete on table", () -> { - shell.executeStatement("DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'"); - }); - } - - @Test - public void testUpdateStatementFormatV1() { - // create and insert an initial batch of records - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2); - // insert one more batch so that we have multiple data files within the same partition - shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, - TableIdentifier.of("default", "customers"), false)); - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Attempt to do update or delete on table", () -> { - shell.executeStatement("UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'"); - }); - } - - @Test - public void testDMLFailsForCopyOnMergeDeleteMode() { - // No need to check this for each file format - Assume.assumeTrue(fileFormat == FileFormat.ORC && testTableType == TestTables.TestTableType.HIVE_CATALOG); - - // create and insert an initial batch of records - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); - - // simulate copy-on-write setting on the table (i.e. set by Spark or anything else) - shell.executeStatement("ALTER TABLE customers SET TBLPROPERTIES ('write.delete.mode'='copy-on-write')"); - - // attempt a delete - try { - shell.executeStatement("DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'"); - } catch (Throwable e) { - while (e.getCause() != null) { - e = e.getCause(); - } - Assert.assertTrue(e.getMessage().contains("Hive doesn't support copy-on-write mode")); - } - - // attempt an update - try { - shell.executeStatement("UPDATE customers set customer_id=3 where first_name='Joanna'"); - } catch (Throwable e) { - while (e.getCause() != null) { - e = e.getCause(); - } - Assert.assertTrue(e.getMessage().contains("Hive doesn't support copy-on-write mode")); - } - - // Try read queries, they shouldn't fail - shell.executeStatement("select * from customers where first_name='Joanna'"); - shell.executeStatement("select * from customers limit 1"); - shell.executeStatement("select count(*) from customers"); - } - @Test public void testConcurrent2Deletes() { Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && testTableType == TestTables.TestTableType.HIVE_CATALOG); testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); String sql = "DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'"; - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp, executionEngine); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(sql); - shell.closeSession(); - }); + try { + Tasks.range(2) + .executeWith(Executors.newFixedThreadPool(2)) + .run(i -> { + init(shell, testTables, temp, executionEngine); + HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, + RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); + shell.executeStatement(sql); + shell.closeSession(); + }); + } catch (Throwable ex) { + Assert.assertEquals(1, (int) formatVersion); + Throwable cause = Throwables.getRootCause(ex); + Assert.assertTrue(cause instanceof ValidationException); + Assert.assertTrue(cause.getMessage().startsWith("Found conflicting files")); + } List res = shell.executeStatement("SELECT * FROM customers"); Assert.assertEquals(4, res.size()); } @@ -682,7 +640,8 @@ public void testConcurrent2Updates() { testTableType == TestTables.TestTableType.HIVE_CATALOG); testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); String sql = "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'"; try { Tasks.range(2) @@ -708,10 +667,11 @@ public void testConcurrent2Updates() { @Test public void testConcurrentUpdateAndDelete() { Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); String[] sql = new String[]{ "DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'", "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'" @@ -748,7 +708,8 @@ public void testConcurrent2MergeInserts() { testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1); testTables.createTable(shell, "target", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, + formatVersion); String sql = "MERGE INTO target t USING source s on t.customer_id = s.customer_id WHEN Not MATCHED THEN " + "INSERT values (s.customer_id, s.first_name, s.last_name)"; diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java index 8cecbc759ce5..015893233472 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java @@ -50,10 +50,14 @@ */ public class TestHiveIcebergCTAS extends HiveIcebergStorageHandlerWithEngineBase { + @Override + protected void validateTestParams() { + Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG && + isVectorized && formatVersion == 1); + } + @Test public void testCTASFromHiveTable() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); - shell.executeStatement("CREATE TABLE source (id bigint, name string) PARTITIONED BY (dept string) STORED AS ORC"); shell.executeStatement(testTables.getInsertQuery( HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, TableIdentifier.of("default", "source"), false)); @@ -71,8 +75,6 @@ public void testCTASFromHiveTable() { @Test public void testCTASPartitionedFromHiveTable() throws TException, InterruptedException { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); - shell.executeStatement("CREATE TABLE source (id bigint, name string) PARTITIONED BY (dept string) STORED AS ORC"); shell.executeStatement(testTables.getInsertQuery( HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, TableIdentifier.of("default", "source"), false)); @@ -104,8 +106,6 @@ public void testCTASPartitionedFromHiveTable() throws TException, InterruptedExc @Test public void testCTASTblPropsAndLocationClause() throws Exception { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); - shell.executeStatement("CREATE TABLE source (id bigint, name string) PARTITIONED BY (dept string) STORED AS ORC"); shell.executeStatement(testTables.getInsertQuery( HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, TableIdentifier.of("default", "source"), false)); @@ -133,8 +133,6 @@ public void testCTASTblPropsAndLocationClause() throws Exception { @Test public void testCTASPartitionedBySpec() throws TException, InterruptedException { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); - Schema schema = new Schema( optional(1, "id", Types.LongType.get()), optional(2, "year_field", Types.DateType.get()), @@ -187,8 +185,6 @@ public void testCTASPartitionedBySpec() throws TException, InterruptedException @Test public void testCTASFailureRollback() throws IOException { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); - // force an execution error by passing in a committer class that Tez won't be able to load shell.setHiveSessionValue("hive.tez.mapreduce.output.committer.class", "org.apache.NotExistingClass"); @@ -209,8 +205,6 @@ public void testCTASFailureRollback() throws IOException { @Test public void testCTASFollowedByTruncate() throws IOException { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); - testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); @@ -230,7 +224,6 @@ public void testCTASFollowedByTruncate() throws IOException { @Test public void testCTASUnsupportedTypeWithoutAutoConversion() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); Map notSupportedTypes = ImmutableMap.of( "TINYINT", Types.IntegerType.get(), "SMALLINT", Types.IntegerType.get(), @@ -256,7 +249,6 @@ public void testCTASUnsupportedTypeWithoutAutoConversion() { @Test public void testCTASUnsupportedTypeWithAutoConversion() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); Map notSupportedTypes = ImmutableMap.of( "TINYINT", Types.IntegerType.get(), "SMALLINT", Types.IntegerType.get(), @@ -284,7 +276,6 @@ public void testCTASUnsupportedTypeWithAutoConversion() { @Test public void testCTASForAllColumnTypes() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); shell.setHiveSessionValue(InputFormatConfig.SCHEMA_AUTO_CONVERSION, "true"); String sourceCreate = "CREATE EXTERNAL TABLE source (" + "timestamp_col_1 TIMESTAMP, " + diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java index 8fd808f4fed1..0d65c6c73666 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java @@ -23,9 +23,11 @@ import java.text.SimpleDateFormat; import java.util.Date; import org.apache.commons.collections4.IterableUtils; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import static org.apache.iceberg.TableProperties.MAX_SNAPSHOT_AGE_MS; @@ -35,6 +37,12 @@ */ public class TestHiveIcebergExpireSnapshots extends HiveIcebergStorageHandlerWithEngineBase { + @Override + protected void validateTestParams() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test public void testExpireSnapshotsWithTimestamp() throws IOException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "source"); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java index 74cc5c901596..73e0f66d8cf7 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java @@ -69,7 +69,7 @@ public void testSortedInsert() throws IOException { .build(); testTables.createTable(shell, identifier.name(), schema, order, PartitionSpec.unpartitioned(), fileFormat, - ImmutableList.of(), 1, ImmutableMap.of()); + ImmutableList.of(), formatVersion, ImmutableMap.of()); shell.executeStatement(String.format("INSERT INTO TABLE %s VALUES (4, 'a'), (1, 'a'), (3, 'a'), (2, 'a'), " + "(null, 'a'), (3, 'b'), (3, null)", identifier.name())); @@ -91,7 +91,7 @@ public void testSortedAndTransformedInsert() throws IOException { .build(); testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, order, - PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), 1, ImmutableMap.of()); + PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of(), formatVersion, ImmutableMap.of()); StringBuilder insertQuery = new StringBuilder().append(String.format("INSERT INTO %s VALUES ", identifier.name())); HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2.forEach(record -> insertQuery.append("(") @@ -125,8 +125,8 @@ public void testSortedAndTransformedInsertIntoPartitionedTable() throws IOExcept PartitionSpec partitionSpec = PartitionSpec.builderFor(schema) .bucket("b", 2) .build(); - testTables.createTable(shell, identifier.name(), schema, order, partitionSpec, fileFormat, ImmutableList.of(), 1, - ImmutableMap.of()); + testTables.createTable(shell, identifier.name(), schema, order, partitionSpec, fileFormat, ImmutableList.of(), + formatVersion, ImmutableMap.of()); shell.executeStatement(String.format("INSERT INTO %s VALUES (1, 'EUR', 10), (5, 'HUF', 30), (2, 'EUR', 10), " + "(8, 'PLN', 20), (6, 'USD', null)", identifier.name())); List result = shell.executeStatement(String.format("SELECT * FROM %s", identifier.name())); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java index 2bf7d11a680b..4f6fb9699f90 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java @@ -23,6 +23,7 @@ import java.io.IOException; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.junit.Assert; @@ -35,6 +36,12 @@ */ public class TestHiveIcebergRollback extends HiveIcebergStorageHandlerWithEngineBase { + @Override + protected void validateTestParams() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test public void testRollbackToTimestamp() throws IOException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "source"); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java index 0fddae1b83be..35f252443142 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java @@ -34,6 +34,12 @@ */ public class TestHiveIcebergSetCurrentSnapshot extends HiveIcebergStorageHandlerWithEngineBase { + @Override + protected void validateTestParams() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test public void testSetCurrentSnapshot() throws IOException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "source"); @@ -66,9 +72,6 @@ public void testSetCurrentSnapshot() throws IOException, InterruptedException { @Test public void testSetCurrentSnapshotBySnapshotRef() throws IOException, InterruptedException { - // enough to test once - Assume.assumeTrue(fileFormat == FileFormat.ORC && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "source"); Table table = testTables.createTableWithVersions(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java index 3919328fd266..915a9a83641d 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java @@ -22,14 +22,23 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import static org.apache.iceberg.mr.hive.HiveIcebergTestUtils.timestampAfterSnapshot; public class TestHiveIcebergTagOperation extends HiveIcebergStorageHandlerWithEngineBase { + + @Override + protected void validateTestParams() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test public void testCreateTagWithDefaultConfig() throws InterruptedException, IOException { Table table = diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java index 7c64eb68c93f..e4c2490d5ce8 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java @@ -21,10 +21,12 @@ import java.io.IOException; import java.util.List; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.Table; import org.apache.iceberg.types.Types; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import static org.apache.iceberg.mr.hive.HiveIcebergTestUtils.timestampAfterSnapshot; @@ -34,6 +36,12 @@ */ public class TestHiveIcebergTimeTravel extends HiveIcebergStorageHandlerWithEngineBase { + @Override + protected void validateTestParams() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test public void testSelectAsOfTimestamp() throws IOException, InterruptedException { Table table = testTables.createTableWithVersions(shell, "customers", diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java index ae7fe59ec6ee..4f56c812e09b 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java @@ -155,8 +155,7 @@ public void testTruncateTableWithPartitionSpecOnUnpartitionedTable() throws IOEx } @Test - public void testTruncateTableWithPartitionSpecOnPartitionedTable() throws IOException, TException, - InterruptedException { + public void testTruncateTableWithPartitionSpecOnPartitionedTable() { // Create an Iceberg table with some record and try to run a truncate table command with partition // spec. The command should fail as the table is unpartitioned in Hive. Then check if the // initial data and the table statistics are not changed. diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java index 305364085b58..cd4aa88e7807 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java @@ -36,14 +36,17 @@ public class TestOptimisticRetry extends HiveIcebergStorageHandlerWithEngineBase { - @Test - public void testConcurrentOverlappingUpdates() { - + @Override + protected void validateTestParams() { Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + } + @Test + public void testConcurrentOverlappingUpdates() { testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); String sql = "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'"; try { @@ -74,12 +77,9 @@ public void testConcurrentOverlappingUpdates() { @Test public void testNonOverlappingConcurrent2Updates() { - - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion); String[] sql = new String[]{"UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'", "UPDATE customers SET last_name='Changed2' WHERE customer_id=2 and first_name='Jake'"}; @@ -112,13 +112,11 @@ public void testNonOverlappingConcurrent2Updates() { @Test public void testConcurrent2MergeInserts() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); - testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1); testTables.createTable(shell, "target", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, + formatVersion); String sql = "MERGE INTO target t USING source s on t.customer_id = s.customer_id WHEN Not MATCHED THEN " + "INSERT values (s.customer_id, s.first_name, s.last_name)"; diff --git a/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q b/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q index e7dd10fec231..540a15ab4d6e 100644 --- a/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q +++ b/iceberg/iceberg-handler/src/test/queries/negative/merge_split_update_off.q @@ -2,7 +2,7 @@ set hive.split.update=false; drop table if exists test_merge_target; drop table if exists test_merge_source; -create external table test_merge_target (a int, b string, c int) stored by iceberg stored as orc; +create external table test_merge_target (a int, b string, c int) stored by iceberg stored as orc tblproperties ('format-version'='2'); create external table test_merge_source (a int, b string, c int) stored by iceberg stored as orc; explain diff --git a/iceberg/iceberg-handler/src/test/queries/negative/truncate_table_partition_spec_with_part_evolution.q b/iceberg/iceberg-handler/src/test/queries/negative/truncate_table_partition_spec_with_part_evolution.q index 3ed5d3d93086..6864149545c1 100644 --- a/iceberg/iceberg-handler/src/test/queries/negative/truncate_table_partition_spec_with_part_evolution.q +++ b/iceberg/iceberg-handler/src/test/queries/negative/truncate_table_partition_spec_with_part_evolution.q @@ -1,5 +1,5 @@ -- Truncate table on an partitioned Iceberg V1 table with partition evolution must result in an exception since deletes are not possible on Iceberg V1 table. -create external table test_truncate_part_evolution (id int, value string) stored by iceberg stored as orc; +create external table test_truncate_part_evolution (id int, value string) stored by iceberg stored as orc tblproperties ('format-version'='1'); insert into test_truncate_part_evolution values (1, 'one'),(2,'two'),(3,'three'),(4,'four'),(5,'five'); alter table test_truncate_part_evolution set partition spec(id); alter table test_truncate_part_evolution set tblproperties('external.table.purge'='true'); diff --git a/iceberg/iceberg-handler/src/test/queries/negative/update_split_update_off.q b/iceberg/iceberg-handler/src/test/queries/negative/update_split_update_off.q index fa3379c2e715..77aa0f71d653 100644 --- a/iceberg/iceberg-handler/src/test/queries/negative/update_split_update_off.q +++ b/iceberg/iceberg-handler/src/test/queries/negative/update_split_update_off.q @@ -1,7 +1,7 @@ set hive.split.update=false; drop table if exists test_update; -create external table test_update (id int, value string) stored by iceberg stored as orc; +create external table test_update (id int, value string) stored by iceberg stored as orc tblproperties ('format-version'='2'); explain update test_update set value='anything' where id=1; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_partitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_partitioned.q new file mode 100644 index 000000000000..826b730a3b07 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_partitioned.q @@ -0,0 +1,32 @@ +-- SORT_QUERY_RESULTS +set hive.explain.user=false; + +drop table if exists target_ice; +drop table if exists source; + +create external table target_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.merge.mode'='copy-on-write'); +create table source(a int, b string, c int); + +insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56); +insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55); + +-- merge +explain +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10; + +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10; + +select * from target_ice; + +explain +merge into target_ice as t using source src ON t.a = src.a +when not matched then insert values (src.a, src.b, src.c); + +merge into target_ice as t using source src ON t.a = src.a +when not matched then insert values (src.a, src.b, src.c); + +select * from target_ice; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q new file mode 100644 index 000000000000..34ac6ffe9781 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q @@ -0,0 +1,35 @@ +-- SORT_QUERY_RESULTS +set hive.explain.user=false; + +drop table if exists target_ice; +drop table if exists source; + +create external table target_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.merge.mode'='copy-on-write'); +create table source(a int, b string, c int); + +insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56); +insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55); + +-- merge +explain +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +when not matched then insert values (src.a, src.b, src.c); + +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +when not matched then insert values (src.a, src.b, src.c); + +select * from target_ice; + +-- update all +explain +merge into target_ice as t using source src ON t.a = src.a +when matched then update set b = 'Merged', c = t.c - 10; + +merge into target_ice as t using source src ON t.a = src.a +when matched then update set b = 'Merged', c = t.c - 10; + +select * from target_ice; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q index 5bf9304000a3..694184fa93a0 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/update_iceberg_copy_on_write_unpartitioned.q @@ -34,3 +34,9 @@ explain update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice update tbl_ice set b='The last one' where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); select * from tbl_ice order by a, b, c; + +-- update all rows replaced with IOW +explain update tbl_ice set b='All'; + +update tbl_ice set b='All'; +select * from tbl_ice order by a, b, c; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out b/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out index d089db3abaff..e85fe71b7a1d 100644 --- a/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out +++ b/iceberg/iceberg-handler/src/test/results/negative/merge_split_update_off.q.out @@ -10,11 +10,11 @@ PREHOOK: Output: database:default POSTHOOK: query: drop table if exists test_merge_source POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default -PREHOOK: query: create external table test_merge_target (a int, b string, c int) stored by iceberg stored as orc +PREHOOK: query: create external table test_merge_target (a int, b string, c int) stored by iceberg stored as orc tblproperties ('format-version'='2') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@test_merge_target -POSTHOOK: query: create external table test_merge_target (a int, b string, c int) stored by iceberg stored as orc +POSTHOOK: query: create external table test_merge_target (a int, b string, c int) stored by iceberg stored as orc tblproperties ('format-version'='2') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@test_merge_target @@ -26,4 +26,4 @@ POSTHOOK: query: create external table test_merge_source (a int, b string, c int POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@test_merge_source -FAILED: UnsupportedOperationException Hive doesn't support copy-on-write mode as write.merge.mode. Please set 'write.merge.mode'='merge-on-read' on test_merge_target before running ACID operations on it. +FAILED: SemanticException [Error 10435]: Update and Merge to a non-native ACID table in "merge-on-read" mode is only supported when "hive.split.update"="true" diff --git a/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out b/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out index 616297d09986..241854bbea41 100644 --- a/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/negative/truncate_table_partition_spec_with_part_evolution.q.out @@ -1,8 +1,8 @@ -PREHOOK: query: create external table test_truncate_part_evolution (id int, value string) stored by iceberg stored as orc +PREHOOK: query: create external table test_truncate_part_evolution (id int, value string) stored by iceberg stored as orc tblproperties ('format-version'='1') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@test_truncate_part_evolution -POSTHOOK: query: create external table test_truncate_part_evolution (id int, value string) stored by iceberg stored as orc +POSTHOOK: query: create external table test_truncate_part_evolution (id int, value string) stored by iceberg stored as orc tblproperties ('format-version'='1') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@test_truncate_part_evolution @@ -29,4 +29,4 @@ POSTHOOK: query: alter table test_truncate_part_evolution set tblproperties('ext POSTHOOK: type: ALTERTABLE_PROPERTIES POSTHOOK: Input: default@test_truncate_part_evolution POSTHOOK: Output: default@test_truncate_part_evolution -FAILED: UnsupportedOperationException Hive doesn't support copy-on-write mode as write.merge.mode. Please set 'write.merge.mode'='merge-on-read' on test_truncate_part_evolution before running ACID operations on it. +FAILED: SemanticException Truncate conversion to delete is not possible since its not an Iceberg V2 table. Consider converting the table to Iceberg's V2 format specification. diff --git a/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out b/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out index cbacc9179b6c..2e486c83ab3d 100644 --- a/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out +++ b/iceberg/iceberg-handler/src/test/results/negative/update_split_update_off.q.out @@ -4,12 +4,12 @@ PREHOOK: Output: database:default POSTHOOK: query: drop table if exists test_update POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default -PREHOOK: query: create external table test_update (id int, value string) stored by iceberg stored as orc +PREHOOK: query: create external table test_update (id int, value string) stored by iceberg stored as orc tblproperties ('format-version'='2') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@test_update -POSTHOOK: query: create external table test_update (id int, value string) stored by iceberg stored as orc +POSTHOOK: query: create external table test_update (id int, value string) stored by iceberg stored as orc tblproperties ('format-version'='2') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@test_update -FAILED: UnsupportedOperationException Hive doesn't support copy-on-write mode as write.merge.mode. Please set 'write.merge.mode'='merge-on-read' on test_update before running ACID operations on it. +FAILED: SemanticException [Error 10435]: Update and Merge to a non-native ACID table in "merge-on-read" mode is only supported when "hive.split.update"="true" diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out index 91fd18b711f7..6206f479aac6 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out @@ -67,12 +67,12 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product -Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product -Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product -Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product -Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product -Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[226][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[233][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 16' is a cross product PREHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -92,27 +92,28 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Reducer 21 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) - Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) - Reducer 12 <- Reducer 11 (SIMPLE_EDGE) - Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) - Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) - Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) - Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) - Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Union 7 (CONTAINS) - Reducer 18 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 19 <- Map 1 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) - Reducer 20 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 10 <- Reducer 20 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) + Reducer 13 <- Reducer 12 (SIMPLE_EDGE) + Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 19 (XPROD_EDGE) + Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE) + Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) + Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) + Reducer 18 <- Reducer 17 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 19 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE) + Reducer 20 <- Map 1 (SIMPLE_EDGE) Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 22 <- Map 1 (SIMPLE_EDGE) - Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 20 (XPROD_EDGE) - Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) - Reducer 5 <- Reducer 22 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) - Reducer 6 <- Reducer 23 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Union 7 (CONTAINS) - Reducer 8 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) - Reducer 9 <- Reducer 19 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 22 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 23 <- Map 1 (SIMPLE_EDGE) + Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) + Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 23 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 6 <- Reducer 24 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Union 7 (CONTAINS) + Reducer 8 <- Union 7 (SIMPLE_EDGE) + Reducer 9 <- Map 1 (XPROD_EDGE), Reducer 19 (XPROD_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -232,6 +233,21 @@ STAGE PLANS: value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col1, _col2, _col3, _col5 + Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) + Reducer 11 Reduce Operator Tree: Merge Join Operator condition map: @@ -248,7 +264,7 @@ STAGE PLANS: Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) - Reducer 11 + Reducer 12 Reduce Operator Tree: Merge Join Operator condition map: @@ -267,7 +283,7 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 12 + Reducer 13 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -313,7 +329,7 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 13 + Reducer 14 Reduce Operator Tree: Merge Join Operator condition map: @@ -330,7 +346,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - Reducer 14 + Reducer 15 Reduce Operator Tree: Merge Join Operator condition map: @@ -345,7 +361,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) - Reducer 15 + Reducer 16 Reduce Operator Tree: Merge Join Operator condition map: @@ -362,7 +378,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: int) Statistics: Num rows: 6 Data size: 1870 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean), _col9 (type: bigint) - Reducer 16 + Reducer 17 Reduce Operator Tree: Merge Join Operator condition map: @@ -382,7 +398,7 @@ STAGE PLANS: Map-reduce partition columns: _col5 (type: string) Statistics: Num rows: 6 Data size: 1894 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) - Reducer 17 + Reducer 18 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -415,15 +431,14 @@ STAGE PLANS: expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice - Reducer 18 + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 19 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -441,7 +456,22 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 19 + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 20 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -474,22 +504,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 2 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 - Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) - Reducer 20 + Reducer 21 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -502,7 +517,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 21 + Reducer 22 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -520,7 +535,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 22 + Reducer 23 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -553,7 +568,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 23 + Reducer 24 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -647,15 +662,29 @@ STAGE PLANS: expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 8 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 9 Reduce Operator Tree: Merge Join Operator condition map: @@ -672,21 +701,6 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) - Reducer 9 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col1, _col2, _col3, _col5 - Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) Union 7 Vertex: Union 7 @@ -709,12 +723,12 @@ STAGE PLANS: Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product -Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product -Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product -Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product -Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product -Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[226][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[233][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product +Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 16' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -755,7 +769,7 @@ POSTHOOK: query: insert into tbl_ice_other values (10, 'ten'), (333, 'hundred') POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice_other -Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[217][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -777,16 +791,17 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 11 <- Reducer 10 (SIMPLE_EDGE) - Reducer 12 <- Reducer 8 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) - Reducer 4 <- Reducer 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) - Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE) + Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE) + Reducer 11 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE) + Reducer 13 <- Reducer 9 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE) + Reducer 3 <- Reducer 10 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) + Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Union 5 (SIMPLE_EDGE) + Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 8 <- Reducer 7 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -847,7 +862,7 @@ STAGE PLANS: Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string) Execution mode: vectorized - Map 13 + Map 14 Map Operator Tree: TableScan alias: t2 @@ -868,6 +883,19 @@ STAGE PLANS: Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized Reducer 10 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 11 Reduce Operator Tree: Merge Join Operator condition map: @@ -883,7 +911,7 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 11 + Reducer 12 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -929,7 +957,7 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 12 + Reducer 13 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -1005,15 +1033,29 @@ STAGE PLANS: expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 7 Reduce Operator Tree: Merge Join Operator condition map: @@ -1030,7 +1072,7 @@ STAGE PLANS: Map-reduce partition columns: _col5 (type: string) Statistics: Num rows: 2 Data size: 586 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) - Reducer 7 + Reducer 8 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -1063,15 +1105,14 @@ STAGE PLANS: expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice - Reducer 8 + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 9 Reduce Operator Tree: Merge Join Operator condition map: @@ -1116,19 +1157,6 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 9 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Union 5 Vertex: Union 5 @@ -1149,7 +1177,7 @@ STAGE PLANS: Stats Work Basic Stats Work: -Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[217][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -1193,7 +1221,7 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_standard_other POSTHOOK: Lineage: tbl_standard_other.a SCRIPT [] POSTHOOK: Lineage: tbl_standard_other.b SCRIPT [] -Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[217][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_ice_other t2 on t1.a = t2.a) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -1215,16 +1243,17 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Reducer 8 (SIMPLE_EDGE) - Reducer 11 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 12 <- Reducer 11 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) - Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) - Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE) + Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE) + Reducer 11 <- Reducer 9 (SIMPLE_EDGE) + Reducer 12 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 13 <- Reducer 12 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE) + Reducer 3 <- Reducer 10 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) + Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Union 5 (SIMPLE_EDGE) + Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 8 <- Reducer 7 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -1285,7 +1314,7 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string) Execution mode: vectorized - Map 13 + Map 14 Map Operator Tree: TableScan alias: t2 @@ -1306,6 +1335,19 @@ STAGE PLANS: Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized Reducer 10 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 11 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -1324,7 +1366,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 11 + Reducer 12 Reduce Operator Tree: Merge Join Operator condition map: @@ -1340,7 +1382,7 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 12 + Reducer 13 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -1443,15 +1485,29 @@ STAGE PLANS: expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 7 Reduce Operator Tree: Merge Join Operator condition map: @@ -1468,7 +1524,7 @@ STAGE PLANS: Map-reduce partition columns: _col5 (type: string) Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) - Reducer 7 + Reducer 8 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -1501,15 +1557,14 @@ STAGE PLANS: expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice - Reducer 8 + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 9 Reduce Operator Tree: Merge Join Operator condition map: @@ -1574,19 +1629,6 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Reducer 9 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) - mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Union 5 Vertex: Union 5 @@ -1607,7 +1649,7 @@ STAGE PLANS: Stats Work Basic Stats Work: -Warning: Shuffle Join MERGEJOIN[178][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[180][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product PREHOOK: query: delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out index 4660af717bfb..c25bdd0c554f 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_mixed.q.out @@ -48,21 +48,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -162,20 +166,24 @@ STAGE PLANS: TableScan alias: tbl_ice_mixed Statistics: Num rows: 19 Data size: 1748 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(a) - keys: b (type: string) - minReductionHashAggr: 0.4736842 - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: a (type: int), b (type: string) + outputColumnNames: a, b + Statistics: Num rows: 19 Data size: 1748 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(a) + keys: b (type: string) + minReductionHashAggr: 0.4736842 + mode: hash + outputColumnNames: _col0, _col1 Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -227,21 +235,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -378,21 +390,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 10:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64, 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 14:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -491,21 +507,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2), 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -585,20 +605,24 @@ STAGE PLANS: TableScan alias: tbl_ice_mixed_all_types Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(t_float) - keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) - minReductionHashAggr: 0.99 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Select Operator + expressions: t_float (type: float), t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + outputColumnNames: t_float, t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) - null sort order: zzzzzzzzz - sort order: +++++++++ - Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Group By Operator + aggregations: max(t_float) + keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col9 (type: float) + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + null sort order: zzzzzzzzz + sort order: +++++++++ + Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col9 (type: float) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -687,20 +711,24 @@ STAGE PLANS: TableScan alias: tbl_ice_mixed_all_types Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(t_float) - keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) - minReductionHashAggr: 0.99 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Select Operator + expressions: t_float (type: float), t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + outputColumnNames: t_float, t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) - null sort order: zzzzzzzzz - sort order: +++++++++ - Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Group By Operator + aggregations: max(t_float) + keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col9 (type: float) + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + null sort order: zzzzzzzzz + sort order: +++++++++ + Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Statistics: Num rows: 2 Data size: 746 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col9 (type: float) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -883,7 +911,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out index 2e8c70a64bf0..8ed0bb79949c 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_orc.q.out @@ -57,20 +57,24 @@ STAGE PLANS: TableScan alias: tbl_ice_orc Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(a) - keys: b (type: string) - minReductionHashAggr: 0.5 - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: a (type: int), b (type: string) + outputColumnNames: a, b + Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(a) + keys: b (type: string) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -135,21 +139,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -265,20 +273,24 @@ STAGE PLANS: TableScan alias: tbl_ice_orc_all_types Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(t_float) - keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) - minReductionHashAggr: 0.99 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Select Operator + expressions: t_float (type: float), t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + outputColumnNames: t_float, t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) - null sort order: zzzzzzzzz - sort order: +++++++++ - Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Group By Operator + aggregations: max(t_float) + keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col9 (type: float) + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + null sort order: zzzzzzzzz + sort order: +++++++++ + Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col9 (type: float) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -347,21 +359,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 10:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64, 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 14:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -504,7 +520,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:p1:string, 2:b:string, 3:p2:string] + vectorizationSchemaColumns: [0:a:int, 1:p1:string, 2:b:string, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true @@ -634,7 +650,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true @@ -747,21 +763,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] - Group By Vectorization: - aggregators: VectorUDAFMinString(col 1:string) -> string - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 0:string, col 2:int, col 3:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:string, 1:int, 2:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 3:string + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3] + Group By Vectorization: + aggregators: VectorUDAFMinString(col 1:string) -> string + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 0:string, col 2:int, col 3:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:string, 1:int, 2:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 3:string Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -917,7 +937,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:arrayofprimitives:array, 2:arrayofarrays:array>, 3:arrayofmaps:array>, 4:arrayofstructs:array>, 5:mapofprimitives:map, 6:mapofarrays:map>, 7:mapofmaps:map>, 8:mapofstructs:map>, 9:structofprimitives:struct, 10:structofarrays:struct,birthdays:array>, 11:structofmaps:struct,map2:map>] + vectorizationSchemaColumns: [0:a:int, 1:arrayofprimitives:array, 2:arrayofarrays:array>, 3:arrayofmaps:array>, 4:arrayofstructs:array>, 5:mapofprimitives:map, 6:mapofarrays:map>, 7:mapofmaps:map>, 8:mapofstructs:map>, 9:structofprimitives:struct, 10:structofarrays:struct,birthdays:array>, 11:structofmaps:struct,map2:map>, 12:PARTITION__SPEC__ID:int, 13:PARTITION__HASH:bigint, 14:FILE__PATH:string, 15:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out index 76204ac92a2d..0fbbade47c8f 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/vectorized_iceberg_read_parquet.q.out @@ -57,20 +57,24 @@ STAGE PLANS: TableScan alias: tbl_ice_parquet Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(a) - keys: b (type: string) - minReductionHashAggr: 0.5 - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: a (type: int), b (type: string) + outputColumnNames: a, b + Statistics: Num rows: 10 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: max(a) + keys: b (type: string) + minReductionHashAggr: 0.5 + mode: hash + outputColumnNames: _col0, _col1 Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int) + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -122,21 +126,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -265,20 +273,24 @@ STAGE PLANS: TableScan alias: tbl_ice_parquet_all_types Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: max(t_float) - keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) - minReductionHashAggr: 0.99 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 + Select Operator + expressions: t_float (type: float), t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + outputColumnNames: t_float, t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) - null sort order: zzzzzzzzz - sort order: +++++++++ - Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Group By Operator + aggregations: max(t_float) + keys: t_double (type: double), t_boolean (type: boolean), t_int (type: int), t_bigint (type: bigint), t_binary (type: binary), t_string (type: string), t_timestamp (type: timestamp), t_date (type: date), t_decimal (type: decimal(4,2)) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col9 (type: float) + Reduce Output Operator + key expressions: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + null sort order: zzzzzzzzz + sort order: +++++++++ + Map-reduce partition columns: _col0 (type: double), _col1 (type: boolean), _col2 (type: int), _col3 (type: bigint), _col4 (type: binary), _col5 (type: string), _col6 (type: timestamp), _col7 (type: date), _col8 (type: decimal(4,2)) + Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col9 (type: float) Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Reducer 2 @@ -336,21 +348,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2), 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized, llap LLAP IO: all inputs (cache only) Map Vectorization: @@ -537,7 +553,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out new file mode 100644 index 000000000000..1227db6101d0 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out @@ -0,0 +1,579 @@ +PREHOOK: query: drop table if exists target_ice +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists target_ice +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists source +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists source +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table target_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.merge.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@target_ice +POSTHOOK: query: create external table target_ice(a int, b string, c int) partitioned by spec (bucket(16, a), truncate(3, b)) stored by iceberg tblproperties ('format-version'='2', 'write.merge.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@target_ice +PREHOOK: query: create table source(a int, b string, c int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@source +POSTHOOK: query: create table source(a int, b string, c int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@source +PREHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@target_ice +POSTHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@target_ice +PREHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@source +POSTHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@source +POSTHOOK: Lineage: source.a SCRIPT [] +POSTHOOK: Lineage: source.b SCRIPT [] +POSTHOOK: Lineage: source.c SCRIPT [] +PREHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 10 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Union 3 (SIMPLE_EDGE) + Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE) + Reducer 7 <- Reducer 10 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + filterExpr: ((a <= 100) or a is not null) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a <= 100) (type: boolean) + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Map 11 + Map Operator Tree: + TableScan + alias: target_ice + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col3 (type: int) + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col4 (type: string), _col5 (type: int) + Filter Operator + predicate: (a <= 100) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int) + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: FILE__PATH (type: string), a (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Execution mode: vectorized + Reducer 10 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col4 IS DISTINCT FROM _col7) (type: boolean) + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 8 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col4 (type: int) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: string), _col4 (type: bigint), _col5 (type: int), 'Merged' (type: string), (_col6 + 10) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 302 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 4 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 7 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col3 (type: int) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: int) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col3 ASC NULLS FIRST + partition by: _col3 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: string), -1L (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 598 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Union 3 + Vertex: Union 3 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +PREHOOK: query: select * from target_ice +PREHOOK: type: QUERY +PREHOOK: Input: default@target_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from target_ice +POSTHOOK: type: QUERY +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Merged 60 +2 Merged 61 +333 two 56 +PREHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when not matched then insert values (src.a, src.b, src.c) +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when not matched then insert values (src.a, src.b, src.c) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: target_ice + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 3 Data size: 624 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 3 Data size: 624 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + Execution mode: vectorized + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Full Outer Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 12 Data size: 1216 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col4 is null (type: boolean) + Statistics: Num rows: 9 Data size: 912 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int), _col6 (type: string), _col7 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 3 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + File Output Operator + compressed: false + Dp Sort State: PARTITION_SORTED + Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when not matched then insert values (src.a, src.b, src.c) +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when not matched then insert values (src.a, src.b, src.c) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +PREHOOK: query: select * from target_ice +PREHOOK: type: QUERY +PREHOOK: Input: default@target_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from target_ice +POSTHOOK: type: QUERY +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Merged 60 +111 one 55 +2 Merged 61 +3 three 52 +333 two 56 +4 four 53 +5 five 54 diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out new file mode 100644 index 000000000000..6728fbf39bf7 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out @@ -0,0 +1,848 @@ +PREHOOK: query: drop table if exists target_ice +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists target_ice +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists source +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists source +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create external table target_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.merge.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@target_ice +POSTHOOK: query: create external table target_ice(a int, b string, c int) stored by iceberg tblproperties ('format-version'='2', 'write.merge.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@target_ice +PREHOOK: query: create table source(a int, b string, c int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@source +POSTHOOK: query: create table source(a int, b string, c int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@source +PREHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@target_ice +POSTHOOK: query: insert into target_ice values (1, 'one', 50), (2, 'two', 51), (111, 'one', 55), (333, 'two', 56) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@target_ice +PREHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@source +POSTHOOK: query: insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), (4, 'four', 53), (5, 'five', 54), (111, 'one', 55) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@source +POSTHOOK: Lineage: source.a SCRIPT [] +POSTHOOK: Lineage: source.b SCRIPT [] +POSTHOOK: Lineage: source.c SCRIPT [] +PREHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +when not matched then insert values (src.a, src.b, src.c) +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +when not matched then insert values (src.a, src.b, src.c) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 7 <- Reducer 6 (SIMPLE_EDGE) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + filterExpr: ((a <= 100) or a is not null) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a <= 100) (type: boolean) + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Map 10 + Map Operator Tree: + TableScan + alias: target_ice + filterExpr: (a <= 100) (type: boolean) + Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (a <= 100) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int) + Execution mode: vectorized + Map 11 + Map Operator Tree: + TableScan + alias: target_ice + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 4 Data size: 832 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 4 Data size: 832 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: FILE__PATH (type: string), a (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Filter Operator + predicate: (FILE__PATH is not null and a is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col3 (type: int) + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col4 (type: string), _col5 (type: int) + Execution mode: vectorized + Map 13 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int) + Execution mode: vectorized + Reducer 12 + Reduce Operator Tree: + Merge Join Operator + condition map: + Full Outer Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 14 Data size: 1520 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col4 is null (type: boolean) + Statistics: Num rows: 10 Data size: 1216 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int), _col6 (type: string), _col7 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 10 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 17 Data size: 3296 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col4 (type: int) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: string), _col4 (type: bigint), _col5 (type: int), 'Merged' (type: string), (_col6 + 10) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 302 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 17 Data size: 3296 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 4 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col4 IS DISTINCT FROM _col7) (type: boolean) + Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 8 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 17 Data size: 3296 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col1 (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 7 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1 + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col1: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col1 ASC NULLS FIRST + partition by: _col1 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col3 (type: int) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: int) + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col3 ASC NULLS FIRST + partition by: _col3 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 2 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: string), -1L (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 598 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 17 Data size: 3296 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Union 3 + Vertex: Union 3 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +when not matched then insert values (src.a, src.b, src.c) +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when matched and t.a > 100 THEN DELETE +when matched then update set b = 'Merged', c = t.c + 10 +when not matched then insert values (src.a, src.b, src.c) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +PREHOOK: query: select * from target_ice +PREHOOK: type: QUERY +PREHOOK: Input: default@target_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from target_ice +POSTHOOK: type: QUERY +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Merged 60 +2 Merged 61 +3 three 52 +333 two 56 +4 four 53 +5 five 54 +PREHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when matched then update set b = 'Merged', c = t.c - 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: explain +merge into target_ice as t using source src ON t.a = src.a +when matched then update set b = 'Merged', c = t.c - 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE) + Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 7 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: target_ice + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: FILE__PATH is not null (type: boolean) + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1794 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 6 Data size: 1794 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int) + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1746 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col3 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col3 (type: int) + Statistics: Num rows: 6 Data size: 1746 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col4 (type: string), _col5 (type: int) + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1272 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 6 Data size: 1272 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int) + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: FILE__PATH (type: string), a (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 6 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Execution mode: vectorized + Map 10 + Map Operator Tree: + TableScan + alias: src + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: a is not null (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 12 Data size: 3616 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col4 IS DISTINCT FROM _col7) (type: boolean) + Statistics: Num rows: 12 Data size: 3616 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 12 Data size: 3588 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 12 Data size: 3588 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1794 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 4503 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 5 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1746 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 6 Data size: 1746 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: int), _col4 (type: string), _col5 (type: int) + Reducer 6 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1746 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 6 Data size: 1746 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 3 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), -1L (type: bigint), _col3 (type: int), _col4 (type: string), _col5 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 3 Data size: 897 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 4503 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 7 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col4 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 6 Data size: 1272 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), 'Merged' (type: string), (_col5 - 10) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1812 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 15 Data size: 4503 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + Reducer 8 + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 9 + Execution mode: vectorized + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col0: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col0 ASC NULLS FIRST + partition by: _col0 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Union 4 + Vertex: Union 4 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when matched then update set b = 'Merged', c = t.c - 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@source +PREHOOK: Input: default@target_ice +PREHOOK: Output: default@target_ice +POSTHOOK: query: merge into target_ice as t using source src ON t.a = src.a +when matched then update set b = 'Merged', c = t.c - 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@source +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: default@target_ice +PREHOOK: query: select * from target_ice +PREHOOK: type: QUERY +PREHOOK: Input: default@target_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from target_ice +POSTHOOK: type: QUERY +POSTHOOK: Input: default@target_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 Merged 50 +2 Merged 51 +3 Merged 42 +333 two 56 +4 Merged 43 +5 Merged 44 diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out index da2c5e0292d1..98c24ce144c9 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out @@ -2130,3 +2130,83 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 333 Changed forever 56 444 The last one 800 555 Changed again 801 +PREHOOK: query: explain update tbl_ice set b='All' +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: explain update tbl_ice set b='All' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: tbl_ice + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), 'All' (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 9 Data size: 855 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 855 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Execution mode: vectorized + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + + Stage: Stage-3 + Stats Work + Basic Stats Work: + +PREHOOK: query: update tbl_ice set b='All' +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: default@tbl_ice +POSTHOOK: query: update tbl_ice set b='All' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: default@tbl_ice +PREHOOK: query: select * from tbl_ice order by a, b, c +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice order by a, b, c +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 All 50 +2 All 51 +3 All 52 +4 All 53 +5 All 54 +111 All 55 +333 All 56 +444 All 800 +555 All 801 diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out index 71c7149c6c79..038bd8085352 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_mixed.q.out @@ -48,21 +48,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized Map Vectorization: enabled: true @@ -154,16 +158,18 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_10] - Group By Operator [GBY_9] (rows=10 width=92) + File Output Operator [FS_11] + Group By Operator [GBY_10] (rows=10 width=92) Output:["_col0","_col1"],aggregations:["max(VALUE._col0)"],keys:KEY._col0 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0 - Group By Operator [GBY_7] (rows=10 width=92) + Group By Operator [GBY_8] (rows=10 width=92) Output:["_col0","_col1"],aggregations:["max(a)"],keys:b - TableScan [TS_0] (rows=19 width=92) - default@tbl_ice_mixed,tbl_ice_mixed,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] + Select Operator [SEL_7] (rows=19 width=92) + Output:["a","b"] + TableScan [TS_0] (rows=19 width=92) + default@tbl_ice_mixed,tbl_ice_mixed,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] PREHOOK: query: explain vectorization only detail select b, max(a) from tbl_ice_mixed group by b PREHOOK: type: QUERY @@ -191,21 +197,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized Map Vectorization: enabled: true @@ -341,21 +351,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 10:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64, 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 14:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized Map Vectorization: enabled: true @@ -453,21 +467,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2), 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized Map Vectorization: enabled: true @@ -539,18 +557,20 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_11] - Select Operator [SEL_10] (rows=2 width=373) + File Output Operator [FS_12] + Select Operator [SEL_11] (rows=2 width=373) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] - Group By Operator [GBY_9] (rows=2 width=373) + Group By Operator [GBY_10] (rows=2 width=373) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Group By Operator [GBY_7] (rows=2 width=373) + Group By Operator [GBY_8] (rows=2 width=373) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(t_float)"],keys:t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal - TableScan [TS_0] (rows=2 width=373) - default@tbl_ice_mixed_all_types,tbl_ice_mixed_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + Select Operator [SEL_7] (rows=2 width=373) + Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + TableScan [TS_0] (rows=2 width=373) + default@tbl_ice_mixed_all_types,tbl_ice_mixed_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] PREHOOK: query: select max(t_float), t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal from tbl_ice_mixed_all_types group by t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal @@ -604,18 +624,20 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_11] - Select Operator [SEL_10] (rows=2 width=373) + File Output Operator [FS_12] + Select Operator [SEL_11] (rows=2 width=373) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] - Group By Operator [GBY_9] (rows=2 width=373) + Group By Operator [GBY_10] (rows=2 width=373) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Group By Operator [GBY_7] (rows=2 width=373) + Group By Operator [GBY_8] (rows=2 width=373) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(t_float)"],keys:t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal - TableScan [TS_0] (rows=2 width=373) - default@tbl_ice_mixed_all_types,tbl_ice_mixed_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + Select Operator [SEL_7] (rows=2 width=373) + Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + TableScan [TS_0] (rows=2 width=373) + default@tbl_ice_mixed_all_types,tbl_ice_mixed_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] PREHOOK: query: select max(t_float), t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal from tbl_ice_mixed_all_types group by t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal @@ -770,7 +792,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out index ba851000d4c1..3474ed9945ce 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_orc.q.out @@ -50,16 +50,18 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_10] - Group By Operator [GBY_9] (rows=5 width=92) + File Output Operator [FS_11] + Group By Operator [GBY_10] (rows=5 width=92) Output:["_col0","_col1"],aggregations:["max(VALUE._col0)"],keys:KEY._col0 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0 - Group By Operator [GBY_7] (rows=5 width=92) + Group By Operator [GBY_8] (rows=5 width=92) Output:["_col0","_col1"],aggregations:["max(a)"],keys:b - TableScan [TS_0] (rows=10 width=92) - default@tbl_ice_orc,tbl_ice_orc,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] + Select Operator [SEL_7] (rows=10 width=92) + Output:["a","b"] + TableScan [TS_0] (rows=10 width=92) + default@tbl_ice_orc,tbl_ice_orc,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] PREHOOK: query: select b, max(a) from tbl_ice_orc group by b PREHOOK: type: QUERY @@ -100,21 +102,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized Map Vectorization: enabled: true @@ -222,18 +228,20 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_11] - Select Operator [SEL_10] (rows=1 width=372) + File Output Operator [FS_12] + Select Operator [SEL_11] (rows=1 width=372) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] - Group By Operator [GBY_9] (rows=1 width=372) + Group By Operator [GBY_10] (rows=1 width=372) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Group By Operator [GBY_7] (rows=1 width=372) + Group By Operator [GBY_8] (rows=1 width=372) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(t_float)"],keys:t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal - TableScan [TS_0] (rows=1 width=372) - default@tbl_ice_orc_all_types,tbl_ice_orc_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + Select Operator [SEL_7] (rows=1 width=372) + Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + TableScan [TS_0] (rows=1 width=372) + default@tbl_ice_orc_all_types,tbl_ice_orc_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] PREHOOK: query: select max(t_float), t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal from tbl_ice_orc_all_types group by t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal @@ -274,21 +282,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 10:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)/DECIMAL_64, 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, ConvertDecimal64ToDecimal(col 9:decimal(4,2)/DECIMAL_64) -> 14:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized Map Vectorization: enabled: true @@ -430,7 +442,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:p1:string, 2:b:string, 3:p2:string] + vectorizationSchemaColumns: [0:a:int, 1:p1:string, 2:b:string, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true @@ -559,7 +571,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true @@ -671,21 +683,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] - Group By Vectorization: - aggregators: VectorUDAFMinString(col 1:string) -> string - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 0:string, col 2:int, col 3:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:string, 1:int, 2:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 3:string + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3] + Group By Vectorization: + aggregators: VectorUDAFMinString(col 1:string) -> string + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 0:string, col 2:int, col 3:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:string, 1:int, 2:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 3:string Execution mode: vectorized Map Vectorization: enabled: true @@ -840,7 +856,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:arrayofprimitives:array, 2:arrayofarrays:array>, 3:arrayofmaps:array>, 4:arrayofstructs:array>, 5:mapofprimitives:map, 6:mapofarrays:map>, 7:mapofmaps:map>, 8:mapofstructs:map>, 9:structofprimitives:struct, 10:structofarrays:struct,birthdays:array>, 11:structofmaps:struct,map2:map>] + vectorizationSchemaColumns: [0:a:int, 1:arrayofprimitives:array, 2:arrayofarrays:array>, 3:arrayofmaps:array>, 4:arrayofstructs:array>, 5:mapofprimitives:map, 6:mapofarrays:map>, 7:mapofmaps:map>, 8:mapofstructs:map>, 9:structofprimitives:struct, 10:structofarrays:struct,birthdays:array>, 11:structofmaps:struct,map2:map>, 12:PARTITION__SPEC__ID:int, 13:PARTITION__HASH:bigint, 14:FILE__PATH:string, 15:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out index 4cf978f18fef..aad6b082c974 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_read_parquet.q.out @@ -50,16 +50,18 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_10] - Group By Operator [GBY_9] (rows=5 width=92) + File Output Operator [FS_11] + Group By Operator [GBY_10] (rows=5 width=92) Output:["_col0","_col1"],aggregations:["max(VALUE._col0)"],keys:KEY._col0 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0 - Group By Operator [GBY_7] (rows=5 width=92) + Group By Operator [GBY_8] (rows=5 width=92) Output:["_col0","_col1"],aggregations:["max(a)"],keys:b - TableScan [TS_0] (rows=10 width=92) - default@tbl_ice_parquet,tbl_ice_parquet,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] + Select Operator [SEL_7] (rows=10 width=92) + Output:["a","b"] + TableScan [TS_0] (rows=10 width=92) + default@tbl_ice_parquet,tbl_ice_parquet,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] PREHOOK: query: explain vectorization only detail select b, max(a) from tbl_ice_parquet group by b PREHOOK: type: QUERY @@ -87,21 +89,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:a:int, 1:b:string] - Group By Vectorization: - aggregators: VectorUDAFMaxLong(col 0:int) -> int - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:string - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkStringOperator - keyColumns: 0:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 1:int + vectorizationSchemaColumns: [0:a:int, 1:b:string, 2:PARTITION__SPEC__ID:int, 3:PARTITION__HASH:bigint, 4:FILE__PATH:string, 5:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Group By Vectorization: + aggregators: VectorUDAFMaxLong(col 0:int) -> int + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:string + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkStringOperator + keyColumns: 0:string + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 1:int Execution mode: vectorized Map Vectorization: enabled: true @@ -222,18 +228,20 @@ Stage-0 limit:-1 Stage-1 Reducer 2 vectorized - File Output Operator [FS_11] - Select Operator [SEL_10] (rows=1 width=372) + File Output Operator [FS_12] + Select Operator [SEL_11] (rows=1 width=372) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] - Group By Operator [GBY_9] (rows=1 width=372) + Group By Operator [GBY_10] (rows=1 width=372) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_8] + SHUFFLE [RS_9] PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Group By Operator [GBY_7] (rows=1 width=372) + Group By Operator [GBY_8] (rows=1 width=372) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["max(t_float)"],keys:t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal - TableScan [TS_0] (rows=1 width=372) - default@tbl_ice_parquet_all_types,tbl_ice_parquet_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + Select Operator [SEL_7] (rows=1 width=372) + Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] + TableScan [TS_0] (rows=1 width=372) + default@tbl_ice_parquet_all_types,tbl_ice_parquet_all_types,Tbl:COMPLETE,Col:COMPLETE,Output:["t_float","t_double","t_boolean","t_int","t_bigint","t_binary","t_string","t_timestamp","t_date","t_decimal"] PREHOOK: query: explain vectorization only detail select max(t_float), t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal from tbl_ice_parquet_all_types group by t_double, t_boolean, t_int, t_bigint, t_binary, t_string, t_timestamp, t_date, t_decimal @@ -263,21 +271,25 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2)] - Group By Vectorization: - aggregators: VectorUDAFMaxDouble(col 0:float) -> float - className: VectorGroupByOperator - groupByMode: HASH - keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) - native: false - vectorProcessingMode: HASH - projectedOutputColumnNums: [0] - Reduce Sink Vectorization: - className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 9:float + vectorizationSchemaColumns: [0:t_float:float, 1:t_double:double, 2:t_boolean:boolean, 3:t_int:int, 4:t_bigint:bigint, 5:t_binary:binary, 6:t_string:string, 7:t_timestamp:timestamp, 8:t_date:date, 9:t_decimal:decimal(4,2), 10:PARTITION__SPEC__ID:int, 11:PARTITION__HASH:bigint, 12:FILE__PATH:string, 13:ROW__POSITION:bigint] + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] + Group By Vectorization: + aggregators: VectorUDAFMaxDouble(col 0:float) -> float + className: VectorGroupByOperator + groupByMode: HASH + keyExpressions: col 1:double, col 2:boolean, col 3:int, col 4:bigint, col 5:binary, col 6:string, col 7:timestamp, col 8:date, col 9:decimal(4,2) + native: false + vectorProcessingMode: HASH + projectedOutputColumnNums: [0] + Reduce Sink Vectorization: + className: VectorReduceSinkMultiKeyOperator + keyColumns: 0:double, 1:boolean, 2:int, 3:bigint, 4:binary, 5:string, 6:timestamp, 7:date, 8:decimal(4,2) + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 9:float Execution mode: vectorized Map Vectorization: enabled: true @@ -463,7 +475,7 @@ STAGE PLANS: Map Operator Tree: TableScan Vectorization: native: true - vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string] + vectorizationSchemaColumns: [0:p1:string, 1:b:string, 2:a:int, 3:p2:string, 4:PARTITION__SPEC__ID:int, 5:PARTITION__HASH:bigint, 6:FILE__PATH:string, 7:ROW__POSITION:bigint] Select Vectorization: className: VectorSelectOperator native: true diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java index 7e8dc5d3b7ab..17e4d4ae724b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -240,7 +240,7 @@ public void setLocation(Path location) { */ public enum Operation {UPDATE, DELETE, MERGE, IOW, OTHER} public enum DestClausePrefix { - INSERT("insclause-"), UPDATE("updclause-"), DELETE("delclause-"); + INSERT("insclause-"), UPDATE("updclause-"), DELETE("delclause-"), MERGE("mergeclause-"); private final String prefix; DestClausePrefix(String prefix) { this.prefix = prefix; @@ -338,11 +338,12 @@ private DestClausePrefix getMergeDestClausePrefix(ASTNode curNode) { ASTNode query = (ASTNode) insert.getParent(); assert query != null && query.getType() == HiveParser.TOK_QUERY; - for(int childIdx = 1; childIdx < query.getChildCount(); childIdx++) {//1st child is TOK_FROM + int tokFromIdx = query.getFirstChildWithType(HiveParser.TOK_FROM).getChildIndex(); + for (int childIdx = tokFromIdx + 1; childIdx < query.getChildCount(); childIdx++) { assert query.getChild(childIdx).getType() == HiveParser.TOK_INSERT; - if(insert == query.getChild(childIdx)) { - DestClausePrefix prefix = insertBranchToNamePrefix.get(childIdx); - if(prefix == null) { + if (insert == query.getChild(childIdx)) { + DestClausePrefix prefix = insertBranchToNamePrefix.get(childIdx - tokFromIdx); + if (prefix == null) { throw new IllegalStateException("Found a node w/o branch mapping: '" + getMatchedText(insert) + "'"); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java index c7ab47cc637b..c9020b7cbdd8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java @@ -301,7 +301,7 @@ private RelNode applyRecordIncrementalRebuildPlan( // If we succeed, we modify the plan and afterwards the AST. // MV should be an acid table. boolean acidView = AcidUtils.isFullAcidTable(mvTable.getTTable()) - || AcidUtils.isNonNativeAcidTable(mvTable, true); + || AcidUtils.isNonNativeAcidTable(mvTable); MaterializedViewRewritingRelVisitor visitor = new MaterializedViewRewritingRelVisitor(acidView); visitor.go(basePlan); if (visitor.isRewritingAllowed()) { @@ -491,7 +491,7 @@ protected ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, CalcitePlan private MaterializedViewASTBuilder getMaterializedViewASTBuilder() { if (AcidUtils.isFullAcidTable(mvTable.getTTable())) { return new NativeAcidMaterializedViewASTBuilder(); - } else if (AcidUtils.isNonNativeAcidTable(mvTable, true)) { + } else if (AcidUtils.isNonNativeAcidTable(mvTable)) { return new NonNativeAcidMaterializedViewASTBuilder(mvTable); } else { throw new UnsupportedOperationException("Incremental rebuild is supported only for fully ACID materialized " + diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java index 0a367c8f8377..febe8a2b1e90 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java @@ -3378,18 +3378,9 @@ public static void tryInvalidateDirCache(org.apache.hadoop.hive.metastore.api.Ta } } - public static boolean isNonNativeAcidTable(Table table, boolean isWriteOperation) { + public static boolean isNonNativeAcidTable(Table table) { return table != null && table.getStorageHandler() != null && - table.getStorageHandler().supportsAcidOperations(table, isWriteOperation) != HiveStorageHandler.AcidSupportType.NONE; - } - - public static boolean isCopyOnWriteMode(Table table, Context.Operation operation) { - boolean copyOnWriteMode = false; - HiveStorageHandler storageHandler = table.getStorageHandler(); - if (storageHandler != null) { - copyOnWriteMode = storageHandler.shouldOverwrite(table, operation); - } - return copyOnWriteMode; + table.getStorageHandler().supportsAcidOperations() != HiveStorageHandler.AcidSupportType.NONE; } /** @@ -3402,7 +3393,7 @@ public static List getAcidVirtualColumns(Table table) { if (isTransactionalTable(table)) { return Lists.newArrayList(VirtualColumn.ROWID); } else { - if (isNonNativeAcidTable(table, false)) { + if (isNonNativeAcidTable(table)) { return table.getStorageHandler().acidVirtualColumns(); } } @@ -3411,7 +3402,7 @@ public static List getAcidVirtualColumns(Table table) { public static boolean acidTableWithoutTransactions(Table table) { return table != null && table.getStorageHandler() != null && - table.getStorageHandler().supportsAcidOperations(table, true) == + table.getStorageHandler().supportsAcidOperations() == HiveStorageHandler.AcidSupportType.WITHOUT_TRANSACTIONS; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index e2ef165101c6..7d646e8b7ab5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -425,8 +425,7 @@ enum AcidSupportType { * * @return the table's ACID support type */ - default AcidSupportType supportsAcidOperations(org.apache.hadoop.hive.ql.metadata.Table table, - boolean isWriteOperation) { + default AcidSupportType supportsAcidOperations() { return AcidSupportType.NONE; } @@ -435,7 +434,7 @@ default AcidSupportType supportsAcidOperations(org.apache.hadoop.hive.ql.metadat * for tables that support ACID operations. * * Should only return a non-empty list if - * {@link HiveStorageHandler#supportsAcidOperations(org.apache.hadoop.hive.ql.metadata.Table, boolean)} ()} returns something + * {@link HiveStorageHandler#supportsAcidOperations()} returns something * other NONE. * * @return the list of ACID virtual columns @@ -455,7 +454,7 @@ default List acidVirtualColumns() { * This method specifies which columns should be injected into the <selectCols> part of the rewritten query. * * Should only return a non-empty list if - * {@link HiveStorageHandler#supportsAcidOperations(org.apache.hadoop.hive.ql.metadata.Table, boolean)} returns something + * {@link HiveStorageHandler#supportsAcidOperations()} returns something * other NONE. * * @param table the table which is being deleted/updated/merged into @@ -479,7 +478,7 @@ default FieldSchema getRowId() { * This method specifies which columns should be injected into the <sortCols> part of the rewritten query. * * Should only return a non-empty list if - * {@link HiveStorageHandler#supportsAcidOperations(org.apache.hadoop.hive.ql.metadata.Table, boolean)} returns something + * {@link HiveStorageHandler#supportsAcidOperations()} returns something * other NONE. * * @param table the table which is being deleted/updated/merged into diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java index 850f42e5259b..b105de8174ee 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java @@ -2960,7 +2960,7 @@ private RelNode genTableLogicalPlan(String tableAlias, QB qb) throws SemanticExc List virtualCols = new ArrayList<>(); if (tableType == TableType.NATIVE) { virtualCols = VirtualColumn.getRegistry(conf); - if (AcidUtils.isNonNativeAcidTable(tabMetaData, false)) { + if (AcidUtils.isNonNativeAcidTable(tabMetaData)) { virtualCols.addAll(tabMetaData.getStorageHandler().acidVirtualColumns()); } if (tabMetaData.isNonNative() && tabMetaData.getStorageHandler().areSnapshotsSupported() && diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java index 163dda00d2e5..94fb812a3001 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.ql.parse.rewrite.MergeStatement; import org.apache.hadoop.hive.ql.parse.rewrite.RewriterFactory; - import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -120,8 +119,11 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) String sourceName = getSimpleTableName(source); ASTNode onClause = (ASTNode) tree.getChild(2); String onClauseAsText = getMatchedText(onClause); + MergeStatement.MergeStatementBuilder mergeStatementBuilder = MergeStatement .withTarget(targetTable, getFullTableNameForSQL(targetNameNode), targetAlias) + .sourceName(sourceName) + .sourceAlias(getSourceAlias(source, sourceName)) .onClauseAsText(onClauseAsText); int whenClauseBegins = 3; @@ -134,8 +136,6 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) } List whenClauses = findWhenClauses(tree, whenClauseBegins); - mergeStatementBuilder.sourceAlias(getSourceAlias(source, sourceName)); - // Add the hint if any if (hasHint) { mergeStatementBuilder.hintStr(String.format(" /*+ %s */ ", qHint.getText())); @@ -155,8 +155,14 @@ WHEN NOT MATCHED THEN INSERT VALUES (source.a2, source.b2) switch (getWhenClauseOperation(whenClause).getType()) { case HiveParser.TOK_INSERT: numInsertClauses++; + + OnClauseAnalyzer oca = new OnClauseAnalyzer(onClause, targetTable, targetAlias, + conf, onClauseAsText); + oca.analyze(); + mergeStatementBuilder.addWhenClause( - handleInsert(whenClause, onClause, targetTable, targetAlias, onClauseAsText)); + handleInsert(whenClause, oca.getPredicate(), targetTable)) + .onClausePredicate(oca.getPredicate()); break; case HiveParser.TOK_UPDATE: numWhenMatchedUpdateClauses++; @@ -321,12 +327,11 @@ private String getWhenClausePredicate(ASTNode whenClause) { /** * Generates the Insert leg of the multi-insert SQL to represent WHEN NOT MATCHED THEN INSERT clause. - * @param targetTableNameInSourceQuery - simple name/alias * @throws SemanticException */ - private MergeStatement.InsertClause handleInsert(ASTNode whenNotMatchedClause, ASTNode onClause, - Table targetTable, String targetTableNameInSourceQuery, - String onClauseAsString) throws SemanticException { + private MergeStatement.InsertClause handleInsert(ASTNode whenNotMatchedClause, String onClausePredicate, + Table targetTable) throws SemanticException { + ASTNode whenClauseOperation = getWhenClauseOperation(whenNotMatchedClause); assert whenNotMatchedClause.getType() == HiveParser.TOK_NOT_MATCHED; assert whenClauseOperation.getType() == HiveParser.TOK_INSERT; @@ -345,11 +350,6 @@ private MergeStatement.InsertClause handleInsert(ASTNode whenNotMatchedClause, A throw new SemanticException(String.format("Column schema must have the same length as values (%d vs %d)", columnListNode.getChildCount(), valuesNode.getChildCount() - 1)); } - - OnClauseAnalyzer oca = new OnClauseAnalyzer(onClause, targetTable, targetTableNameInSourceQuery, - conf, onClauseAsString); - oca.analyze(); - UnparseTranslator defaultValuesTranslator = new UnparseTranslator(conf); defaultValuesTranslator.enable(); List targetSchema = processTableColumnNames(columnListNode, targetTable.getFullyQualifiedName()); @@ -360,7 +360,7 @@ private MergeStatement.InsertClause handleInsert(ASTNode whenNotMatchedClause, A String extraPredicate = getWhenClausePredicate(whenNotMatchedClause); return new MergeStatement.InsertClause( - getMatchedText(columnListNode), valuesClause, oca.getPredicate(), extraPredicate); + getMatchedText(columnListNode), valuesClause, onClausePredicate, extraPredicate); } private void collectDefaultValues( diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index ba1fd1b49b2e..9fa27638eb0e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -2508,9 +2508,9 @@ private void getMetaData(QB qb, ReadEntity parentInput) HiveUtils.getTableSnapshotRef(ts.tableHandle.getSnapshotRef())); } // Disallow update and delete on non-acid tables - final boolean isWriteOperation = updating(name) || deleting(name); + boolean isWriteOperation = updating(name) || deleting(name); boolean isFullAcid = AcidUtils.isFullAcidTable(ts.tableHandle) || - AcidUtils.isNonNativeAcidTable(ts.tableHandle, isWriteOperation); + AcidUtils.isNonNativeAcidTable(ts.tableHandle); if (isWriteOperation && !isFullAcid) { if (!AcidUtils.isInsertOnlyTable(ts.tableHandle)) { // Whether we are using an acid compliant transaction manager has already been caught in @@ -7080,7 +7080,7 @@ private Operator genBucketingSortingDest(String dest, Operator input, QB qb, } } else { // Non-native acid tables should handle their own bucketing for updates/deletes - if ((updating(dest) || deleting(dest)) && !AcidUtils.isNonNativeAcidTable(dest_tab, true)) { + if ((updating(dest) || deleting(dest)) && !AcidUtils.isNonNativeAcidTable(dest_tab)) { partnCols = getPartitionColsFromBucketColsForUpdateDelete(input, true); sortCols = getPartitionColsFromBucketColsForUpdateDelete(input, false); createSortOrderForUpdateDelete(sortCols, order, nullOrder); @@ -8085,9 +8085,9 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) inputRR = opParseCtx.get(input).getRowResolver(); List vecCol = new ArrayList(); - - if (updating(dest) || deleting(dest)) { - if (AcidUtils.isNonNativeAcidTable(destinationTable, true)) { + + if (updating(dest) || deleting(dest) || merging(dest)) { + if (AcidUtils.isNonNativeAcidTable(destinationTable)) { destinationTable.getStorageHandler().acidVirtualColumns().stream() .map(col -> new ColumnInfo(col.getName(), col.getTypeInfo(), "", true)) .forEach(vecCol::add); @@ -8879,9 +8879,9 @@ private Operator genConversionSelectOperator(String dest, QB qb, Operator input, if (dynPart && dpCtx != null && !alreadyContainsPartCols) { outColumnCnt += dpCtx.getNumDPCols(); } - + // The numbers of input columns and output columns should match for regular query - if (!updating(dest) && !deleting(dest) && inColumnCnt != outColumnCnt) { + if (!updating(dest) && !deleting(dest) && !merging(dest) && inColumnCnt != outColumnCnt) { String reason = "Table " + dest + " has " + outColumnCnt + " columns, but query has " + inColumnCnt + " columns."; throw new SemanticException(ASTErrorUtils.getMsg( @@ -8899,7 +8899,7 @@ private Operator genConversionSelectOperator(String dest, QB qb, Operator input, if (!(deserializer instanceof MetadataTypedColumnsetSerDe) && !deleting(dest)) { // If we're updating, add the required virtual columns. - int virtualColumnSize = updating(dest) ? AcidUtils.getAcidVirtualColumns(table).size() : 0; + int virtualColumnSize = updating(dest) || merging(dest) ? AcidUtils.getAcidVirtualColumns(table).size() : 0; for (int i = 0; i < virtualColumnSize; i++) { expressions.add(new ExprNodeColumnDesc(rowFields.get(i).getType(), rowFields.get(i).getInternalName(), "", true)); @@ -8914,7 +8914,7 @@ private Operator genConversionSelectOperator(String dest, QB qb, Operator input, // For Non-Native ACID tables we should convert the new values as well rowFieldsOffset = expressions.size(); - if (updating(dest) && AcidUtils.isNonNativeAcidTable(table, true) + if (updating(dest) && AcidUtils.isNonNativeAcidTable(table) && rowFields.size() >= rowFieldsOffset + columnNumber) { for (int i = 0; i < columnNumber; i++) { ExprNodeDesc column = handleConversion(tableFields.get(i), rowFields.get(rowFieldsOffset-columnNumber + i), converted, dest, i); @@ -11943,7 +11943,7 @@ private Operator genTablePlan(String alias, QB qb) throws SemanticException { if (!tab.isNonNative()) { vcList.addAll(VirtualColumn.getRegistry(conf)); } - if (tab.isNonNative() && AcidUtils.isNonNativeAcidTable(tab, false)) { + if (tab.isNonNative() && AcidUtils.isNonNativeAcidTable(tab)) { vcList.addAll(tab.getStorageHandler().acidVirtualColumns()); } @@ -15543,14 +15543,13 @@ private boolean isAcidOutputFormat(Class of) { // figure out if a table is Acid or not. private AcidUtils.Operation getAcidType(String destination) { return deleting(destination) ? AcidUtils.Operation.DELETE : - (updating(destination) ? AcidUtils.Operation.UPDATE : - AcidUtils.Operation.INSERT); + updating(destination) ? AcidUtils.Operation.UPDATE : AcidUtils.Operation.INSERT; } private Context.Operation getWriteOperation(String destination) { return deleting(destination) ? Context.Operation.DELETE : - (updating(destination) ? Context.Operation.UPDATE : - Context.Operation.OTHER); + updating(destination) ? Context.Operation.UPDATE : + merging(destination) ? Context.Operation.MERGE : Context.Operation.OTHER; } private AcidUtils.Operation getAcidType(Class of, String dest, @@ -15577,6 +15576,10 @@ protected boolean updating(String destination) { private boolean deleting(String destination) { return destination.startsWith(Context.DestClausePrefix.DELETE.toString()); } + + private boolean merging(String destination) { + return destination.startsWith(Context.DestClausePrefix.MERGE.toString()); + } // Make sure the proper transaction manager that supports ACID is being used private void checkAcidTxnManager(Table table) throws SemanticException { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java index f5fbeaf0449b..5c4366b5ba40 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java @@ -34,11 +34,10 @@ public class CopyOnWriteDeleteRewriter implements Rewriter { protected final SqlGeneratorFactory sqlGeneratorFactory; private final COWWithClauseBuilder cowWithClauseBuilder; - public CopyOnWriteDeleteRewriter( - HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, COWWithClauseBuilder cowWithClauseBuilder) { + public CopyOnWriteDeleteRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { this.sqlGeneratorFactory = sqlGeneratorFactory; this.conf = conf; - this.cowWithClauseBuilder = cowWithClauseBuilder; + this.cowWithClauseBuilder = new COWWithClauseBuilder(); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java new file mode 100644 index 000000000000..c99324a8be72 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse.rewrite; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.VirtualColumn; +import org.apache.hadoop.hive.ql.parse.ParseUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.MultiInsertSqlGenerator; +import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.function.UnaryOperator; + +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.TARGET_PREFIX; + +public class CopyOnWriteMergeRewriter extends MergeRewriter { + + public CopyOnWriteMergeRewriter(Hive db, HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { + super(db, conf, sqlGeneratorFactory); + } + + @Override + public ParseUtils.ReparseResult rewrite(Context ctx, MergeStatement mergeStatement) throws SemanticException { + + setOperation(ctx); + MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); + handleSource(mergeStatement, sqlGenerator); + + sqlGenerator.append('\n'); + sqlGenerator.append("INSERT INTO ").appendTargetTableName(); + sqlGenerator.append('\n'); + + List whenClauses = Lists.newArrayList(mergeStatement.getWhenClauses()); + + Optional extraPredicate = whenClauses.stream() + .filter(whenClause -> !(whenClause instanceof MergeStatement.InsertClause)) + .map(MergeStatement.WhenClause::getExtraPredicate) + .map(Strings::nullToEmpty) + .reduce((p1, p2) -> isNotBlank(p2) ? p1 + " OR " + p2 : p2); + + whenClauses.removeIf(whenClause -> whenClause instanceof MergeStatement.DeleteClause); + extraPredicate.ifPresent(p -> whenClauses.add(new MergeStatement.DeleteClause(p, null))); + + MergeStatement.MergeSqlGenerator mergeSqlGenerator = createMergeSqlGenerator(mergeStatement, sqlGenerator); + + for (MergeStatement.WhenClause whenClause : whenClauses) { + whenClause.toSql(mergeSqlGenerator); + } + + // TODO: handleCardinalityViolation; + + ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(ctx, sqlGenerator.toString()); + Context rewrittenCtx = rr.rewrittenCtx; + setOperation(rewrittenCtx); + + //set dest name mapping on new context; 1st child is TOK_FROM + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.MERGE); + return rr; + } + + @Override + protected CopyOnWriteMergeWhenClauseSqlGenerator createMergeSqlGenerator( + MergeStatement mergeStatement, MultiInsertSqlGenerator sqlGenerator) { + return new CopyOnWriteMergeWhenClauseSqlGenerator(conf, sqlGenerator, mergeStatement); + } + + private void handleSource(MergeStatement mergeStatement, MultiInsertSqlGenerator sqlGenerator) { + boolean hasWhenNotMatchedInsertClause = mergeStatement.hasWhenNotMatchedInsertClause(); + + String sourceName = mergeStatement.getSourceName(); + String sourceAlias = mergeStatement.getSourceAlias(); + + String targetAlias = mergeStatement.getTargetAlias(); + String onClauseAsString = replaceColumnRefsWithTargetPrefix(targetAlias, mergeStatement.getOnClauseAsText()); + + sqlGenerator.newCteExpr(); + + sqlGenerator.append(sourceName + " AS ( SELECT * FROM\n"); + sqlGenerator.append("(SELECT "); + sqlGenerator.appendAcidSelectColumns(Context.Operation.MERGE); + sqlGenerator.appendAllColsOfTargetTable(TARGET_PREFIX); + sqlGenerator.append(" FROM ").appendTargetTableName().append(") "); + sqlGenerator.append(targetAlias); + sqlGenerator.append('\n'); + sqlGenerator.indent().append(hasWhenNotMatchedInsertClause ? "FULL OUTER JOIN" : "LEFT OUTER JOIN").append("\n"); + sqlGenerator.indent().append(sourceAlias); + sqlGenerator.append('\n'); + sqlGenerator.indent().append("ON ").append(onClauseAsString); + sqlGenerator.append('\n'); + sqlGenerator.append(")"); + + sqlGenerator.addCteExpr(); + } + + private static String replaceColumnRefsWithTargetPrefix(String columnRef, String strValue) { + return strValue.replaceAll(columnRef + "\\.(`?)", "$1" + TARGET_PREFIX); + } + + static class CopyOnWriteMergeWhenClauseSqlGenerator extends MergeRewriter.MergeWhenClauseSqlGenerator { + + private final COWWithClauseBuilder cowWithClauseBuilder; + + CopyOnWriteMergeWhenClauseSqlGenerator( + HiveConf conf, MultiInsertSqlGenerator sqlGenerator, MergeStatement mergeStatement) { + super(conf, sqlGenerator, mergeStatement); + this.cowWithClauseBuilder = new COWWithClauseBuilder(); + } + + @Override + public void appendWhenNotMatchedInsertClause(MergeStatement.InsertClause insertClause) { + String targetAlias = mergeStatement.getTargetAlias(); + + if (mergeStatement.getWhenClauses().size() > 1) { + sqlGenerator.append("union all\n"); + } + sqlGenerator.append(" -- insert clause\n").append("SELECT "); + + if (isNotBlank(hintStr)) { + sqlGenerator.append(hintStr); + hintStr = null; + } + List values = sqlGenerator.getDeleteValues(Context.Operation.MERGE); + values.add(insertClause.getValuesClause()); + + sqlGenerator.append(StringUtils.join(values, ",")); + sqlGenerator.append("\nFROM " + mergeStatement.getSourceName()); + sqlGenerator.append("\n WHERE "); + + StringBuilder whereClause = new StringBuilder(insertClause.getPredicate()); + + if (insertClause.getExtraPredicate() != null) { + //we have WHEN NOT MATCHED AND THEN INSERT + whereClause.append(" AND ").append(insertClause.getExtraPredicate()); + } + sqlGenerator.append( + replaceColumnRefsWithTargetPrefix(targetAlias, whereClause.toString())); + sqlGenerator.append('\n'); + } + + @Override + public void appendWhenMatchedUpdateClause(MergeStatement.UpdateClause updateClause) { + Table targetTable = mergeStatement.getTargetTable(); + String targetAlias = mergeStatement.getTargetAlias(); + String onClauseAsString = mergeStatement.getOnClauseAsText(); + + UnaryOperator columnRefsFunc = value -> replaceColumnRefsWithTargetPrefix(targetAlias, value); + sqlGenerator.append(" -- update clause (insert part)\n").append("SELECT "); + + if (isNotBlank(hintStr)) { + sqlGenerator.append(hintStr); + hintStr = null; + } + List values = new ArrayList<>(targetTable.getCols().size() + targetTable.getPartCols().size()); + values.addAll(sqlGenerator.getDeleteValues(Context.Operation.MERGE)); + addValues(targetTable, targetAlias, updateClause.getNewValuesMap(), values); + + sqlGenerator.append(columnRefsFunc.apply(StringUtils.join(values, ","))); + sqlGenerator.append("\nFROM " + mergeStatement.getSourceName()); + + addWhereClauseOfUpdate( + onClauseAsString, updateClause.getExtraPredicate(), updateClause.getDeleteExtraPredicate(), sqlGenerator, + columnRefsFunc); + sqlGenerator.append("\n"); + } + + @Override + protected String getRhsExpValue(String newValue, String alias) { + return String.format("%s AS %s", newValue, alias); + } + + @Override + protected void handleWhenMatchedDelete(String onClauseAsString, String extraPredicate, String updateExtraPredicate, + String hintStr, MultiInsertSqlGenerator sqlGenerator) { + String targetAlias = mergeStatement.getTargetAlias(); + String sourceName = mergeStatement.getSourceName(); + String onClausePredicate = mergeStatement.getOnClausePredicate(); + + UnaryOperator columnRefsFunc = value -> replaceColumnRefsWithTargetPrefix(targetAlias, value); + List deleteValues = sqlGenerator.getDeleteValues(Context.Operation.DELETE); + + List whenClauses = mergeStatement.getWhenClauses(); + if (whenClauses.size() > 1 || whenClauses.get(0) instanceof MergeStatement.UpdateClause) { + sqlGenerator.append("union all\n"); + } + sqlGenerator.append(" -- delete clause\n").append("SELECT "); + + if (isNotBlank(hintStr)) { + sqlGenerator.append(hintStr); + } + sqlGenerator.append(StringUtils.join(deleteValues, ",")); + sqlGenerator.append("\nFROM " + sourceName); + sqlGenerator.indent().append("WHERE "); + + StringBuilder whereClause = new StringBuilder(onClauseAsString); + if (isNotBlank(extraPredicate)) { + //we have WHEN MATCHED AND THEN DELETE + whereClause.append(" AND ").append(extraPredicate); + } + String whereClauseStr = columnRefsFunc.apply(whereClause.toString()); + String filePathCol = HiveUtils.unparseIdentifier(TARGET_PREFIX + VirtualColumn.FILE_PATH.getName(), conf); + + sqlGenerator.append("\n").indent(); + sqlGenerator.append("NOT(").append(whereClauseStr.replace("=","<=>")); + + if (isNotBlank(onClausePredicate)) { + sqlGenerator.append(" OR "); + sqlGenerator.append(columnRefsFunc.apply(mergeStatement.getOnClausePredicate())); + } + sqlGenerator.append(")\n").indent(); + // Add the file path filter that matches the delete condition. + sqlGenerator.append("AND ").append(filePathCol); + sqlGenerator.append(" IN ( select ").append(filePathCol).append(" from t )"); + sqlGenerator.append("\nunion all"); + sqlGenerator.append("\nselect * from t"); + + cowWithClauseBuilder.appendWith(sqlGenerator, sourceName, filePathCol, whereClauseStr, false); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java index dc53a0304970..bf88c1da3d47 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java @@ -45,40 +45,51 @@ public class CopyOnWriteUpdateRewriter implements Rewriter { private final SetClausePatcher setClausePatcher; - public CopyOnWriteUpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, - COWWithClauseBuilder cowWithClauseBuilder, SetClausePatcher setClausePatcher) { + public CopyOnWriteUpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { this.conf = conf; this.sqlGeneratorFactory = sqlGeneratorFactory; - this.cowWithClauseBuilder = cowWithClauseBuilder; - this.setClausePatcher = setClausePatcher; + this.cowWithClauseBuilder = new COWWithClauseBuilder(); + this.setClausePatcher = new SetClausePatcher(); } @Override public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateBlock) throws SemanticException { - Tree wherePredicateNode = updateBlock.getWhereTree().getChild(0); - String whereClause = context.getTokenRewriteStream().toString( - wherePredicateNode.getTokenStartIndex(), wherePredicateNode.getTokenStopIndex()); String filePathCol = HiveUtils.unparseIdentifier(VirtualColumn.FILE_PATH.getName(), conf); - MultiInsertSqlGenerator sqlGenerator = sqlGeneratorFactory.createSqlGenerator(); - cowWithClauseBuilder.appendWith(sqlGenerator, filePathCol, whereClause); - - sqlGenerator.append("insert into table "); + String whereClause = null; + int columnOffset = 0; + + boolean shouldOverwrite = updateBlock.getWhereTree() == null; + if (shouldOverwrite) { + sqlGenerator.append("insert overwrite table "); + } else { + Tree wherePredicateNode = updateBlock.getWhereTree().getChild(0); + whereClause = context.getTokenRewriteStream().toString( + wherePredicateNode.getTokenStartIndex(), wherePredicateNode.getTokenStopIndex()); + + cowWithClauseBuilder.appendWith(sqlGenerator, filePathCol, whereClause); + sqlGenerator.append("insert into table "); + + columnOffset = sqlGenerator.getDeleteValues(Context.Operation.UPDATE).size(); + } sqlGenerator.appendTargetTableName(); sqlGenerator.appendPartitionColsOfTarget(); - - int columnOffset = sqlGenerator.getDeleteValues(Context.Operation.UPDATE).size(); + sqlGenerator.append(" select "); - sqlGenerator.appendAcidSelectColumns(Context.Operation.UPDATE); - sqlGenerator.removeLastChar(); + if (!shouldOverwrite) { + sqlGenerator.appendAcidSelectColumns(Context.Operation.UPDATE); + sqlGenerator.removeLastChar(); + } Map setColExprs = new HashMap<>(updateBlock.getSetCols().size()); List nonPartCols = updateBlock.getTargetTable().getCols(); for (int i = 0; i < nonPartCols.size(); i++) { - sqlGenerator.append(','); + if (columnOffset > 0 || i > 0) { + sqlGenerator.append(','); + } String name = nonPartCols.get(i).getName(); ASTNode setCol = updateBlock.getSetCols().get(name); String identifier = HiveUtils.unparseIdentifier(name, this.conf); @@ -95,7 +106,7 @@ public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateB sqlGenerator.append(" from "); sqlGenerator.appendTargetTableName(); - if (updateBlock.getWhereTree() != null) { + if (whereClause != null) { sqlGenerator.append("\nwhere "); sqlGenerator.append(whereClause); sqlGenerator.append("\nunion all"); @@ -118,13 +129,19 @@ public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateB Context rewrittenCtx = rr.rewrittenCtx; ASTNode rewrittenTree = rr.rewrittenTree; - ASTNode rewrittenInsert = (ASTNode) new CalcitePlanner.ASTSearcher().simpleBreadthFirstSearch( - rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_UNIONALL).getChild(0).getChild(0) + ASTNode rewrittenInsert = (ASTNode) (!shouldOverwrite ? + new CalcitePlanner.ASTSearcher().simpleBreadthFirstSearch( + rewrittenTree, HiveParser.TOK_FROM, HiveParser.TOK_SUBQUERY, HiveParser.TOK_UNIONALL) + .getChild(0).getChild(0) : rewrittenTree) .getChild(1); - rewrittenCtx.setOperation(Context.Operation.UPDATE); - rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); - + if (shouldOverwrite) { + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.INSERT); + } else { + rewrittenCtx.setOperation(Context.Operation.UPDATE); + rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE); + } + setClausePatcher.patchProjectionForUpdate(rewrittenInsert, setColExprs); // Note: this will overwrite this.ctx with rewrittenCtx diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java index 505760e6d969..5066a5307860 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java @@ -30,9 +30,9 @@ public class DeleteRewriter implements Rewriter { protected final SqlGeneratorFactory sqlGeneratorFactory; private final WhereClausePatcher whereClausePatcher; - public DeleteRewriter(SqlGeneratorFactory sqlGeneratorFactory, WhereClausePatcher whereClausePatcher) { + public DeleteRewriter(SqlGeneratorFactory sqlGeneratorFactory) { this.sqlGeneratorFactory = sqlGeneratorFactory; - this.whereClausePatcher = whereClausePatcher; + this.whereClausePatcher = new WhereClausePatcher(); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java index 7cb7cf990030..02630d352487 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriterFactory.java @@ -21,9 +21,7 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; -import org.apache.hadoop.hive.ql.parse.rewrite.sql.WhereClausePatcher; import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.DELETE_PREFIX; @@ -45,9 +43,9 @@ public Rewriter createRewriter(Table table, String targetTableF table, targetTableFullName, conf, null, DELETE_PREFIX); if (copyOnWriteMode) { - return new CopyOnWriteDeleteRewriter(conf, sqlGeneratorFactory, new COWWithClauseBuilder()); + return new CopyOnWriteDeleteRewriter(conf, sqlGeneratorFactory); } else { - return new DeleteRewriter(sqlGeneratorFactory, new WhereClausePatcher()); + return new DeleteRewriter(sqlGeneratorFactory); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java index ed2c853a4092..64152f419b6b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java @@ -40,6 +40,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.function.UnaryOperator; import static org.apache.commons.lang3.StringUtils.isNotBlank; @@ -47,7 +48,7 @@ public class MergeRewriter implements Rewriter, MergeStatement.D private final Hive db; protected final HiveConf conf; - private final SqlGeneratorFactory sqlGeneratorFactory; + protected final SqlGeneratorFactory sqlGeneratorFactory; public MergeRewriter(Hive db, HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { this.db = db; @@ -161,7 +162,7 @@ protected void setOperation(Context context) { protected static class MergeWhenClauseSqlGenerator implements MergeStatement.MergeSqlGenerator { - private final HiveConf conf; + protected final HiveConf conf; protected final MultiInsertSqlGenerator sqlGenerator; protected final MergeStatement mergeStatement; protected String hintStr; @@ -218,30 +219,43 @@ public void appendWhenMatchedUpdateClause(MergeStatement.UpdateClause updateClau protected void addValues(Table targetTable, String targetAlias, Map newValues, List values) { + UnaryOperator formatter = name -> String.format("%s.%s", targetAlias, + HiveUtils.unparseIdentifier(name, conf)); + for (FieldSchema fieldSchema : targetTable.getCols()) { if (newValues.containsKey(fieldSchema.getName())) { - values.add(newValues.get(fieldSchema.getName())); + String rhsExp = newValues.get(fieldSchema.getName()); + values.add(getRhsExpValue(rhsExp, formatter.apply(fieldSchema.getName()))); } else { - values.add( - String.format("%s.%s", targetAlias, HiveUtils.unparseIdentifier(fieldSchema.getName(), conf))); + values.add(formatter.apply(fieldSchema.getName())); } } - - targetTable.getPartCols().forEach(fieldSchema -> - values.add( - String.format("%s.%s", targetAlias, HiveUtils.unparseIdentifier(fieldSchema.getName(), conf)))); + + targetTable.getPartCols().forEach(fieldSchema -> values.add( + formatter.apply(fieldSchema.getName()))); + } + + protected String getRhsExpValue(String newValue, String alias) { + return newValue; } protected void addWhereClauseOfUpdate(String onClauseAsString, String extraPredicate, String deleteExtraPredicate, MultiInsertSqlGenerator sqlGenerator) { - sqlGenerator.indent().append("WHERE ").append(onClauseAsString); + addWhereClauseOfUpdate(onClauseAsString, extraPredicate, deleteExtraPredicate, sqlGenerator, UnaryOperator.identity()); + } + + protected void addWhereClauseOfUpdate(String onClauseAsString, String extraPredicate, String deleteExtraPredicate, + MultiInsertSqlGenerator sqlGenerator, UnaryOperator columnRefsFunc) { + StringBuilder whereClause = new StringBuilder(onClauseAsString); if (extraPredicate != null) { //we have WHEN MATCHED AND THEN DELETE - sqlGenerator.append(" AND ").append(extraPredicate); + whereClause.append(" AND ").append(extraPredicate); } if (deleteExtraPredicate != null) { - sqlGenerator.append(" AND NOT(").append(deleteExtraPredicate).append(")"); + whereClause.append(" AND NOT(").append(deleteExtraPredicate).append(")"); } + sqlGenerator.indent().append("WHERE "); + sqlGenerator.append(columnRefsFunc.apply(whereClause.toString())); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java index 145639bbdf54..f63429e15102 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriterFactory.java @@ -17,16 +17,20 @@ */ package org.apache.hadoop.hive.ql.parse.rewrite; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; +import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.TARGET_PREFIX; +import static org.apache.commons.lang3.StringUtils.EMPTY; + public class MergeRewriterFactory implements RewriterFactory { private final Hive db; private final HiveConf conf; @@ -40,25 +44,28 @@ public MergeRewriterFactory(HiveConf conf) throws SemanticException { this.conf = conf; } - @Override public Rewriter createRewriter(Table table, String targetTableFullName, String subQueryAlias) throws SemanticException { boolean splitUpdate = HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPLIT_UPDATE); - boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(table, true); - if (nonNativeAcid && !splitUpdate) { - throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); + boolean copyOnWriteMode = false; + HiveStorageHandler storageHandler = table.getStorageHandler(); + if (storageHandler != null) { + copyOnWriteMode = storageHandler.shouldOverwrite(table, Context.Operation.MERGE); } - + SqlGeneratorFactory sqlGeneratorFactory = new SqlGeneratorFactory( - table, - targetTableFullName, - conf, - subQueryAlias, - StringUtils.EMPTY); + table, targetTableFullName, conf, !copyOnWriteMode ? subQueryAlias : null, + copyOnWriteMode ? TARGET_PREFIX : EMPTY); - if (splitUpdate) { + if (copyOnWriteMode) { + return new CopyOnWriteMergeRewriter(db, conf, sqlGeneratorFactory); + } else if (splitUpdate) { return new SplitMergeRewriter(db, conf, sqlGeneratorFactory); + } else { + if (AcidUtils.isNonNativeAcidTable(table)) { + throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); + } + return new MergeRewriter(db, conf, sqlGeneratorFactory); } - return new MergeRewriter(db, conf, sqlGeneratorFactory); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java index 64d9c384161c..341f2a8853b9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeStatement.java @@ -42,6 +42,7 @@ public static class MergeStatementBuilder { private final String targetAlias; private String sourceName; private String sourceAlias; + private String onClausePredicate; private String onClauseAsText; private String hintStr; private final List whenClauses; @@ -61,6 +62,10 @@ public MergeStatementBuilder sourceAlias(String sourceAlias) { this.sourceAlias = sourceAlias; return this; } + public MergeStatementBuilder onClausePredicate(String onClausePredicate) { + this.onClausePredicate = onClausePredicate; + return this; + } public MergeStatementBuilder onClauseAsText(String onClauseAsText) { this.onClauseAsText = onClauseAsText; return this; @@ -76,7 +81,8 @@ public MergeStatementBuilder addWhenClause(WhenClause whenClause) { } public MergeStatement build() { - return new MergeStatement(targetTable, targetName, targetAlias, sourceName, sourceAlias, onClauseAsText, hintStr, + return new MergeStatement(targetTable, targetName, targetAlias, sourceName, sourceAlias, + onClausePredicate, onClauseAsText, hintStr, Collections.unmodifiableList(whenClauses)); } } @@ -86,17 +92,19 @@ public MergeStatement build() { private final String targetAlias; private final String sourceName; private final String sourceAlias; + private final String onClausePredicate; private final String onClauseAsText; private final String hintStr; private final List whenClauses; private MergeStatement(Table targetTable, String targetName, String targetAlias, String sourceName, String sourceAlias, - String onClauseAsText, String hintStr, List whenClauses) { + String onClausePredicate, String onClauseAsText, String hintStr, List whenClauses) { this.targetTable = targetTable; this.targetName = targetName; this.targetAlias = targetAlias; this.sourceName = sourceName; this.sourceAlias = sourceAlias; + this.onClausePredicate = onClausePredicate; this.onClauseAsText = onClauseAsText; this.hintStr = hintStr; this.whenClauses = whenClauses; @@ -122,6 +130,10 @@ public String getSourceAlias() { return sourceAlias; } + public String getOnClausePredicate() { + return onClausePredicate; + } + public String getOnClauseAsText() { return onClauseAsText; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java index a79ec57fdb25..75e46d2c78fe 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java @@ -43,10 +43,10 @@ public class SplitUpdateRewriter implements Rewriter { protected final SqlGeneratorFactory sqlGeneratorFactory; private final SetClausePatcher setClausePatcher; - public SplitUpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, SetClausePatcher setClausePatcher) { + public SplitUpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { this.conf = conf; this.sqlGeneratorFactory = sqlGeneratorFactory; - this.setClausePatcher = setClausePatcher; + this.setClausePatcher = new SetClausePatcher(); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java index 1e7a5ec7c326..6595d1b303b0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java @@ -42,12 +42,11 @@ public class UpdateRewriter implements Rewriter { private final WhereClausePatcher whereClausePatcher; private final SetClausePatcher setClausePatcher; - public UpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory, - WhereClausePatcher whereClausePatcher, SetClausePatcher setClausePatcher) { + public UpdateRewriter(HiveConf conf, SqlGeneratorFactory sqlGeneratorFactory) { this.conf = conf; this.sqlGeneratorFactory = sqlGeneratorFactory; - this.whereClausePatcher = whereClausePatcher; - this.setClausePatcher = setClausePatcher; + this.whereClausePatcher = new WhereClausePatcher(); + this.setClausePatcher = new SetClausePatcher(); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java index d17d5e97b3fe..88949c9c6bd7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriterFactory.java @@ -24,10 +24,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.rewrite.sql.COWWithClauseBuilder; -import org.apache.hadoop.hive.ql.parse.rewrite.sql.SetClausePatcher; import org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory; -import org.apache.hadoop.hive.ql.parse.rewrite.sql.WhereClausePatcher; import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.DELETE_PREFIX; import static org.apache.hadoop.hive.ql.parse.rewrite.sql.SqlGeneratorFactory.SUB_QUERY_ALIAS; @@ -52,15 +49,14 @@ public Rewriter createRewriter(Table table, String targetTableF table, targetTableFullName, conf, splitUpdate && !copyOnWriteMode ? SUB_QUERY_ALIAS : null, DELETE_PREFIX); if (copyOnWriteMode) { - return new CopyOnWriteUpdateRewriter( - conf, sqlGeneratorFactory, new COWWithClauseBuilder(), new SetClausePatcher()); + return new CopyOnWriteUpdateRewriter(conf, sqlGeneratorFactory); } else if (splitUpdate) { - return new SplitUpdateRewriter(conf, sqlGeneratorFactory, new SetClausePatcher()); + return new SplitUpdateRewriter(conf, sqlGeneratorFactory); } else { - if (AcidUtils.isNonNativeAcidTable(table, true)) { + if (AcidUtils.isNonNativeAcidTable(table)) { throw new SemanticException(ErrorMsg.NON_NATIVE_ACID_UPDATE.getErrorCodedMsg()); } - return new UpdateRewriter(conf, sqlGeneratorFactory, new WhereClausePatcher(), new SetClausePatcher()); + return new UpdateRewriter(conf, sqlGeneratorFactory); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java index 0c4e5ac6ebea..de5b45c38c74 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/COWWithClauseBuilder.java @@ -22,23 +22,31 @@ public class COWWithClauseBuilder { public void appendWith(MultiInsertSqlGenerator sqlGenerator, String filePathCol, String whereClause) { - sqlGenerator.append("WITH t AS ("); - sqlGenerator.append("\n"); + appendWith(sqlGenerator, null, filePathCol, whereClause, true); + } + public void appendWith(MultiInsertSqlGenerator sqlGenerator, String sourceName, String filePathCol, + String whereClause, boolean skipPrefix) { + sqlGenerator.newCteExpr(); + + sqlGenerator.append("t AS ("); + sqlGenerator.append("\n").indent(); sqlGenerator.append("select "); - sqlGenerator.appendAcidSelectColumnsForDeletedRecords(Context.Operation.DELETE); + sqlGenerator.appendAcidSelectColumnsForDeletedRecords(Context.Operation.DELETE, skipPrefix); sqlGenerator.removeLastChar(); sqlGenerator.append(" from ("); - sqlGenerator.append("\n"); + sqlGenerator.append("\n").indent().indent(); sqlGenerator.append("select "); - sqlGenerator.appendAcidSelectColumnsForDeletedRecords(Context.Operation.DELETE); + sqlGenerator.appendAcidSelectColumnsForDeletedRecords(Context.Operation.DELETE, skipPrefix); sqlGenerator.append(" row_number() OVER (partition by ").append(filePathCol).append(") rn"); sqlGenerator.append(" from "); - sqlGenerator.append(sqlGenerator.getTargetTableFullName()); - sqlGenerator.append("\n"); + sqlGenerator.append(sourceName == null ? sqlGenerator.getTargetTableFullName() : sourceName); + sqlGenerator.append("\n").indent().indent(); sqlGenerator.append("where ").append(whereClause); - sqlGenerator.append("\n"); + sqlGenerator.append("\n").indent(); sqlGenerator.append(") q"); - sqlGenerator.append("\n"); + sqlGenerator.append("\n").indent(); sqlGenerator.append("where rn=1\n)\n"); + + sqlGenerator.addCteExpr(); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java index 2e568a6f51c4..554b6d394ee5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java @@ -24,7 +24,9 @@ import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; +import java.util.ArrayDeque; import java.util.List; +import java.util.Deque; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; @@ -36,8 +38,11 @@ public abstract class MultiInsertSqlGenerator { protected final String targetTableFullName; protected final HiveConf conf; protected final String subQueryAlias; - protected final StringBuilder queryStr; - + protected StringBuilder queryStr; + private Deque stack = new ArrayDeque<>(); + + private int nextCteExprPos = 0; + protected MultiInsertSqlGenerator( Table targetTable, String targetTableFullName, HiveConf conf, String subQueryAlias) { this.targetTable = targetTable; @@ -58,6 +63,10 @@ public String getTargetTableFullName() { public abstract void appendAcidSelectColumns(Context.Operation operation); public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation) { + appendAcidSelectColumnsForDeletedRecords(operation, true); + } + + public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation, boolean skipPrefix) { throw new UnsupportedOperationException(); } @@ -136,7 +145,7 @@ public MultiInsertSqlGenerator append(String sqlTextFragment) { public String toString() { return queryStr.toString(); } - + public void removeLastChar() { queryStr.setLength(queryStr.length() - 1); } @@ -146,18 +155,22 @@ public void appendPartColsOfTargetTableWithComma(String alias) { return; } queryStr.append(','); - appendCols(targetTable.getPartCols(), alias); + appendCols(targetTable.getPartCols(), alias, null); } + public void appendAllColsOfTargetTable(String prefix) { + appendCols(targetTable.getAllCols(), null, prefix); + } + public void appendAllColsOfTargetTable() { appendCols(targetTable.getAllCols()); } public void appendCols(List columns) { - appendCols(columns, null); + appendCols(columns, null, null); } - public void appendCols(List columns, String alias) { + public void appendCols(List columns, String alias, String prefix) { if (columns == null) { return; } @@ -179,6 +192,12 @@ public void appendCols(List columns, String alias) { queryStr.append(quotedAlias).append('.'); } queryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf)); + + if (isNotBlank(prefix)) { + queryStr.append(" AS "); + String prefixedIdentifier = HiveUtils.unparseIdentifier(prefix + fschema.getName(), this.conf); + queryStr.append(prefixedIdentifier); + } } } @@ -201,4 +220,16 @@ public MultiInsertSqlGenerator appendSubQueryAlias() { queryStr.append(subQueryAlias); return this; } + + public MultiInsertSqlGenerator newCteExpr(){ + stack.push(queryStr); + queryStr = new StringBuilder(nextCteExprPos > 0 ? ",\n" : "WITH "); + return this; + } + + public MultiInsertSqlGenerator addCteExpr(){ + queryStr = stack.pop().insert(nextCteExprPos, queryStr); + nextCteExprPos = queryStr.length(); + return this; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java index cbd4014ab1f5..6893a8810bc3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java @@ -39,24 +39,30 @@ public NonNativeAcidMultiInsertSqlGenerator( @Override public void appendAcidSelectColumns(Context.Operation operation) { - appendAcidSelectColumns(operation, false); + appendAcidSelectColumns(operation, false, false); } @Override - public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation) { - appendAcidSelectColumns(operation, true); + public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation, boolean skipPrefix) { + appendAcidSelectColumns(operation, true, skipPrefix); } - private void appendAcidSelectColumns(Context.Operation operation, boolean markRowIdAsDeleted) { + private void appendAcidSelectColumns(Context.Operation operation, boolean markRowIdAsDeleted, boolean skipPrefix) { List acidSelectColumns = targetTable.getStorageHandler().acidSelectColumns(targetTable, operation); for (FieldSchema fieldSchema : acidSelectColumns) { - String identifier = markRowIdAsDeleted && fieldSchema.equals(targetTable.getStorageHandler().getRowId()) ? + boolean deletedRowId = markRowIdAsDeleted && fieldSchema.equals(targetTable.getStorageHandler().getRowId()); + String identifier = deletedRowId ? "-1" : HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); - queryStr.append(identifier); + if (!markRowIdAsDeleted || skipPrefix) { + queryStr.append(identifier); + } - if (StringUtils.isNotEmpty(deletePrefix) && !markRowIdAsDeleted) { - queryStr.append(" AS "); - String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); + if (StringUtils.isNotEmpty(deletePrefix) && (!markRowIdAsDeleted || !skipPrefix)) { + if (!markRowIdAsDeleted) { + queryStr.append(" AS "); + } + String prefixedIdentifier = deletedRowId ? + "-1" : HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); queryStr.append(prefixedIdentifier); } queryStr.append(","); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java index 6c3e48a954c8..6910ea34623d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/SqlGeneratorFactory.java @@ -23,6 +23,7 @@ public class SqlGeneratorFactory { public static final String DELETE_PREFIX = "__d__"; + public static final String TARGET_PREFIX = "t__"; public static final String SUB_QUERY_ALIAS = "s"; private final Table targetTable; @@ -41,7 +42,7 @@ public SqlGeneratorFactory( } public MultiInsertSqlGenerator createSqlGenerator() { - boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable, true); + boolean nonNativeAcid = AcidUtils.isNonNativeAcidTable(targetTable); return nonNativeAcid ? new NonNativeAcidMultiInsertSqlGenerator(targetTable, targetTableFullName, conf, subQueryAlias, deletePrefix) : new NativeAcidMultiInsertSqlGenerator(targetTable, targetTableFullName, conf, subQueryAlias); } From f4c4d6b2ec1555abb844b6f348f879719b0a67c3 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 27 Nov 2023 00:52:29 +0800 Subject: [PATCH 067/179] HIVE-27896: Remove common-lang usage (#4891). (Cheng Pan, reviewed by Ayush Saxena) --- .../org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java index d12a25b19905..17002e045bed 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java @@ -1211,7 +1211,7 @@ private static void createReplImportTasks( } if (tblDesc.getLocation() == null) { - if (parentDb != null && !tblDesc.isExternal() && org.apache.commons.lang.StringUtils.isNotBlank(parentDb.getManagedLocationUri())) { + if (parentDb != null && !tblDesc.isExternal() && StringUtils.isNotBlank(parentDb.getManagedLocationUri())) { tblDesc.setLocation(new Path(parentDb.getManagedLocationUri(), tblDesc.getTableName()).toString()); LOG.info("Setting the location for table {} as {}", tblDesc.getTableName(), tblDesc.getLocation()); } else if (!waitOnPrecursor) { From 9b4ea7affa4902fc2849f1a88b68103940fc9866 Mon Sep 17 00:00:00 2001 From: Ramesh Kumar Date: Mon, 27 Nov 2023 07:41:28 -0800 Subject: [PATCH 068/179] HIVE_27843 Add QueryOperation to Hive proto logger for post execution hook information (Ramesh Kumar, reviewed by Attila Turoczy, Ayush Saxena) --- .../org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java | 3 ++- .../apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java index 618c7108f677..74d6ac4ce017 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java @@ -168,7 +168,7 @@ public enum EventType { public enum OtherInfoType { QUERY, STATUS, TEZ, MAPRED, INVOKER_INFO, SESSION_ID, THREAD_NAME, VERSION, CLIENT_IP_ADDRESS, - HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID, ERROR_MESSAGE + HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID, ERROR_MESSAGE, QUERY_TYPE } public enum ExecutionMode { @@ -445,6 +445,7 @@ private HiveHookEventProtoPartialBuilder getPostHookEvent(HookContext hookContex } addMapEntry(builder, OtherInfoType.STATUS, Boolean.toString(success)); addMapEntry(builder, OtherInfoType.ERROR_MESSAGE, hookContext.getErrorMessage()); + addMapEntry(builder, OtherInfoType.QUERY_TYPE, hookContext.getQueryState().getCommandType()); JSONObject perfObj = new JSONObject(); for (String key : hookContext.getPerfLogger().getEndTimes().keySet()) { perfObj.put(key, hookContext.getPerfLogger().getDuration(key)); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java index 6b65f0c7873d..bff605967cff 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveProtoLoggingHook.java @@ -86,6 +86,7 @@ public void setup() throws Exception { tmpFolder = folder.newFolder().getAbsolutePath(); conf.setVar(HiveConf.ConfVars.HIVE_PROTO_EVENTS_BASE_PATH, tmpFolder); QueryState state = new QueryState.Builder().withHiveConf(conf).build(); + state.setCommandType(HiveOperation.QUERY); @SuppressWarnings("serial") QueryPlan queryPlan = new QueryPlan(HiveOperation.QUERY) {}; queryPlan.setQueryId("test_queryId"); @@ -246,6 +247,7 @@ public void testPostEventLog() throws Exception { Assert.assertEquals("test_op_id", event.getOperationId()); assertOtherInfo(event, OtherInfoType.STATUS, Boolean.TRUE.toString()); + assertOtherInfo(event, OtherInfoType.QUERY_TYPE, HiveOperation.QUERY.toString()); String val = findOtherInfo(event, OtherInfoType.PERF); Map map = new ObjectMapper().readValue(val, new TypeReference>() {}); From 6b2e21a93ef3c1776b689a7953fc59dbf52e4be4 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 28 Nov 2023 11:47:18 +0530 Subject: [PATCH 069/179] HIVE-27906: Iceberg: Implement Delete Orphan Files. (#4897). (Ayush Saxena, reviewed by zhangbutao) --- .../mr/hive/HiveIcebergStorageHandler.java | 31 +++ .../actions/HiveIcebergDeleteOrphanFiles.java | 232 ++++++++++++++++++ .../hive/TestHiveIcebergExpireSnapshots.java | 55 +++++ .../hadoop/hive/ql/parse/AlterClauseParser.g | 2 + .../hadoop/hive/ql/parse/HiveLexerParent.g | 3 + .../execute/AlterTableExecuteAnalyzer.java | 27 +- .../hive/ql/parse/AlterTableExecuteSpec.java | 29 ++- 7 files changed, 377 insertions(+), 2 deletions(-) create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/actions/HiveIcebergDeleteOrphanFiles.java diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 10a00db534af..c729fcef4300 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -158,6 +158,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; +import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -171,6 +172,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.mr.hive.actions.HiveIcebergDeleteOrphanFiles; import org.apache.iceberg.puffin.Blob; import org.apache.iceberg.puffin.BlobMetadata; import org.apache.iceberg.puffin.Puffin; @@ -849,12 +851,41 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable, IcebergTableUtil.performMetadataDelete(icebergTable, deleteMetadataSpec.getBranchName(), deleteMetadataSpec.getSarg()); break; + case DELETE_ORPHAN_FILES: + int numDeleteThreads = conf.getInt(HiveConf.ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.varname, + HiveConf.ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.defaultIntVal); + AlterTableExecuteSpec.DeleteOrphanFilesDesc deleteOrphanFilesSpec = + (AlterTableExecuteSpec.DeleteOrphanFilesDesc) executeSpec.getOperationParams(); + deleteOrphanFiles(icebergTable, deleteOrphanFilesSpec.getTimestampMillis(), numDeleteThreads); + break; default: throw new UnsupportedOperationException( String.format("Operation type %s is not supported", executeSpec.getOperationType().name())); } } + private void deleteOrphanFiles(Table icebergTable, long timestampMillis, int numThreads) { + ExecutorService deleteExecutorService = null; + try { + if (numThreads > 0) { + LOG.info("Executing delete orphan files on iceberg table {} with {} threads", icebergTable.name(), numThreads); + deleteExecutorService = getDeleteExecutorService(icebergTable.name(), numThreads); + } + + HiveIcebergDeleteOrphanFiles deleteOrphanFiles = new HiveIcebergDeleteOrphanFiles(conf, icebergTable); + deleteOrphanFiles.olderThan(timestampMillis); + if (deleteExecutorService != null) { + deleteOrphanFiles.executeDeleteWith(deleteExecutorService); + } + DeleteOrphanFiles.Result result = deleteOrphanFiles.execute(); + LOG.debug("Cleaned files {} for {}", result.orphanFileLocations(), icebergTable); + } finally { + if (deleteExecutorService != null) { + deleteExecutorService.shutdown(); + } + } + } + private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnapshotsSpec expireSnapshotsSpec, int numThreads) { ExecutorService deleteExecutorService = null; diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/actions/HiveIcebergDeleteOrphanFiles.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/actions/HiveIcebergDeleteOrphanFiles.java new file mode 100644 index 000000000000..3c2e466208f3 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/actions/HiveIcebergDeleteOrphanFiles.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.actions; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.iceberg.DataTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.iceberg.MetadataTableType.ALL_ENTRIES; + +public class HiveIcebergDeleteOrphanFiles implements DeleteOrphanFiles { + + public static final String METADATA_FOLDER_NAME = "metadata"; + public static final String DATA_FOLDER_NAME = "data"; + private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergDeleteOrphanFiles.class); + private String tableLocation; + private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); + private Consumer deleteFunc; + private ExecutorService deleteExecutorService = MoreExecutors.newDirectExecutorService(); + + private final Configuration conf; + private final Table table; + + public HiveIcebergDeleteOrphanFiles(Configuration conf, Table table) { + this.conf = conf; + this.table = table; + this.deleteFunc = file -> table.io().deleteFile(file); + this.tableLocation = table.location(); + } + + @Override + public HiveIcebergDeleteOrphanFiles location(String location) { + this.tableLocation = location; + return this; + } + + @Override + public HiveIcebergDeleteOrphanFiles olderThan(long newOlderThanTimestamp) { + this.olderThanTimestamp = newOlderThanTimestamp; + return this; + } + + // TODO: Implement later, if there is any use case. + @Override + public HiveIcebergDeleteOrphanFiles deleteWith(Consumer newDeleteFunc) { + this.deleteFunc = newDeleteFunc; + return this; + } + + @Override + public HiveIcebergDeleteOrphanFiles executeDeleteWith(ExecutorService executorService) { + this.deleteExecutorService = executorService; + return this; + } + + @Override + public Result execute() { + LOG.info("Cleaning orphan files for {}", table.name()); + HiveIcebergDeleteOrphanFilesResult result = new HiveIcebergDeleteOrphanFilesResult(); + result.addDeletedFiles(cleanContentFiles(olderThanTimestamp)); + result.addDeletedFiles(cleanMetadata(olderThanTimestamp)); + + LOG.debug("Deleting {} files while cleaning orphan files for {}", result.deletedFiles.size(), table.name()); + Tasks.foreach(result.deletedFiles).executeWith(deleteExecutorService).retry(3) + .stopRetryOn(FileNotFoundException.class).suppressFailureWhenFinished().onFailure((file, thrown) -> + LOG.warn("Delete failed for file: {}", file, thrown)).run(deleteFunc::accept); + return result; + } + + private Set cleanContentFiles(long lastTime) { + Set validFiles = Sets.union(getAllContentFilePath(), getAllStatisticsFilePath(table)); + LOG.debug("Valid content file for {} are {}", table.name(), validFiles.size()); + try { + Path dataPath = new Path(tableLocation, DATA_FOLDER_NAME); + return getFilesToBeDeleted(lastTime, validFiles, dataPath); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + + public Set getAllContentFilePath() { + Set validFilesPath = Sets.newHashSet(); + Table metadatTable = getMetadataTable(); + + TableScan tableScan = metadatTable.newScan(); + CloseableIterable manifestFileScanTasks = tableScan.planFiles(); + CloseableIterable entries = CloseableIterable.concat(entriesOfManifest(manifestFileScanTasks)); + + for (StructLike entry : entries) { + StructLike fileRecord = entry.get(4, StructLike.class); + String filePath = fileRecord.get(1, String.class); + validFilesPath.add(getUriPath(filePath)); + } + return validFilesPath; + } + + private Iterable> entriesOfManifest( + CloseableIterable fileScanTasks) { + return Iterables.transform( + fileScanTasks, + task -> { + assert task != null; + return ((DataTask) task).rows(); + }); + } + + public static Set getAllStatisticsFilePath(Table table) { + return ReachableFileUtil.statisticsFilesLocations(table).stream().map(HiveIcebergDeleteOrphanFiles::getUriPath) + .collect(Collectors.toSet()); + } + + protected Set cleanMetadata(long lastTime) { + LOG.info("{} start clean metadata files", table.name()); + try { + Set validFiles = getValidMetadataFiles(table); + LOG.debug("Valid metadata files for {} are {}", table.name(), validFiles); + Path metadataLocation = new Path(tableLocation, METADATA_FOLDER_NAME); + return getFilesToBeDeleted(lastTime, validFiles, metadataLocation); + } catch (IOException ioe) { + throw new UncheckedIOException(ioe); + } + } + + private Set getFilesToBeDeleted(long lastTime, Set validFiles, Path location) + throws IOException { + Set filesToDelete = Sets.newHashSet(); + FileSystem fs = location.getFileSystem(conf); + RemoteIterator metadataLocations = fs.listFiles(location, true); + while (metadataLocations.hasNext()) { + LocatedFileStatus metadataFile = metadataLocations.next(); + if (metadataFile.getModificationTime() < lastTime && !validFiles.contains( + getUriPath(metadataFile.getPath().toString()))) { + filesToDelete.add(metadataFile.getPath().toString()); + } + } + return filesToDelete; + } + + private Table getMetadataTable() { + return MetadataTableUtils.createMetadataTableInstance(((HasTableOperations) table).operations(), table.name(), + table.name() + "#" + ALL_ENTRIES.name(), ALL_ENTRIES); + } + + private static Set getValidMetadataFiles(Table icebergTable) { + Set validFiles = Sets.newHashSet(); + Iterable snapshots = icebergTable.snapshots(); + for (Snapshot snapshot : snapshots) { + String manifestListLocation = snapshot.manifestListLocation(); + validFiles.add(getUriPath(manifestListLocation)); + + List manifestFiles = snapshot.allManifests(icebergTable.io()); + for (ManifestFile manifestFile : manifestFiles) { + validFiles.add(getUriPath(manifestFile.path())); + } + } + Stream.of( + ReachableFileUtil.metadataFileLocations(icebergTable, false).stream(), + ReachableFileUtil.statisticsFilesLocations(icebergTable).stream(), + Stream.of(ReachableFileUtil.versionHintLocation(icebergTable))) + .reduce(Stream::concat) + .orElse(Stream.empty()) + .map(HiveIcebergDeleteOrphanFiles::getUriPath) + .forEach(validFiles::add); + + return validFiles; + } + + private static String getUriPath(String path) { + return URI.create(path).getPath(); + } + + static class HiveIcebergDeleteOrphanFilesResult implements Result { + + private final Set deletedFiles = Sets.newHashSet(); + + @Override + public Iterable orphanFileLocations() { + return deletedFiles; + } + + public void addDeletedFiles(Set files) { + this.deletedFiles.addAll(files); + } + } +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java index 0d65c6c73666..4a3b951bde40 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java @@ -22,10 +22,15 @@ import java.io.IOException; import java.text.SimpleDateFormat; import java.util.Date; +import java.util.List; import org.apache.commons.collections4.IterableUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.functional.RemoteIterators; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -111,4 +116,54 @@ public void testExpireSnapshotsWithRetainLast() throws IOException, InterruptedE table.refresh(); Assert.assertEquals(5, IterableUtils.size(table.snapshots())); } + + @Test + public void testDeleteOrphanFiles() throws IOException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "source"); + Table table = + testTables.createTableWithVersions(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 5); + Assert.assertEquals(5, table.history().size()); + + List rows = shell.executeStatement("SELECT * FROM " + identifier.name()); + List originalRecords = HiveIcebergTestUtils.valueForRow(table.schema(), rows); + Path orphanDataFile = new Path(table.location(), "data/dataFile"); + Path orphanMetadataFile = new Path(table.location(), "metadata/metafile"); + FileSystem fs = orphanDataFile.getFileSystem(shell.getHiveConf()); + fs.create(orphanDataFile).close(); + fs.create(orphanMetadataFile).close(); + + int numDataFiles = RemoteIterators.toList(fs.listFiles(new Path(table.location(), "data"), true)).size(); + int numMetadataFiles = RemoteIterators.toList(fs.listFiles(new Path(table.location(), "metadata"), true)).size(); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE DELETE ORPHAN-FILES"); + + Assert.assertEquals(numDataFiles, + RemoteIterators.toList(fs.listFiles(new Path(table.location(), "data"), true)).size()); + + Assert.assertEquals(numMetadataFiles, + RemoteIterators.toList(fs.listFiles(new Path(table.location(), "metadata"), true)).size()); + + Assert.assertTrue(fs.exists(orphanDataFile)); + Assert.assertTrue(fs.exists(orphanDataFile)); + + long time = System.currentTimeMillis() + 1000; + SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS000000"); + String timeStamp = simpleDateFormat.format(new Date(time)); + shell.executeStatement( + "ALTER TABLE " + identifier.name() + " EXECUTE DELETE ORPHAN-FILES OLDER THAN ('" + timeStamp + "')"); + + Assert.assertEquals(numDataFiles - 1, + RemoteIterators.toList(fs.listFiles(new Path(table.location(), "data"), true)).size()); + + Assert.assertEquals(numMetadataFiles - 1, + RemoteIterators.toList(fs.listFiles(new Path(table.location(), "metadata"), true)).size()); + + Assert.assertFalse(fs.exists(orphanDataFile)); + Assert.assertFalse(fs.exists(orphanDataFile)); + table.refresh(); + + rows = shell.executeStatement("SELECT * FROM " + identifier.name()); + List records = HiveIcebergTestUtils.valueForRow(table.schema(), rows); + HiveIcebergTestUtils.validateData(originalRecords, records, 0); + } } diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g index a184b41e0f44..3e6105957c06 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g @@ -487,6 +487,8 @@ alterStatementSuffixExecute -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS $fromTimestamp $toTimestamp) | KW_EXECUTE KW_EXPIRE_SNAPSHOTS KW_RETAIN KW_LAST numToRetain=Number -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS KW_RETAIN $numToRetain) + | KW_EXECUTE KW_DELETE KW_ORPHAN_FILES (KW_OLDER KW_THAN LPAREN (timestamp=StringLiteral) RPAREN)? + -> ^(TOK_ALTERTABLE_EXECUTE KW_ORPHAN_FILES $timestamp?) ; alterStatementSuffixDropBranch diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g index 883b9774ffb7..a26d66d214d6 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g @@ -399,6 +399,9 @@ KW_RETENTION: 'RETENTION'; KW_TAG: 'TAG'; KW_FAST_FORWARD: 'FAST-FORWARD'; KW_CHERRY_PICK: 'CHERRY-PICK'; +KW_ORPHAN_FILES: 'ORPHAN-FILES'; +KW_OLDER: 'OLDER'; +KW_THAN: 'THAN'; // Operators // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java index cdd6f035d4d3..79448df3b2a5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.parse.HiveParser; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.CherryPickSpec; +import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.DeleteOrphanFilesDesc; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExpireSnapshotsSpec; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.FastForwardSpec; import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.RollbackSpec; @@ -45,9 +46,11 @@ import java.time.ZoneId; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.CHERRY_PICK; +import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.DELETE_ORPHAN_FILES; import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.EXPIRE_SNAPSHOT; import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.FAST_FORWARD; import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.ROLLBACK; @@ -83,7 +86,6 @@ protected void analyzeCommand(TableName tableName, Map partition break; case HiveParser.KW_EXPIRE_SNAPSHOTS: desc = getExpireSnapshotDesc(tableName, partitionSpec, command.getChildren()); - break; case HiveParser.KW_SET_CURRENT_SNAPSHOT: desc = getSetCurrentSnapshotDesc(tableName, partitionSpec, (ASTNode) command.getChild(1)); @@ -94,6 +96,9 @@ protected void analyzeCommand(TableName tableName, Map partition case HiveParser.KW_CHERRY_PICK: desc = getCherryPickDesc(tableName, partitionSpec, (ASTNode) command.getChild(1)); break; + case HiveParser.KW_ORPHAN_FILES: + desc = getDeleteOrphanFilesDesc(tableName, partitionSpec, command.getChildren()); + break; } rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc))); @@ -178,4 +183,24 @@ private static AlterTableExecuteDesc getRollbackDesc(TableName tableName, Map partitionSpec, + List children) throws SemanticException { + + long time = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); + if (children.size() == 2) { + time = getTimeStampMillis((ASTNode) children.get(1)); + } + AlterTableExecuteSpec spec = new AlterTableExecuteSpec(DELETE_ORPHAN_FILES, new DeleteOrphanFilesDesc(time)); + return new AlterTableExecuteDesc(tableName, partitionSpec, spec); + } + + private static long getTimeStampMillis(ASTNode childNode) { + String childNodeText = PlanUtils.stripQuotes(childNode.getText()); + ZoneId timeZone = SessionState.get() == null ? + new HiveConf().getLocalTimeZone() : + SessionState.get().getConf().getLocalTimeZone(); + TimestampTZ time = TimestampTZUtil.parse(PlanUtils.stripQuotes(childNodeText), timeZone); + return time.toEpochMilli(); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java index 2b7ca285e1ca..54c8df3573c6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AlterTableExecuteSpec.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql.parse; import com.google.common.base.MoreObjects; +import com.google.common.base.Preconditions; + import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import java.util.Arrays; @@ -40,7 +42,8 @@ public enum ExecuteOperationType { SET_CURRENT_SNAPSHOT, FAST_FORWARD, CHERRY_PICK, - DELETE_METADATA; + DELETE_METADATA, + DELETE_ORPHAN_FILES; } private final ExecuteOperationType operationType; @@ -270,4 +273,28 @@ public SearchArgument getSarg() { return sarg; } } + + /** + * Value object class, that stores the delete orphan files operation specific parameters. + *
    + *
  • timestampMillis: the time before which files should be considered to be deleted
  • + *
+ */ + public static class DeleteOrphanFilesDesc { + private final long timestampMillis; + + public DeleteOrphanFilesDesc(long timestampMillis) { + Preconditions.checkArgument(timestampMillis >= 0, "TimeStamp Millis shouldn't be negative"); + this.timestampMillis = timestampMillis; + } + + public long getTimestampMillis() { + return timestampMillis; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("timestampMillis", timestampMillis).toString(); + } + } } From 9291b87c57bb19e97c4c7b77e27420b623d485d0 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Wed, 29 Nov 2023 20:33:33 +0800 Subject: [PATCH 070/179] HIVE-27912: Include Iceberg module in nightly builds (#4903). (zhangbutao, reviewed by Ayush Saxena, Attila Turoczy) --- Jenkinsfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Jenkinsfile b/Jenkinsfile index b5080abf79af..0f99c42cc4aa 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -308,7 +308,7 @@ time docker rm -f dev_$dbType || true set -e dev-support/nightly ''' - buildHive("install -Dtest=noMatches -Pdist -pl packaging -am") + buildHive("install -Dtest=noMatches -Pdist -Piceberg -pl packaging -am") } stage('Verify') { sh '''#!/bin/bash From 056430f690ec6389a01a433515970e9a1a811295 Mon Sep 17 00:00:00 2001 From: Devaspati Date: Wed, 29 Nov 2023 18:06:23 +0530 Subject: [PATCH 071/179] HIVE-27907: Upragde aws-java-sdk to 1.12.499 (#4898). (Devaspati Krishnatri , Reviewed by Ayush Saxena, Attila Turoczy) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4edce4eb3a4f..376c698d181a 100644 --- a/pom.xml +++ b/pom.xml @@ -222,7 +222,7 @@ 4.0.3 1.1.0.Final 1.0.1 - 1.12.132 + 1.12.499 2.4.0 5.2.24.RELEASE From eab6809fbbb6274efea07a6649dab19269f48850 Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Thu, 30 Nov 2023 06:07:14 +0100 Subject: [PATCH 072/179] HIVE-27867: Incremental materialized view throws NPE whew Iceberg source table is empty (Krisztian Kasa, reviewed by Denys Kuzmenko) --- ...iveAugmentSnapshotMaterializationRule.java | 38 ++++---- .../views/HiveMaterializedViewUtils.java | 6 +- .../views/HivePushdownSnapshotFilterRule.java | 5 +- ...iveAugmentSnapshotMaterializationRule.java | 90 +++++++++++++++++++ .../TestHivePushdownSnapshotFilterRule.java | 82 +++++++++++++++++ .../calcite/rules/views/TestRuleBase.java | 84 +++++++++++++++++ 6 files changed, 284 insertions(+), 21 deletions(-) create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHiveAugmentSnapshotMaterializationRule.java create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHivePushdownSnapshotFilterRule.java create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestRuleBase.java diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveAugmentSnapshotMaterializationRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveAugmentSnapshotMaterializationRule.java index 26ff609ab6e8..003b7c86ef6a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveAugmentSnapshotMaterializationRule.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveAugmentSnapshotMaterializationRule.java @@ -17,13 +17,14 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views; -import com.google.common.collect.ImmutableList; +import com.google.common.annotations.VisibleForTesting; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelRule; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.fun.SqlStdOperatorTable; @@ -37,12 +38,13 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable; import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter; -import java.util.ArrayList; import java.util.HashSet; -import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import static java.util.Collections.singletonList; + /** * This rule will rewrite the materialized view with information about * its invalidation data. In particular, if any of the tables used by the @@ -90,12 +92,13 @@ HiveAugmentSnapshotMaterializationRule.Config withMvMetaStoredSnapshot( private static RelDataType snapshotIdType = null; - private static RelDataType snapshotIdType(RelBuilder relBuilder) { + @VisibleForTesting + static RelDataType snapshotIdType(RelDataTypeFactory typeFactory) { if (snapshotIdType == null) { try { - snapshotIdType = relBuilder.getTypeFactory().createSqlType( + snapshotIdType = typeFactory.createSqlType( TypeConverter.convert(VirtualColumn.SNAPSHOT_ID.getTypeInfo(), - relBuilder.getTypeFactory()).getSqlTypeName()); + typeFactory).getSqlTypeName()); } catch (CalciteSemanticException e) { throw new RuntimeException(e); } @@ -125,11 +128,17 @@ public void onMatch(RelOptRuleCall call) { Table table = hiveTable.getHiveTableMD(); SnapshotContext mvMetaTableSnapshot = mvMetaStoredSnapshot.get(table.getFullyQualifiedName()); - if (mvMetaTableSnapshot.equals(table.getStorageHandler().getCurrentSnapshotContext(table))) { + if (table.getStorageHandler() == null) { + throw new UnsupportedOperationException(String.format("Table %s does not have Storage handler defined. " + + "Mixing native and non-native tables in a materialized view definition is currently not supported!", + table.getFullyQualifiedName())); + } + if (Objects.equals(mvMetaTableSnapshot, table.getStorageHandler().getCurrentSnapshotContext(table))) { return; } - table.setVersionIntervalFrom(Long.toString(mvMetaTableSnapshot.getSnapshotId())); + Long snapshotId = mvMetaTableSnapshot != null ? mvMetaTableSnapshot.getSnapshotId() : null; + table.setVersionIntervalFrom(Objects.toString(snapshotId, null)); RexBuilder rexBuilder = call.builder().getRexBuilder(); int snapshotIdIndex = tableScan.getTable().getRowType().getField( @@ -139,14 +148,11 @@ public void onMatch(RelOptRuleCall call) { final RelBuilder relBuilder = call.builder(); relBuilder.push(tableScan); - List conds = new ArrayList<>(); - final RexNode literalHighWatermark = rexBuilder.makeLiteral( - mvMetaTableSnapshot.getSnapshotId(), snapshotIdType(relBuilder), false); - conds.add( - rexBuilder.makeCall( - SqlStdOperatorTable.LESS_THAN_OR_EQUAL, - ImmutableList.of(snapshotIdInputRef, literalHighWatermark))); - relBuilder.filter(conds); + final RexNode snapshotIdLiteral = rexBuilder.makeLiteral( + snapshotId, snapshotIdType(relBuilder.getTypeFactory()), false); + final RexNode predicateWithSnapShotId = rexBuilder.makeCall( + SqlStdOperatorTable.LESS_THAN_OR_EQUAL, snapshotIdInputRef, snapshotIdLiteral); + relBuilder.filter(singletonList(predicateWithSnapShotId)); call.transformTo(relBuilder.build()); } } \ No newline at end of file diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java index b00067f9fd26..8c612e48af28 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java @@ -16,6 +16,7 @@ */ package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import java.math.BigDecimal; @@ -79,7 +80,6 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.common.util.TxnIdUtils; -import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -284,8 +284,8 @@ public void visit(RelNode node, int ordinal, RelNode parent) { /** * Method to apply a rule to a query plan. */ - @NotNull - private static RelNode applyRule( + @VisibleForTesting + static RelNode applyRule( RelNode basePlan, RelOptRule relOptRule) { final HepProgramBuilder programBuilder = new HepProgramBuilder(); programBuilder.addRuleInstance(relOptRule); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HivePushdownSnapshotFilterRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HivePushdownSnapshotFilterRule.java index 2de1046d2190..055ff91d510b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HivePushdownSnapshotFilterRule.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HivePushdownSnapshotFilterRule.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; +import java.util.Objects; import java.util.Set; /** @@ -116,7 +117,7 @@ private boolean setSnapShotId(RexNode op1, RexNode op2) { return false; } - long snapshotId = literal.getValueAs(Long.class); + Long snapshotId = literal.getValueAs(Long.class); RelOptTable relOptTable = getRelOptTableOf(op2); if (relOptTable == null) { @@ -124,7 +125,7 @@ private boolean setSnapShotId(RexNode op1, RexNode op2) { } RelOptHiveTable hiveTable = (RelOptHiveTable) relOptTable; - hiveTable.getHiveTableMD().setVersionIntervalFrom(Long.toString(snapshotId)); + hiveTable.getHiveTableMD().setVersionIntervalFrom(Objects.toString(snapshotId, null)); return true; } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHiveAugmentSnapshotMaterializationRule.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHiveAugmentSnapshotMaterializationRule.java new file mode 100644 index 000000000000..746343d81459 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHiveAugmentSnapshotMaterializationRule.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.rel.RelNode; +import org.apache.hadoop.hive.common.type.SnapshotContext; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.mockito.Mockito.doReturn; + +@RunWith(MockitoJUnitRunner.class) +public class TestHiveAugmentSnapshotMaterializationRule extends TestRuleBase { + + @Test + public void testWhenSnapshotAndTableAreEmptyNoFilterAdded() { + RelNode tableScan = createTS(); + RelOptRule rule = HiveAugmentSnapshotMaterializationRule.with(Collections.emptyMap()); + + RelNode newRoot = HiveMaterializedViewUtils.applyRule(tableScan, rule); + + assertThat(newRoot, is(tableScan)); + } + + @Test + public void testWhenNoSnapshotButTableHasNewDataAFilterWithDefaultSnapshotIDAdded() { + doReturn(new SnapshotContext(42)).when(table2storageHandler).getCurrentSnapshotContext(table2); + RelNode tableScan = createTS(); + RelOptRule rule = HiveAugmentSnapshotMaterializationRule.with(Collections.emptyMap()); + + RelNode newRoot = HiveMaterializedViewUtils.applyRule(tableScan, rule); + + assertThat(newRoot, instanceOf(HiveFilter.class)); + HiveFilter filter = (HiveFilter) newRoot; + assertThat(filter.getCondition().toString(), is("<=($3, null)")); + } + + @Test + public void testWhenMVAndTableCurrentSnapshotAreTheSameNoFilterAdded() { + doReturn(new SnapshotContext(42)).when(table2storageHandler).getCurrentSnapshotContext(table2); + RelNode tableScan = createTS(); + Map mvSnapshot = new HashMap<>(); + mvSnapshot.put(table2.getFullyQualifiedName(), new SnapshotContext(42)); + RelOptRule rule = HiveAugmentSnapshotMaterializationRule.with(mvSnapshot); + + RelNode newRoot = HiveMaterializedViewUtils.applyRule(tableScan, rule); + + assertThat(newRoot, is(tableScan)); + } + + @Test + public void testWhenMVSnapshotIsDifferentThanTableCurrentSnapshotHasNewDataAFilterWithMVSnapshotIdAdded() { + doReturn(new SnapshotContext(10)).when(table2storageHandler).getCurrentSnapshotContext(table2); + RelNode tableScan = createTS(); + Map mvSnapshot = new HashMap<>(); + mvSnapshot.put(table2.getFullyQualifiedName(), new SnapshotContext(42)); + RelOptRule rule = HiveAugmentSnapshotMaterializationRule.with(mvSnapshot); + + RelNode newRoot = HiveMaterializedViewUtils.applyRule(tableScan, rule); + + assertThat(newRoot, instanceOf(HiveFilter.class)); + HiveFilter filter = (HiveFilter) newRoot; + assertThat(filter.getCondition().toString(), is("<=($3, 42)")); + } +} \ No newline at end of file diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHivePushdownSnapshotFilterRule.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHivePushdownSnapshotFilterRule.java new file mode 100644 index 000000000000..c60d83a8df4f --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestHivePushdownSnapshotFilterRule.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views; + +import org.apache.calcite.plan.RelOptSchema; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.RelBuilder; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; +import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; +import static org.hamcrest.core.IsInstanceOf.instanceOf; + +@RunWith(MockitoJUnitRunner.class) +public class TestHivePushdownSnapshotFilterRule extends TestRuleBase { + + @Mock + private RelOptSchema schemaMock; + + @Test + public void testFilterIsRemovedAndVersionIntervalFromIsSetWhenFilterHasSnapshotIdPredicate() { + RelNode tableScan = createTS(); + + RelBuilder relBuilder = HiveRelFactories.HIVE_BUILDER.create(relOptCluster, schemaMock); + RelNode root = relBuilder.push(tableScan) + .filter(REX_BUILDER.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, + REX_BUILDER.makeInputRef(HiveAugmentSnapshotMaterializationRule.snapshotIdType(TYPE_FACTORY), 3), + REX_BUILDER.makeLiteral(42, TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), false))) + .build(); + + System.out.println(RelOptUtil.toString(root)); + + RelNode newRoot = HiveMaterializedViewUtils.applyRule(root, HivePushdownSnapshotFilterRule.INSTANCE); + + assertThat(newRoot, instanceOf(HiveTableScan.class)); + HiveTableScan newScan = (HiveTableScan) newRoot; + RelOptHiveTable optHiveTable = (RelOptHiveTable) newScan.getTable(); + assertThat(optHiveTable.getHiveTableMD().getVersionIntervalFrom(), is("42")); + } + + @Test + public void testFilterLeftIntactWhenItDoesNotHaveSnapshotIdPredicate() { + RelNode tableScan = createTS(); + + RelBuilder relBuilder = HiveRelFactories.HIVE_BUILDER.create(relOptCluster, schemaMock); + RelNode root = relBuilder.push(tableScan) + .filter(REX_BUILDER.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, + REX_BUILDER.makeInputRef(TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), 1), + REX_BUILDER.makeLiteral(42, TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), false))) + .build(); + + System.out.println(RelOptUtil.toString(root)); + + RelNode newRoot = HiveMaterializedViewUtils.applyRule(root, HivePushdownSnapshotFilterRule.INSTANCE); + + assertThat(newRoot.getDigest(), is(root.getDigest())); + } +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestRuleBase.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestRuleBase.java new file mode 100644 index 000000000000..2d1d8133dd71 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/TestRuleBase.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views; + +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.VirtualColumn; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveTypeSystemImpl; +import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan; +import org.apache.hadoop.hive.ql.parse.CalcitePlanner; +import org.junit.Before; +import org.junit.BeforeClass; +import org.mockito.Mock; + +import java.util.List; + +import static java.util.Arrays.asList; +import static org.mockito.Mockito.doReturn; + +public class TestRuleBase { + protected static final RexBuilder REX_BUILDER = new RexBuilder(new JavaTypeFactoryImpl(new HiveTypeSystemImpl())); + protected static final RelDataTypeFactory TYPE_FACTORY = REX_BUILDER.getTypeFactory(); + + protected static RelOptCluster relOptCluster; + @Mock + protected RelOptHiveTable table2Mock; + protected static RelDataType table2Type; + protected static Table table2; + @Mock + protected static HiveStorageHandler table2storageHandler; + + @BeforeClass + public static void beforeClass() throws Exception { + RelOptPlanner planner = CalcitePlanner.createPlanner(new HiveConf()); + relOptCluster = RelOptCluster.create(planner, REX_BUILDER); + List t2Schema = asList( + TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), + TYPE_FACTORY.createSqlType(SqlTypeName.VARCHAR), + TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), + HiveAugmentSnapshotMaterializationRule.snapshotIdType(TYPE_FACTORY)); + table2Type = TYPE_FACTORY.createStructType(t2Schema, asList("d", "e", "f", VirtualColumn.SNAPSHOT_ID.getName())); + table2 = new Table(); + table2.setTTable(new org.apache.hadoop.hive.metastore.api.Table()); + table2.setDbName("default"); + table2.setTableName("t2"); + } + + @Before + public void setup() { + doReturn(table2Type).when(table2Mock).getRowType(); + doReturn(table2).when(table2Mock).getHiveTableMD(); + table2.setStorageHandler(table2storageHandler); + } + + protected HiveTableScan createTS() { + return new HiveTableScan(relOptCluster, relOptCluster.traitSetOf(HiveRelNode.CONVENTION), + table2Mock, "t2", null, false, false); + } +} From fc01fbc129a13a3892f25f3d6d63a47c6558378c Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Thu, 30 Nov 2023 15:44:54 +0530 Subject: [PATCH 073/179] HIVE-27903: Iceberg: Implement Expire Snapshot with default table properties. (#4906). (Ayush Saxena, reviewed by zhangbutao) --- .../mr/hive/HiveIcebergStorageHandler.java | 12 +++++++- .../hive/TestHiveIcebergExpireSnapshots.java | 29 +++++++++++++++++++ .../hadoop/hive/ql/parse/AlterClauseParser.g | 4 +-- .../execute/AlterTableExecuteAnalyzer.java | 5 +++- 4 files changed, 46 insertions(+), 4 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index c729fcef4300..9cfb210eec3d 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -894,7 +894,9 @@ private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnap LOG.info("Executing expire snapshots on iceberg table {} with {} threads", icebergTable.name(), numThreads); deleteExecutorService = getDeleteExecutorService(icebergTable.name(), numThreads); } - if (expireSnapshotsSpec.isExpireByTimestampRange()) { + if (expireSnapshotsSpec == null) { + expireSnapshotWithDefaultParams(icebergTable, deleteExecutorService); + } else if (expireSnapshotsSpec.isExpireByTimestampRange()) { expireSnapshotByTimestampRange(icebergTable, expireSnapshotsSpec.getFromTimestampMillis(), expireSnapshotsSpec.getTimestampMillis(), deleteExecutorService); } else if (expireSnapshotsSpec.isExpireByIds()) { @@ -911,6 +913,14 @@ private void expireSnapshot(Table icebergTable, AlterTableExecuteSpec.ExpireSnap } } + private void expireSnapshotWithDefaultParams(Table icebergTable, ExecutorService deleteExecutorService) { + ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots(); + if (deleteExecutorService != null) { + expireSnapshots.executeDeleteWith(deleteExecutorService); + } + expireSnapshots.commit(); + } + private void expireSnapshotRetainLast(Table icebergTable, int numRetainLast, ExecutorService deleteExecutorService) { ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots(); expireSnapshots.retainLast(numRetainLast); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java index 4a3b951bde40..9f036a5615a5 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java @@ -36,6 +36,7 @@ import org.junit.Test; import static org.apache.iceberg.TableProperties.MAX_SNAPSHOT_AGE_MS; +import static org.apache.iceberg.TableProperties.MIN_SNAPSHOTS_TO_KEEP; /** * Tests covering the rollback feature @@ -117,6 +118,34 @@ public void testExpireSnapshotsWithRetainLast() throws IOException, InterruptedE Assert.assertEquals(5, IterableUtils.size(table.snapshots())); } + @Test + public void testExpireSnapshotsWithDefaultParams() throws IOException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "source"); + Table table = testTables.createTableWithVersions(shell, identifier.name(), + HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 10); + // No snapshot should expire, since the max snapshot age to expire is by default 5 days + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE EXPIRE_SNAPSHOTS RETAIN LAST 5"); + table.refresh(); + Assert.assertEquals(10, IterableUtils.size(table.snapshots())); + + // Change max snapshot age to expire to 1 ms & min snapshots to keep as 3 & re-execute + shell.executeStatement( + "ALTER TABLE " + identifier.name() + " SET TBLPROPERTIES('" + MAX_SNAPSHOT_AGE_MS + "'='1'" + ",'" + + MIN_SNAPSHOTS_TO_KEEP + "'='3')"); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE EXPIRE_SNAPSHOTS"); + table.refresh(); + Assert.assertEquals(3, IterableUtils.size(table.snapshots())); + + // Change the min snapshot to keep as 2 + shell.executeStatement( + "ALTER TABLE " + identifier.name() + " SET TBLPROPERTIES('" + MIN_SNAPSHOTS_TO_KEEP + "'='2')"); + shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE EXPIRE_SNAPSHOTS"); + table.refresh(); + Assert.assertEquals(2, IterableUtils.size(table.snapshots())); + + } + @Test public void testDeleteOrphanFiles() throws IOException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "source"); diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g index 3e6105957c06..8e8ec4e33f99 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g @@ -475,8 +475,8 @@ alterStatementSuffixExecute @after { gParent.popMsg(state); } : KW_EXECUTE KW_ROLLBACK LPAREN (rollbackParam=(StringLiteral | Number)) RPAREN -> ^(TOK_ALTERTABLE_EXECUTE KW_ROLLBACK $rollbackParam) - | KW_EXECUTE KW_EXPIRE_SNAPSHOTS LPAREN (expireParam=StringLiteral) RPAREN - -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS $expireParam) + | KW_EXECUTE KW_EXPIRE_SNAPSHOTS (LPAREN (expireParam=StringLiteral) RPAREN)? + -> ^(TOK_ALTERTABLE_EXECUTE KW_EXPIRE_SNAPSHOTS $expireParam?) | KW_EXECUTE KW_SET_CURRENT_SNAPSHOT LPAREN (snapshotParam=expression) RPAREN -> ^(TOK_ALTERTABLE_EXECUTE KW_SET_CURRENT_SNAPSHOT $snapshotParam) | KW_EXECUTE KW_FAST_FORWARD sourceBranch=StringLiteral (targetBranch=StringLiteral)? diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java index 79448df3b2a5..275a0e1a4c56 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/execute/AlterTableExecuteAnalyzer.java @@ -141,7 +141,10 @@ private static AlterTableExecuteDesc getSetCurrentSnapshotDesc(TableName tableNa private static AlterTableExecuteDesc getExpireSnapshotDesc(TableName tableName, Map partitionSpec, List children) throws SemanticException { AlterTableExecuteSpec spec; - + if (children.size() == 1) { + spec = new AlterTableExecuteSpec(EXPIRE_SNAPSHOT, null); + return new AlterTableExecuteDesc(tableName, partitionSpec, spec); + } ZoneId timeZone = SessionState.get() == null ? new HiveConf().getLocalTimeZone() : SessionState.get().getConf().getLocalTimeZone(); From c247adb28b50452ad3dbc77d2f3ee803990376de Mon Sep 17 00:00:00 2001 From: cxzl25 Date: Thu, 30 Nov 2023 19:05:03 +0800 Subject: [PATCH 074/179] HIVE-27633: HMS: MTable to Table process reduces view related SQL (cxzl25, reviewed by Butao Zhang, Denys Kuzmenko) Closes #4616 --- .../apache/hadoop/hive/metastore/ObjectStore.java | 11 +++++++++-- .../client/TestTablesCreateDropAlterTruncate.java | 6 +----- .../hive/metastore/client/TestTablesGetExists.java | 13 +++++++++---- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 98f7f6f85be2..436ebd932acc 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -2303,6 +2303,8 @@ private Table convertToTable(MTable mtbl) throws MetaException { return null; } String tableType = mtbl.getTableType(); + String viewOriginalText = null; + String viewExpandedText = null; if (tableType == null) { // for backwards compatibility with old metastore persistence if (mtbl.getViewOriginalText() != null) { @@ -2312,14 +2314,19 @@ private Table convertToTable(MTable mtbl) throws MetaException { } else { tableType = TableType.MANAGED_TABLE.toString(); } + } else { + if (tableType.equals(TableType.VIRTUAL_VIEW.toString()) || tableType.equals(TableType.MATERIALIZED_VIEW.toString())) { + viewOriginalText = mtbl.getViewOriginalText(); + viewExpandedText = mtbl.getViewExpandedText(); + } } Map parameters = convertMap(mtbl.getParameters()); boolean isAcidTable = TxnUtils.isAcidTable(parameters); final Table t = new Table(mtbl.getTableName(), mtbl.getDatabase() != null ? mtbl.getDatabase().getName() : null, mtbl.getOwner(), mtbl.getCreateTime(), mtbl.getLastAccessTime(), mtbl.getRetention(), convertToStorageDescriptor(mtbl.getSd(), false, isAcidTable), - convertToFieldSchemas(mtbl.getPartitionKeys()), parameters, mtbl.getViewOriginalText(), - mtbl.getViewExpandedText(), tableType); + convertToFieldSchemas(mtbl.getPartitionKeys()), parameters, viewOriginalText, + viewExpandedText, tableType); if (Strings.isNullOrEmpty(mtbl.getOwnerType())) { // Before the ownerType exists in an old Hive schema, USER was the default type for owner. diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java index 8dc4d1dfcf09..0d38b628abb7 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java @@ -262,10 +262,6 @@ public void testCreateGetDeleteTable() throws Exception { createdTable.unsetId(); Assert.assertEquals("create/get table data", table, createdTable); - // Check that the directory is created - Assert.assertTrue("The directory should not be created", - metaStore.isPathExists(new Path(createdTable.getSd().getLocation()))); - client.dropTable(table.getDbName(), table.getTableName(), true, false); try { client.getTable(table.getDbName(), table.getTableName()); @@ -1595,7 +1591,7 @@ private Table getTableWithAllParametersSet() throws MetaException { .setNumBuckets(4) .setRetention(30000) .setRewriteEnabled(true) - .setType("VIEW") + .setType(TableType.VIRTUAL_VIEW.name()) .setViewExpandedText("viewExplainedText") .setViewOriginalText("viewOriginalText") .setSerdeLib("serdelib") diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java index f2937bc85760..84ac0f872102 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hive.metastore.ColumnType; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.MetaStoreTestUtils; +import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest; import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder; @@ -87,7 +88,7 @@ public void setUp() throws Exception { .setDbName(DEFAULT_DATABASE) .setTableName("test_view") .addCol("test_col", "int") - .setType("VIEW") + .setType(TableType.VIRTUAL_VIEW.name()) .create(client, metaStore.getConf()); testTables[2] = @@ -102,7 +103,7 @@ public void setUp() throws Exception { .setDbName(DEFAULT_DATABASE) .setTableName("test_table_to_find_2") .addCol("test_col", "int") - .setType("VIEW") + .setType(TableType.VIRTUAL_VIEW.name()) .create(client, metaStore.getConf()); testTables[4] = @@ -526,7 +527,7 @@ public void testGetTableObjectsWithProjectionOfMultipleField_2() throws Exceptio tableNames.add(testTables[1].getTableName()); GetProjectionsSpec projectSpec = (new GetTableProjectionsSpecBuilder()).includeOwner().includeOwnerType(). - includeSdLocation().build(); + includeSdLocation().includeTableType().build(); List
tables = client.getTables(null, DEFAULT_DATABASE, tableNames, projectSpec); @@ -540,7 +541,11 @@ public void testGetTableObjectsWithProjectionOfMultipleField_2() throws Exceptio Assert.assertTrue(table.isSetOwnerType()); Assert.assertTrue(table.isSetOwner()); StorageDescriptor sd = table.getSd(); - Assert.assertTrue(sd.isSetLocation()); + if (TableType.VIRTUAL_VIEW.toString().equals(table.getTableType())) { + Assert.assertFalse(sd.isSetLocation()); + } else { + Assert.assertTrue(sd.isSetLocation()); + } } } From 70f34e27349dccf5fabbfc6c63e63c7be0785360 Mon Sep 17 00:00:00 2001 From: seonggon Date: Thu, 30 Nov 2023 21:36:18 +0900 Subject: [PATCH 075/179] HIVE-27269: Create multiple MatchTracker when using VectorFastHashMapContainer (Seonggon Namgung, reviewed by Denys Kuzmenko) Closes #4857 --- .../VectorMapJoinFastHashTableLoader.java | 19 +- ...VectorMapJoinFastLongHashMapContainer.java | 21 +- ...orMapJoinFastMultiKeyHashMapContainer.java | 24 +- ...ctorMapJoinFastStringHashMapContainer.java | 24 +- ...apJoinFastStringHashMultiSetContainer.java | 2 +- ...MapJoinFastHashMapContainerNonMatched.java | 317 ++++++++++++++ ...louter_mapjoin_multithread_fast_htloader.q | 60 +++ ...er_mapjoin_multithread_fast_htloader.q.out | 394 ++++++++++++++++++ 8 files changed, 831 insertions(+), 30 deletions(-) create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/TestVectorMapJoinFastHashMapContainerNonMatched.java create mode 100644 ql/src/test/queries/clientpositive/vector_fullouter_mapjoin_multithread_fast_htloader.q create mode 100644 ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_multithread_fast_htloader.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java index 0a5a4f308d55..460f05b0e174 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java @@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAccumulator; +import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hive.common.Pool; import org.apache.hadoop.hive.llap.LlapDaemonInfo; @@ -106,9 +107,21 @@ public void init(ExecMapperContext context, MapredContext mrContext, } private void initHTLoadingService(long estKeyCount) { - // Avoid many small HTs that will rehash multiple times causing GCs - this.numLoadThreads = (estKeyCount < VectorMapJoinFastHashTable.FIRST_SIZE_UP) ? 1 : - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEMAPJOINPARALELHASHTABLETHREADS); + if (estKeyCount < VectorMapJoinFastHashTable.FIRST_SIZE_UP) { + // Avoid many small HTs that will rehash multiple times causing GCs + this.numLoadThreads = 1; + } else { + int initialValue = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEMAPJOINPARALELHASHTABLETHREADS); + Preconditions.checkArgument(initialValue > 0, "The number of HT-loading-threads should be positive."); + + int adjustedValue = Integer.highestOneBit(initialValue); + if (initialValue != adjustedValue) { + LOG.info("Adjust the number of HT-loading-threads to {}. (Previous value: {})", + adjustedValue, initialValue); + } + + this.numLoadThreads = adjustedValue; + } this.totalEntries = new LongAccumulator(Long::sum, 0L); this.loadExecService = Executors.newFixedThreadPool(numLoadThreads, new ThreadFactoryBuilder() diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashMapContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashMapContainer.java index 266fdd823b80..6ef9b64cba9d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashMapContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastLongHashMapContainer.java @@ -109,10 +109,11 @@ public static class NonMatchedLongHashMapIterator extends VectorMapJoinFastNonMa private NonMatchedLongHashMapIterator(MatchTracker matchTracker, VectorMapJoinFastLongHashMap[] vectorMapJoinFastLongHashMaps, int numThreads) { super(matchTracker); + hashMapIterators = new VectorMapJoinFastLongHashMap.NonMatchedLongHashMapIterator[numThreads]; for (int i = 0; i < numThreads; ++i) { - hashMapIterators[i] = new VectorMapJoinFastLongHashMap.NonMatchedLongHashMapIterator(matchTracker, - vectorMapJoinFastLongHashMaps[i]); + hashMapIterators[i] = new VectorMapJoinFastLongHashMap.NonMatchedLongHashMapIterator( + matchTracker.getPartition(i), vectorMapJoinFastLongHashMaps[i]); } index = 0; this.numThreads = numThreads; @@ -186,7 +187,11 @@ public JoinUtil.JoinResult lookup(long key, VectorMapJoinHashMapResult hashMapRe public JoinUtil.JoinResult lookup(long key, VectorMapJoinHashMapResult hashMapResult, MatchTracker matchTracker) { long hashCode = HashCodeUtil.calculateLongHashCode(key); - return vectorMapJoinFastLongHashMaps[(int) ((numThreads - 1) & hashCode)].lookup(key, hashMapResult, matchTracker); + int partition = (int) ((numThreads - 1) & hashCode); + MatchTracker childMatchTracker = matchTracker != null ? matchTracker.getPartition(partition) : null; + + return vectorMapJoinFastLongHashMaps[partition].lookup(key, hashMapResult, + childMatchTracker); } public long getEstimatedMemorySize() { @@ -208,11 +213,13 @@ public int size() { @Override public MatchTracker createMatchTracker() { - int count = 0; - for (int i = 0; i < numThreads; ++i) { - count += vectorMapJoinFastLongHashMaps[i].logicalHashBucketCount; + MatchTracker parentMatchTracker = MatchTracker.createPartitioned(numThreads); + for (int i = 0; i < numThreads; i++) { + int childSize = vectorMapJoinFastLongHashMaps[i].logicalHashBucketCount; + parentMatchTracker.addPartition(i, childSize); } - return MatchTracker.create(count); + + return parentMatchTracker; } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastMultiKeyHashMapContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastMultiKeyHashMapContainer.java index bb3bba504704..7a36d37c153b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastMultiKeyHashMapContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastMultiKeyHashMapContainer.java @@ -43,7 +43,6 @@ public class VectorMapJoinFastMultiKeyHashMapContainer private static final Logger LOG = LoggerFactory.getLogger(VectorMapJoinFastMultiKeyHashMapContainer.class); private final VectorMapJoinFastMultiKeyHashMap[] vectorMapJoinFastMultiKeyHashMaps; - private BytesWritable testKeyBytesWritable; private final int numThreads; public VectorMapJoinFastMultiKeyHashMapContainer( @@ -68,10 +67,10 @@ public static class NonMatchedBytesHashMapParallelIterator extends VectorMapJoin NonMatchedBytesHashMapParallelIterator(MatchTracker matchTracker, VectorMapJoinFastBytesHashMap[] hashMaps, int numThreads) { super(matchTracker); - hashMapIterators = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator[4]; + hashMapIterators = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator[numThreads]; for (int i = 0; i < numThreads; ++i) { - hashMapIterators[i] = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator(matchTracker, - hashMaps[i]); + hashMapIterators[i] = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator( + matchTracker.getPartition(i), hashMaps[i]); } index = 0; this.numThreads = numThreads; @@ -154,11 +153,13 @@ public int size() { @Override public MatchTracker createMatchTracker() { - int count = 0; - for (int i = 0; i < numThreads; ++i) { - count += vectorMapJoinFastMultiKeyHashMaps[i].logicalHashBucketCount; + MatchTracker parentMatchTracker = MatchTracker.createPartitioned(numThreads); + for (int i = 0; i < numThreads; i++) { + int childSize = vectorMapJoinFastMultiKeyHashMaps[i].logicalHashBucketCount; + parentMatchTracker.addPartition(i, childSize); } - return MatchTracker.create(count); + + return parentMatchTracker; } @Override @@ -182,8 +183,11 @@ public JoinUtil.JoinResult lookup(byte[] keyBytes, int keyStart, int keyLength, public JoinUtil.JoinResult lookup(byte[] keyBytes, int keyStart, int keyLength, VectorMapJoinHashMapResult hashMapResult, MatchTracker matchTracker) throws IOException { long hashCode = HashCodeUtil.murmurHash(keyBytes, keyStart, keyLength); - return vectorMapJoinFastMultiKeyHashMaps[(int) ((numThreads - 1) & hashCode)].lookup(keyBytes, keyStart, keyLength, hashMapResult, - matchTracker); + int partition = (int) ((numThreads - 1) & hashCode); + MatchTracker childMatchTracker = matchTracker != null ? matchTracker.getPartition(partition) : null; + + return vectorMapJoinFastMultiKeyHashMaps[partition].lookup(keyBytes, keyStart, keyLength, hashMapResult, + childMatchTracker); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMapContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMapContainer.java index ab7e3bf42ab8..e55b9aa236b3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMapContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMapContainer.java @@ -74,10 +74,11 @@ private static class NonMatchedBytesHashMapIterator extends VectorMapJoinFastNon NonMatchedBytesHashMapIterator(MatchTracker matchTracker, VectorMapJoinFastStringHashMap[] hashMaps, int numThreads) { super(matchTracker); - hashMapIterators = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator[4]; + + hashMapIterators = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator[numThreads]; for (int i = 0; i < numThreads; ++i) { - hashMapIterators[i] = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator(matchTracker, - hashMaps[i]); + hashMapIterators[i] = new VectorMapJoinFastBytesHashMap.NonMatchedBytesHashMapIterator( + matchTracker.getPartition(i), hashMaps[i]); } index = 0; this.numThreads = numThreads; @@ -172,11 +173,13 @@ public int size() { @Override public MatchTracker createMatchTracker() { - int count = 0; - for (int i = 0; i < numThreads; ++i) { - count += vectorMapJoinFastStringHashMaps[i].logicalHashBucketCount; + MatchTracker parentMatchTracker = MatchTracker.createPartitioned(numThreads); + for (int i = 0; i < numThreads; i++) { + int childSize = vectorMapJoinFastStringHashMaps[i].logicalHashBucketCount; + parentMatchTracker.addPartition(i, childSize); } - return MatchTracker.create(count); + + return parentMatchTracker; } @Override @@ -200,8 +203,11 @@ public JoinUtil.JoinResult lookup(byte[] keyBytes, int keyStart, int keyLength, public JoinUtil.JoinResult lookup(byte[] keyBytes, int keyStart, int keyLength, VectorMapJoinHashMapResult hashMapResult, MatchTracker matchTracker) throws IOException { long hashCode = HashCodeUtil.murmurHash(keyBytes, keyStart, keyLength); - return vectorMapJoinFastStringHashMaps[(int) ((numThreads - 1) & hashCode)].lookup(keyBytes, keyStart, keyLength, hashMapResult, - matchTracker); + int partition = (int) ((numThreads - 1) & hashCode); + MatchTracker childMatchTracker = matchTracker != null ? matchTracker.getPartition(partition) : null; + + return vectorMapJoinFastStringHashMaps[partition].lookup(keyBytes, keyStart, keyLength, hashMapResult, + childMatchTracker); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMultiSetContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMultiSetContainer.java index df4ac1eb7215..1755bc7c4726 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMultiSetContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastStringHashMultiSetContainer.java @@ -53,7 +53,7 @@ public VectorMapJoinFastStringHashMultiSetContainer( boolean isFullOuter, int initialCapacity, float loadFactor, int writeBuffersSize, long estimatedKeyCount, TableDesc tableDesc, int numHTs) { - vectorMapJoinFastStringHashMultiSets = new VectorMapJoinFastStringHashMultiSet[4]; + vectorMapJoinFastStringHashMultiSets = new VectorMapJoinFastStringHashMultiSet[numHTs]; LOG.info("Initializing {} HT Containers ", numHTs); for (int i = 0; i < numHTs; ++i) { vectorMapJoinFastStringHashMultiSets[i] = new VectorMapJoinFastStringHashMultiSet( diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/TestVectorMapJoinFastHashMapContainerNonMatched.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/TestVectorMapJoinFastHashMapContainerNonMatched.java new file mode 100644 index 000000000000..e857cafe2348 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/TestVectorMapJoinFastHashMapContainerNonMatched.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast; + +import org.apache.hadoop.hive.ql.exec.persistence.MatchTracker; +import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource; +import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashMapResult; +import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinNonMatchedIterator; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc; +import org.apache.hadoop.hive.serde2.ByteStream; +import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hive.common.util.HashCodeUtil; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Properties; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestVectorMapJoinFastHashMapContainerNonMatched { + private static final int numHashTable = 2; + private static final int initialCapacity = 8; + private static final float loadFactor = 0.9f; + private static final int writeBufferSize = 1024 * 1024; + private static final int estimatedKeyCount = -1; + + private BytesWritable serializeLong(long value, Properties properties) throws Exception { + BinarySortableSerializeWrite serializeWrite = BinarySortableSerializeWrite.with(properties, 1); + ByteStream.Output output = new ByteStream.Output(); + serializeWrite.set(output); + serializeWrite.writeLong(value); + + BytesWritable writable = new BytesWritable(); + writable.set(output.getData(), 0, output.getLength()); + + return writable; + } + + private void addToHashMap(VectorMapJoinFastLongHashMapContainer hashMap, long value, Properties properties) + throws Exception { + BytesWritable keyWritable = serializeLong(value, properties); + BytesWritable valueWritable = new BytesWritable(keyWritable.copyBytes()); + hashMap.putRow(HashCodeUtil.calculateLongHashCode(value), keyWritable, valueWritable); + } + + private long getHashCode(String key) { + Text keyWritable = new Text(key); + return HashCodeUtil.murmurHash(keyWritable.getBytes(), 0, keyWritable.getLength()); + } + + private BytesWritable serializeString(String value, Properties properties) throws Exception { + BinarySortableSerializeWrite serializeWrite = BinarySortableSerializeWrite.with(properties, 1); + ByteStream.Output output = new ByteStream.Output(); + serializeWrite.set(output); + + Text text = new Text(value); + serializeWrite.writeString(text.getBytes(), 0, text.getLength()); + + BytesWritable writable = new BytesWritable(); + writable.set(output.getData(), 0, output.getLength()); + + return writable; + } + + private void addToHashMap( + VectorMapJoinFastStringHashMapContainer hashMap, String value, Properties properties) throws Exception { + BytesWritable keyWritable = serializeString(value, properties); + BytesWritable valueWritable = new BytesWritable(keyWritable.copyBytes()); + hashMap.putRow(getHashCode(value), keyWritable, valueWritable); + } + + private BytesWritable createRandomMultiKey(Random random, BinarySortableSerializeWrite serializeWrite) + throws Exception { + ByteStream.Output output = new ByteStream.Output(); + serializeWrite.set(output); + + serializeWrite.writeLong(random.nextLong()); + serializeWrite.writeLong(random.nextLong()); + + BytesWritable writable = new BytesWritable(); + writable.set(output.getData(), 0, output.getLength()); + + return writable; + } + + private long getHashCode(BytesWritable key) { + return HashCodeUtil.murmurHash(key.getBytes(), 0, key.getLength()); + } + + private void addToHashMap( + VectorMapJoinFastMultiKeyHashMapContainer hashMap, BytesWritable key) throws Exception { + BytesWritable value = new BytesWritable(key.copyBytes()); + hashMap.putRow(getHashCode(key), key, value); + } + + @Test + public void testLongHashMapContainer() throws Exception { + Random random = new Random(); + long keyA = random.nextLong(); + while ((HashCodeUtil.calculateLongHashCode(keyA) & (initialCapacity - 1)) != 0) { + keyA = random.nextLong(); + } + + long keyB = random.nextLong(); + while ((HashCodeUtil.calculateLongHashCode(keyB) & (initialCapacity - 1)) != 0 || keyB == keyA) { + keyB = random.nextLong(); + } + + long keyC = random.nextLong(); + while ((HashCodeUtil.calculateLongHashCode(keyC) & (initialCapacity - 1)) != 1) { + keyC = random.nextLong(); + } + + TableDesc tableDesc = new TableDesc(); + Properties properties = new Properties(); + tableDesc.setProperties(properties); + + VectorMapJoinFastLongHashMapContainer hashMapContainer = + new VectorMapJoinFastLongHashMapContainer( + true, /* isFullOuter */ + false, /* minMaxEnabled */ + VectorMapJoinDesc.HashTableKeyType.LONG, + initialCapacity, + loadFactor, + writeBufferSize, + estimatedKeyCount, + tableDesc, + numHashTable); + + addToHashMap(hashMapContainer, keyA, properties); + addToHashMap(hashMapContainer, keyB, properties); + addToHashMap(hashMapContainer, keyC, properties); + + MatchTracker matchTracker = hashMapContainer.createMatchTracker(); + VectorMapJoinHashMapResult hashMapResult = hashMapContainer.createHashMapResult(); + + hashMapContainer.lookup(keyB, hashMapResult, matchTracker); + + VectorMapJoinNonMatchedIterator nonMatchedIterator = + hashMapContainer.createNonMatchedIterator(matchTracker); + nonMatchedIterator.init(); + + ArrayList nonMatchedList = new ArrayList(); + while (nonMatchedIterator.findNextNonMatched()) { + boolean isNull = !nonMatchedIterator.readNonMatchedLongKey(); + assertFalse(isNull); + + long key = nonMatchedIterator.getNonMatchedLongKey(); + nonMatchedList.add(key); + } + + assertEquals(2, nonMatchedList.size()); + assertTrue(nonMatchedList.contains(keyA)); + assertTrue(nonMatchedList.contains(keyC)); + } + + @Test + public void testStringHashMapContainer() throws Exception { + Random random = new Random(); + + String keyA = VectorRandomRowSource.getRandString(random, 5, false); + while ((getHashCode(keyA) & (initialCapacity - 1)) != 0) { + keyA = VectorRandomRowSource.getRandString(random, 5, false); + } + + String keyB = VectorRandomRowSource.getRandString(random, 5, false); + while ((getHashCode(keyB) & (initialCapacity - 1)) != 0 || keyB.equals(keyA)) { + keyB = VectorRandomRowSource.getRandString(random, 5, false); + } + + String keyC = VectorRandomRowSource.getRandString(random, 5, false); + while ((getHashCode(keyC) & (initialCapacity - 1)) != 1) { + keyC = VectorRandomRowSource.getRandString(random, 5, false); + } + + TableDesc tableDesc = new TableDesc(); + Properties properties = new Properties(); + tableDesc.setProperties(properties); + + VectorMapJoinFastStringHashMapContainer hashMapContainer = + new VectorMapJoinFastStringHashMapContainer( + true, /* isFullOuter */ + initialCapacity, + loadFactor, + writeBufferSize, + estimatedKeyCount, + tableDesc, + numHashTable); + + addToHashMap(hashMapContainer, keyA, properties); + addToHashMap(hashMapContainer, keyB, properties); + addToHashMap(hashMapContainer, keyC, properties); + + MatchTracker matchTracker = hashMapContainer.createMatchTracker(); + VectorMapJoinHashMapResult hashMapResult = hashMapContainer.createHashMapResult(); + + Text keyTextB = new Text(keyB); + hashMapContainer.lookup(keyTextB.getBytes(), 0, keyTextB.getLength(), hashMapResult, matchTracker); + + VectorMapJoinNonMatchedIterator nonMatchedIterator = + hashMapContainer.createNonMatchedIterator(matchTracker); + nonMatchedIterator.init(); + + ArrayList nonMatchedList = new ArrayList(); + while (nonMatchedIterator.findNextNonMatched()) { + boolean isNull = !nonMatchedIterator.readNonMatchedBytesKey(); + assertFalse(isNull); + + byte[] keyBytes = nonMatchedIterator.getNonMatchedBytes(); + int keyOffset = nonMatchedIterator.getNonMatchedBytesOffset(); + int keyLength = nonMatchedIterator.getNonMatchedBytesLength(); + + byte[] array = new byte[keyLength]; + System.arraycopy(keyBytes, keyOffset, array, 0, keyLength); + Text key = new Text(array); + + nonMatchedList.add(key.toString()); + } + + assertEquals(2, nonMatchedList.size()); + assertTrue(nonMatchedList.contains(keyA)); + assertTrue(nonMatchedList.contains(keyC)); + } + + @Test + public void testMultiKeyHashMapContainer() throws Exception { + Random random = new Random(); + BinarySortableSerializeWrite serializeWrite = + BinarySortableSerializeWrite.with(new Properties(), 2); + + BytesWritable keyA = createRandomMultiKey(random, serializeWrite); + while ((getHashCode(keyA) & (initialCapacity - 1)) != 0) { + keyA = createRandomMultiKey(random, serializeWrite); + } + + BytesWritable keyB = createRandomMultiKey(random, serializeWrite); + while ((getHashCode(keyB) & (initialCapacity - 1)) != 0 || keyB == keyA) { + keyB = createRandomMultiKey(random, serializeWrite); + } + + BytesWritable keyC = createRandomMultiKey(random, serializeWrite); + while ((getHashCode(keyC) & (initialCapacity - 1)) != 1) { + keyC = createRandomMultiKey(random, serializeWrite); + } + + VectorMapJoinFastMultiKeyHashMapContainer hashMapContainer = + new VectorMapJoinFastMultiKeyHashMapContainer( + true, /* isFullOuter */ + initialCapacity, + loadFactor, + writeBufferSize, + estimatedKeyCount, + numHashTable); + + addToHashMap(hashMapContainer, keyA); + addToHashMap(hashMapContainer, keyB); + addToHashMap(hashMapContainer, keyC); + + MatchTracker matchTracker = hashMapContainer.createMatchTracker(); + VectorMapJoinHashMapResult hashMapResult = hashMapContainer.createHashMapResult(); + + hashMapContainer.lookup(keyB.getBytes(), 0, keyB.getLength(), hashMapResult, matchTracker); + + VectorMapJoinNonMatchedIterator nonMatchedIterator = + hashMapContainer.createNonMatchedIterator(matchTracker); + nonMatchedIterator.init(); + + ArrayList nonMatchedList = new ArrayList(); + while (nonMatchedIterator.findNextNonMatched()) { + boolean isNull = !nonMatchedIterator.readNonMatchedBytesKey(); + assertFalse(isNull); + + byte[] keyBytes = nonMatchedIterator.getNonMatchedBytes(); + int keyOffset = nonMatchedIterator.getNonMatchedBytesOffset(); + int keyLength = nonMatchedIterator.getNonMatchedBytesLength(); + + byte[] array = new byte[keyLength]; + System.arraycopy(keyBytes, keyOffset, array, 0, keyLength); + + nonMatchedList.add(array); + } + + final BytesWritable finalKeyA = keyA; + final BytesWritable finalKeyC = keyC; + + assertEquals(2, nonMatchedList.size()); + assertTrue(nonMatchedList.stream().anyMatch(arr -> { + return Arrays.equals(arr, finalKeyA.copyBytes()); + })); + assertTrue(nonMatchedList.stream().anyMatch(arr -> { + return Arrays.equals(arr, finalKeyC.copyBytes()); + })); } +} + diff --git a/ql/src/test/queries/clientpositive/vector_fullouter_mapjoin_multithread_fast_htloader.q b/ql/src/test/queries/clientpositive/vector_fullouter_mapjoin_multithread_fast_htloader.q new file mode 100644 index 000000000000..b7b5745d302b --- /dev/null +++ b/ql/src/test/queries/clientpositive/vector_fullouter_mapjoin_multithread_fast_htloader.q @@ -0,0 +1,60 @@ +set hive.auto.convert.join=true; +set hive.optimize.dynamic.partition.hashjoin=true; +set hive.auto.convert.join.noconditionaltask.size=100000000000; +set hive.exec.reducers.max=1; +set hive.vectorized.execution.mapjoin.native.fast.hashtable.enabled=true; +set hive.mapjoin.hashtable.load.threads=2; + +-- SORT_QUERY_RESULTS + +-- +-- test Long key +-- + +-- This table should be broadcasted and stored in HashTable. +create table small_long_table (key bigint, value string); +insert into small_long_table values (7610878409923211200, "a"); -- key hash % 2097152 == 0 +insert into small_long_table values (-371494529663898262, "b"); -- key hash % 2097152 == 0 +insert into small_long_table values (-2307888158465848362, "c"); -- key hash % 2097152 == 1 + +create table big_long_table (key bigint, value string); +insert into big_long_table values (-2307888158465848362, "c"); -- key hash % 2097152 == 1 + +-- small table size should be larger than VectorMapJoinFastHashTable.FIRST_SIZE_UP. +-- If not, only a single thread loads entire hash table. +alter table big_long_table update statistics set ('numRows'='90000000'); -- should be larger than small table +alter table small_long_table update statistics set ('numRows'='2097152'); -- 2 * VectorMapJoinFastHashTable.FIRST_SIZE_UP + +-- query plan must includes vectorized fullouter mapjoin. +explain +select * from small_long_table full outer join big_long_table on (small_long_table.key = big_long_table.key); + +select * from small_long_table full outer join big_long_table on (small_long_table.key = big_long_table.key); + +-- +-- test String key +-- + +-- This table should be broadcasted and stored in HashTable. +create table small_string_table (key string, value string); +insert into small_string_table values ("affzk", "a"); -- key hash % 2097152 == 0 +insert into small_string_table values ("hbkpa", "b"); -- key hash % 2097152 == 0 +insert into small_string_table values ("kykzm", "c"); -- key hash % 2097152 == 1 + +create table big_string_table (key string, value string); +insert into big_string_table values ("kykzm", "c"); -- key hash % 2097152 == 1 + +-- small table size should be larger than VectorMapJoinFastHashTable.FIRST_SIZE_UP. +-- If not, only a single thread loads entire hash table. +alter table big_string_table update statistics set ('numRows'='90000000'); -- should be larger than small table +alter table small_string_table update statistics set ('numRows'='2097152'); -- 2 * VectorMapJoinFastHashTable.FIRST_SIZE_UP + +-- query plan must includes vectorized fullouter mapjoin. +explain +select * from small_string_table full outer join big_string_table on (small_string_table.key = big_string_table.key); + +select * from small_string_table full outer join big_string_table on (small_string_table.key = big_string_table.key); + +-- To test multikey HashTable, one may use the following configuration. +-- set hive.vectorized.execution.mapjoin.native.multikey.only.enabled=true; + diff --git a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_multithread_fast_htloader.q.out b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_multithread_fast_htloader.q.out new file mode 100644 index 000000000000..a9e5e419ff95 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_multithread_fast_htloader.q.out @@ -0,0 +1,394 @@ +PREHOOK: query: create table small_long_table (key bigint, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@small_long_table +POSTHOOK: query: create table small_long_table (key bigint, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@small_long_table +PREHOOK: query: insert into small_long_table values (7610878409923211200, "a") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@small_long_table +POSTHOOK: query: insert into small_long_table values (7610878409923211200, "a") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@small_long_table +POSTHOOK: Lineage: small_long_table.key SCRIPT [] +POSTHOOK: Lineage: small_long_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 0 +insert into small_long_table values (-371494529663898262, "b") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@small_long_table +POSTHOOK: query: -- key hash % 2097152 == 0 +insert into small_long_table values (-371494529663898262, "b") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@small_long_table +POSTHOOK: Lineage: small_long_table.key SCRIPT [] +POSTHOOK: Lineage: small_long_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 0 +insert into small_long_table values (-2307888158465848362, "c") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@small_long_table +POSTHOOK: query: -- key hash % 2097152 == 0 +insert into small_long_table values (-2307888158465848362, "c") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@small_long_table +POSTHOOK: Lineage: small_long_table.key SCRIPT [] +POSTHOOK: Lineage: small_long_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 1 + +create table big_long_table (key bigint, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@big_long_table +POSTHOOK: query: -- key hash % 2097152 == 1 + +create table big_long_table (key bigint, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@big_long_table +PREHOOK: query: insert into big_long_table values (-2307888158465848362, "c") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@big_long_table +POSTHOOK: query: insert into big_long_table values (-2307888158465848362, "c") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@big_long_table +POSTHOOK: Lineage: big_long_table.key SCRIPT [] +POSTHOOK: Lineage: big_long_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 1 + + + +alter table big_long_table update statistics set ('numRows'='90000000') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@big_long_table +PREHOOK: Output: default@big_long_table +POSTHOOK: query: -- key hash % 2097152 == 1 + + + +alter table big_long_table update statistics set ('numRows'='90000000') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@big_long_table +POSTHOOK: Output: default@big_long_table +PREHOOK: query: -- should be larger than small table +alter table small_long_table update statistics set ('numRows'='2097152') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@small_long_table +PREHOOK: Output: default@small_long_table +POSTHOOK: query: -- should be larger than small table +alter table small_long_table update statistics set ('numRows'='2097152') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@small_long_table +POSTHOOK: Output: default@small_long_table +PREHOOK: query: -- 2 * VectorMapJoinFastHashTable.FIRST_SIZE_UP + + +explain +select * from small_long_table full outer join big_long_table on (small_long_table.key = big_long_table.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@big_long_table +PREHOOK: Input: default@small_long_table +#### A masked pattern was here #### +POSTHOOK: query: -- 2 * VectorMapJoinFastHashTable.FIRST_SIZE_UP + + +explain +select * from small_long_table full outer join big_long_table on (small_long_table.key = big_long_table.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@big_long_table +POSTHOOK: Input: default@small_long_table +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 2 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: small_long_table + Statistics: Num rows: 2097152 Data size: 195035136 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: bigint), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2097152 Data size: 195035136 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 2097152 Data size: 195035136 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), 0S (type: smallint) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 2 + Map Operator Tree: + TableScan + alias: big_long_table + Statistics: Num rows: 90000000 Data size: 8370000000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: bigint), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 90000000 Data size: 8370000000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 90000000 Data size: 8370000000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Map Join Operator + condition map: + Full Outer Join 0 to 1 + keys: + 0 KEY.reducesinkkey0 (type: bigint) + 1 KEY.reducesinkkey0 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + input vertices: + 0 Map 1 + Statistics: Num rows: 62914652097152 Data size: 11702108160000186 Basic stats: COMPLETE Column stats: COMPLETE + DynamicPartitionHashJoin: true + File Output Operator + compressed: false + Statistics: Num rows: 62914652097152 Data size: 11702108160000186 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from small_long_table full outer join big_long_table on (small_long_table.key = big_long_table.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@big_long_table +PREHOOK: Input: default@small_long_table +#### A masked pattern was here #### +POSTHOOK: query: select * from small_long_table full outer join big_long_table on (small_long_table.key = big_long_table.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@big_long_table +POSTHOOK: Input: default@small_long_table +#### A masked pattern was here #### +-2307888158465848362 c -2307888158465848362 c +-371494529663898262 b NULL NULL +7610878409923211200 a NULL NULL +PREHOOK: query: create table small_string_table (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@small_string_table +POSTHOOK: query: create table small_string_table (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@small_string_table +PREHOOK: query: insert into small_string_table values ("affzk", "a") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@small_string_table +POSTHOOK: query: insert into small_string_table values ("affzk", "a") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@small_string_table +POSTHOOK: Lineage: small_string_table.key SCRIPT [] +POSTHOOK: Lineage: small_string_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 0 +insert into small_string_table values ("hbkpa", "b") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@small_string_table +POSTHOOK: query: -- key hash % 2097152 == 0 +insert into small_string_table values ("hbkpa", "b") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@small_string_table +POSTHOOK: Lineage: small_string_table.key SCRIPT [] +POSTHOOK: Lineage: small_string_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 0 +insert into small_string_table values ("kykzm", "c") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@small_string_table +POSTHOOK: query: -- key hash % 2097152 == 0 +insert into small_string_table values ("kykzm", "c") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@small_string_table +POSTHOOK: Lineage: small_string_table.key SCRIPT [] +POSTHOOK: Lineage: small_string_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 1 + +create table big_string_table (key string, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@big_string_table +POSTHOOK: query: -- key hash % 2097152 == 1 + +create table big_string_table (key string, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@big_string_table +PREHOOK: query: insert into big_string_table values ("kykzm", "c") +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@big_string_table +POSTHOOK: query: insert into big_string_table values ("kykzm", "c") +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@big_string_table +POSTHOOK: Lineage: big_string_table.key SCRIPT [] +POSTHOOK: Lineage: big_string_table.value SCRIPT [] +PREHOOK: query: -- key hash % 2097152 == 1 + + + +alter table big_string_table update statistics set ('numRows'='90000000') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@big_string_table +PREHOOK: Output: default@big_string_table +POSTHOOK: query: -- key hash % 2097152 == 1 + + + +alter table big_string_table update statistics set ('numRows'='90000000') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@big_string_table +POSTHOOK: Output: default@big_string_table +PREHOOK: query: -- should be larger than small table +alter table small_string_table update statistics set ('numRows'='2097152') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@small_string_table +PREHOOK: Output: default@small_string_table +POSTHOOK: query: -- should be larger than small table +alter table small_string_table update statistics set ('numRows'='2097152') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@small_string_table +POSTHOOK: Output: default@small_string_table +PREHOOK: query: -- 2 * VectorMapJoinFastHashTable.FIRST_SIZE_UP + + +explain +select * from small_string_table full outer join big_string_table on (small_string_table.key = big_string_table.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@big_string_table +PREHOOK: Input: default@small_string_table +#### A masked pattern was here #### +POSTHOOK: query: -- 2 * VectorMapJoinFastHashTable.FIRST_SIZE_UP + + +explain +select * from small_string_table full outer join big_string_table on (small_string_table.key = big_string_table.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@big_string_table +POSTHOOK: Input: default@small_string_table +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 2 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: small_string_table + Statistics: Num rows: 2097152 Data size: 364904448 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2097152 Data size: 364904448 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 2097152 Data size: 364904448 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), 0S (type: smallint) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 2 + Map Operator Tree: + TableScan + alias: big_string_table + Statistics: Num rows: 90000000 Data size: 15660000000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 90000000 Data size: 15660000000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 90000000 Data size: 15660000000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Map Join Operator + condition map: + Full Outer Join 0 to 1 + keys: + 0 KEY.reducesinkkey0 (type: string) + 1 KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + input vertices: + 0 Map 1 + Statistics: Num rows: 62914652097152 Data size: 21894266880000348 Basic stats: COMPLETE Column stats: COMPLETE + DynamicPartitionHashJoin: true + File Output Operator + compressed: false + Statistics: Num rows: 62914652097152 Data size: 21894266880000348 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from small_string_table full outer join big_string_table on (small_string_table.key = big_string_table.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@big_string_table +PREHOOK: Input: default@small_string_table +#### A masked pattern was here #### +POSTHOOK: query: select * from small_string_table full outer join big_string_table on (small_string_table.key = big_string_table.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@big_string_table +POSTHOOK: Input: default@small_string_table +#### A masked pattern was here #### +affzk a NULL NULL +hbkpa b NULL NULL +kykzm c kykzm c From 33903b825bb949d2a4c3839688817ac5ebe3fbdb Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Fri, 1 Dec 2023 20:38:52 +0530 Subject: [PATCH 076/179] HIVE-27714: Iceberg: metadata location overrides can cause data breach - handling default locations. (#4880). (Ayush Saxena, reviewed by Denys Kuzmenko) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 4 + .../mr/hive/HiveIcebergStorageHandler.java | 48 ++++++++-- .../TestHiveIcebergStorageHandlerNoScan.java | 88 +++++++++++++++++-- .../hive/ql/metadata/HiveStorageHandler.java | 1 + .../hive/ql/parse/SemanticAnalyzer.java | 25 ++++-- .../api/hive_metastoreConstants.java | 2 + 6 files changed, 146 insertions(+), 22 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 027bab6eb53d..ad807386f360 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2226,6 +2226,10 @@ public static enum ConfVars { HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS("hive.iceberg.expire.snapshot.numthreads", 4, "The number of threads to be used for deleting files during expire snapshot. If set to 0 or below it uses the" + " defult DirectExecutorService"), + + HIVE_ICEBERG_MASK_DEFAULT_LOCATION("hive.iceberg.mask.default.location", false, + "If this is set to true the URI for auth will have the default location masked with DEFAULT_TABLE_LOCATION"), + HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true, "If this is set the header for RCFiles will simply be RCF. If this is not\n" + "set the header will be that borrowed from sequence files, e.g. SEQ- followed\n" + diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 9cfb210eec3d..5f4f97b9f72b 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -48,6 +48,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.common.type.Date; import org.apache.hadoop.hive.common.type.SnapshotContext; @@ -214,6 +215,8 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H public static final String MERGE_ON_READ = "merge-on-read"; public static final String STATS = "/stats/snap-"; + public static final String TABLE_DEFAULT_LOCATION = "TABLE_DEFAULT_LOCATION"; + /** * Function template for producing a custom sort expression function: * Takes the source column index and the bucket count to creat a function where Iceberg bucket UDF is used to build @@ -520,11 +523,11 @@ private boolean writeColStats(ColumnStatistics tableColStats, Table tbl, String writer.finish(); return true; } catch (IOException e) { - LOG.warn("Unable to write stats to puffin file", e.getMessage()); + LOG.warn("Unable to write stats to puffin file {}", e.getMessage()); return false; } } catch (InvalidObjectException | IOException e) { - LOG.warn("Unable to invalidate or merge stats: ", e.getMessage()); + LOG.warn("Unable to invalidate or merge stats: {}", e.getMessage()); return false; } } @@ -1053,19 +1056,19 @@ public URI getURIForAuth(org.apache.hadoop.hive.metastore.api.Table hmsTable) th Optional metadataLocation = SessionStateUtil.getProperty(conf, BaseMetastoreTableOperations.METADATA_LOCATION_PROP); if (metadataLocation.isPresent()) { - authURI.append(encodeString(metadataLocation.get())); + authURI.append(getPathForAuth(metadataLocation.get())); } else { Optional locationProperty = SessionStateUtil.getProperty(conf, hive_metastoreConstants.META_TABLE_LOCATION); if (locationProperty.isPresent()) { // this property is set during the create operation before the hive table was created // we are returning a dummy iceberg metadata file - authURI.append(encodeString(URI.create(locationProperty.get()).getPath())) + authURI.append(getPathForAuth(locationProperty.get())) .append(encodeString("/metadata/dummy.metadata.json")); } else { Table table = IcebergTableUtil.getTable(conf, hmsTable); - authURI.append( - encodeString(URI.create(((BaseTable) table).operations().current().metadataFileLocation()).getPath())); + authURI.append(getPathForAuth(((BaseTable) table).operations().current().metadataFileLocation(), + hmsTable.getSd().getLocation())); } } LOG.debug("Iceberg storage handler authorization URI {}", authURI); @@ -1080,6 +1083,39 @@ static String encodeString(String rawString) { return HiveConf.EncoderDecoderFactory.URL_ENCODER_DECODER.encode(rawString); } + String getPathForAuth(String locationProperty) { + return getPathForAuth(locationProperty, + SessionStateUtil.getProperty(conf, hive_metastoreConstants.DEFAULT_TABLE_LOCATION).orElse(null)); + } + + String getPathForAuth(String locationProperty, String defaultTableLocation) { + boolean maskDefaultLocation = conf.getBoolean(HiveConf.ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, + HiveConf.ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION.defaultBoolVal); + String location = URI.create(locationProperty).getPath(); + if (!maskDefaultLocation || defaultTableLocation == null || + !arePathsInSameFs(locationProperty, defaultTableLocation)) { + return encodeString(location); + } + try { + Path locationPath = new Path(location); + Path defaultLocationPath = locationPath.toUri().getScheme() != null ? + FileUtils.makeQualified(new Path(defaultTableLocation), conf) : + Path.getPathWithoutSchemeAndAuthority(new Path(defaultTableLocation)); + return encodeString(location.replaceFirst(defaultLocationPath.toString(), TABLE_DEFAULT_LOCATION)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private boolean arePathsInSameFs(String locationProperty, String defaultTableLocation) { + try { + return FileUtils.equalsFileSystem(new Path(locationProperty).getFileSystem(conf), + new Path(defaultTableLocation).getFileSystem(conf)); + } catch (IOException e) { + LOG.debug("Unable to get FileSystem for path {} and {}", locationProperty, defaultTableLocation); + return false; + } + } @Override public void validateSinkDesc(FileSinkDesc sinkDesc) throws SemanticException { diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 696acd8903b7..48da53e2b685 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -95,6 +95,7 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mockito; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION; import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.junit.runners.Parameterized.Parameter; @@ -1484,31 +1485,59 @@ public void testCommandsWithPartitionClauseThrow() { } @Test - public void testAuthzURI() throws TException, InterruptedException, URISyntaxException { + public void testAuthzURIMasked() throws TException, URISyntaxException, InterruptedException { + testAuthzURI(true); + } + + @Test + public void testAuthzURIUnmasked() throws TException, URISyntaxException, InterruptedException { + testAuthzURI(false); + } + + public void testAuthzURI(boolean masked) throws TException, InterruptedException, URISyntaxException { TableIdentifier target = TableIdentifier.of("default", "target"); Table table = testTables.createTable(shell, target.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), FileFormat.PARQUET, ImmutableList.of()); org.apache.hadoop.hive.metastore.api.Table hmsTable = shell.metastore().getTable(target); HiveIcebergStorageHandler storageHandler = new HiveIcebergStorageHandler(); + shell.getHiveConf().setBoolean(HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, masked); storageHandler.setConf(shell.getHiveConf()); URI uriForAuth = storageHandler.getURIForAuth(hmsTable); + String metadataLocation = + storageHandler.getPathForAuth(((BaseTable) table).operations().current().metadataFileLocation(), + hmsTable.getSd().getLocation()); + + if (masked) { + Assert.assertTrue(metadataLocation.startsWith(HiveIcebergStorageHandler.TABLE_DEFAULT_LOCATION)); + } + Assert.assertEquals("iceberg://" + HiveIcebergStorageHandler.encodeString(target.namespace().toString()) + "/" + HiveIcebergStorageHandler.encodeString(target.name()) + "?snapshot=" + HiveIcebergStorageHandler.encodeString( - URI.create(((BaseTable) table).operations().current().metadataFileLocation()).getPath()), + URI.create(metadataLocation).getPath()), uriForAuth.toString()); Assert.assertEquals("iceberg://" + target.namespace() + "/" + target.name() + "?snapshot=" + - URI.create(((BaseTable) table).operations().current().metadataFileLocation()).getPath(), + URI.create(metadataLocation).getPath(), HiveConf.EncoderDecoderFactory.URL_ENCODER_DECODER.decode(uriForAuth.toString())); } @Test - public void testAuthzURIWithAuthEnabledWithMetadataLocation() throws HiveException { + public void testAuthzURIWithAuthEnabledWithMetadataLocationMasked() throws HiveException { + testAuthzURIWithAuthEnabledWithMetadataLocation(true); + } + + @Test + public void testAuthzURIWithAuthEnabledWithMetadataLocationUnmasked() throws HiveException { + testAuthzURIWithAuthEnabledWithMetadataLocation(false); + } + + public void testAuthzURIWithAuthEnabledWithMetadataLocation(boolean masked) throws HiveException { + shell.getHiveConf().setBoolean(HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, masked); shell.setHiveSessionValue("hive.security.authorization.enabled", true); shell.setHiveSessionValue("hive.security.authorization.manager", "org.apache.iceberg.mr.hive.CustomTestHiveAuthorizerFactory"); @@ -1540,7 +1569,21 @@ public void testAuthzURIWithAuthEnabledWithMetadataLocation() throws HiveExcepti } @Test - public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizer() throws HiveException { + public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizerMasked() + throws HiveException, TException, InterruptedException { + Assume.assumeTrue(testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testAuthzURIWithAuthEnabledAndMockCommandAuthorizer(true); + } + + @Test + public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizerUnmasked() + throws HiveException, TException, InterruptedException { + testAuthzURIWithAuthEnabledAndMockCommandAuthorizer(false); + } + + public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizer(boolean masked) + throws HiveException, TException, InterruptedException { + shell.getHiveConf().setBoolean(HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, masked); shell.setHiveSessionValue("hive.security.authorization.enabled", true); shell.setHiveSessionValue("hive.security.authorization.manager", "org.apache.iceberg.mr.hive.CustomTestHiveAuthorizerFactory"); @@ -1554,8 +1597,18 @@ public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizer() throws HiveExc Optional hivePrivObject = outputHObjsCaptor.getValue().stream() .filter(hpo -> hpo.getType().equals(HivePrivilegeObject.HivePrivilegeObjectType.STORAGEHANDLER_URI)).findAny(); if (hivePrivObject.isPresent()) { + org.apache.hadoop.hive.metastore.api.Table hmsTable = shell.metastore().getTable(target); + HiveIcebergStorageHandler storageHandler = new HiveIcebergStorageHandler(); + storageHandler.setConf(shell.getHiveConf()); + String metadataLocation = HiveConf.EncoderDecoderFactory.URL_ENCODER_DECODER.decode( + storageHandler.getPathForAuth(((BaseTable) table).operations().current().metadataFileLocation(), + hmsTable.getSd().getLocation())); + + if (masked) { + Assert.assertTrue(metadataLocation.startsWith(HiveIcebergStorageHandler.TABLE_DEFAULT_LOCATION)); + } Assert.assertEquals("iceberg://" + target.namespace() + "/" + target.name() + "?snapshot=" + - new Path(((BaseTable) table).operations().current().metadataFileLocation()).getParent().toUri() + new Path(metadataLocation).getParent().toUri() .getPath() + "/dummy.metadata.json", HiveConf.EncoderDecoderFactory.URL_ENCODER_DECODER.decode(hivePrivObject.get().getObjectName())); @@ -1565,7 +1618,18 @@ public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizer() throws HiveExc } @Test - public void testAuthzURIWithAuthEnabled() throws TException, InterruptedException, URISyntaxException { + public void testAuthzURIWithAuthEnabledMasked() throws TException, URISyntaxException, InterruptedException { + Assume.assumeTrue(testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testAuthzURIWithAuthEnabled(true); + } + + @Test + public void testAuthzURIWithAuthEnabledUnmasked() throws TException, URISyntaxException, InterruptedException { + testAuthzURIWithAuthEnabled(false); + } + + public void testAuthzURIWithAuthEnabled(boolean masked) throws TException, InterruptedException, URISyntaxException { + shell.getHiveConf().setBoolean(HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, masked); shell.setHiveSessionValue("hive.security.authorization.enabled", true); TableIdentifier target = TableIdentifier.of("default", "target"); Table table = testTables.createTable(shell, target.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, @@ -1575,9 +1639,15 @@ public void testAuthzURIWithAuthEnabled() throws TException, InterruptedExceptio HiveIcebergStorageHandler storageHandler = new HiveIcebergStorageHandler(); storageHandler.setConf(shell.getHiveConf()); URI uriForAuth = storageHandler.getURIForAuth(hmsTable); + String metadataLocation = + storageHandler.getPathForAuth(((BaseTable) table).operations().current().metadataFileLocation(), + hmsTable.getSd().getLocation()); + + if (masked) { + Assert.assertTrue(metadataLocation.startsWith(HiveIcebergStorageHandler.TABLE_DEFAULT_LOCATION)); + } Assert.assertEquals("iceberg://" + target.namespace() + "/" + target.name() + "?snapshot=" + - URI.create(((BaseTable) table).operations().current() - .metadataFileLocation()).getPath(), + URI.create(metadataLocation).getPath(), HiveConf.EncoderDecoderFactory.URL_ENCODER_DECODER.decode(uriForAuth.toString())); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 7d646e8b7ab5..9fe61c0bdc32 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -24,6 +24,7 @@ import java.net.URISyntaxException; import java.util.Collections; import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hadoop.hive.common.classification.InterfaceStability; import org.apache.hadoop.hive.common.type.SnapshotContext; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 9fa27638eb0e..8cdba7fd8dfe 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -25,6 +25,7 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVEARCHIVEENABLED; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_DEFAULT_STORAGE_HANDLER; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.DEFAULT_TABLE_LOCATION; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.TABLE_IS_CTAS; @@ -13800,6 +13801,11 @@ private Map validateAndAddDefaultProperties( addDbAndTabToOutputs(qualifiedTabName, TableType.EXTERNAL_TABLE, isTemporaryTable, retValue, storageFormat); } + + if (isIcebergTable(retValue)) { + SessionStateUtil.addResourceOrThrow(conf, hive_metastoreConstants.DEFAULT_TABLE_LOCATION, + getDefaultLocation(qualifiedTabName[0], qualifiedTabName[1], true)); + } return retValue; } @@ -14211,13 +14217,7 @@ ASTNode analyzeCreateTable( if (location != null) { tblLocation = location; } else { - try { - Warehouse wh = new Warehouse(conf); - tblLocation = wh.getDefaultTablePath(db.getDatabase(qualifiedTabName.getDb()), qualifiedTabName.getTable(), - isExt).toUri().getPath(); - } catch (MetaException | HiveException e) { - throw new SemanticException(e); - } + tblLocation = getDefaultLocation(qualifiedTabName.getDb(), qualifiedTabName.getTable(), isExt); } try { HiveStorageHandler storageHandler = HiveUtils.getStorageHandler(conf, storageFormat.getStorageHandler()); @@ -14376,6 +14376,17 @@ ASTNode analyzeCreateTable( return null; } + private String getDefaultLocation(String dbName, String tableName, boolean isExt) throws SemanticException { + String tblLocation; + try { + Warehouse wh = new Warehouse(conf); + tblLocation = wh.getDefaultTablePath(db.getDatabase(dbName), tableName, isExt).toUri().getPath(); + } catch (MetaException | HiveException e) { + throw new SemanticException(e); + } + return tblLocation; + } + private static boolean isIcebergTable(Map tblProps) { return AlterTableConvertOperation.ConversionFormats.ICEBERG.properties().get(META_TABLE_STORAGE) .equalsIgnoreCase(tblProps.get(META_TABLE_STORAGE)); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java index f5a102ab9647..776683b882f8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java @@ -97,4 +97,6 @@ public static final java.lang.String EXPECTED_PARAMETER_VALUE = "expected_parameter_value"; + public static final java.lang.String DEFAULT_TABLE_LOCATION = "defaultLocation"; + } From 1787a08ac642fd674629315f171ea5e79dfc50be Mon Sep 17 00:00:00 2001 From: Shivangi Jha <149884343+shivjha30@users.noreply.github.com> Date: Mon, 4 Dec 2023 20:17:22 +0530 Subject: [PATCH 077/179] Tez Progress bar is not displayed in Beeline upon setting session level execution engine to Tez (Shivangi Jha, reviewed by Attila Turoczy, Vikram Ahuja, Ayush Saxena, Laszlo Bodor) --- service/src/java/org/apache/hive/service/cli/CLIService.java | 4 ++++ .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/service/src/java/org/apache/hive/service/cli/CLIService.java b/service/src/java/org/apache/hive/service/cli/CLIService.java index 151b2067576f..a061536f857d 100644 --- a/service/src/java/org/apache/hive/service/cli/CLIService.java +++ b/service/src/java/org/apache/hive/service/cli/CLIService.java @@ -510,6 +510,10 @@ public OperationStatus getOperationStatus(OperationHandle opHandle, boolean getP opStatus.setJobProgressUpdate(progressUpdateLog(getProgressUpdate, operation, conf)); return opStatus; } + public HiveConf getHiveSessionConf(OperationHandle opHandle) throws HiveSQLException { + Operation operation = sessionManager.getOperationManager().getOperation(opHandle); + return operation.getParentSession().getHiveConf(); + } public HiveConf getSessionConf(SessionHandle sessionHandle) throws HiveSQLException { diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index d93d54b33b1e..7d71881dc844 100644 --- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -794,7 +794,7 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th try { OperationStatus operationStatus = cliService.getOperationStatus(operationHandle, req.isGetProgressUpdate()); - + HiveConf sessionConf = cliService.getHiveSessionConf(operationHandle); if (operationStatus.getState().equals(OperationState.FINISHED)) { long numModifiedRows = operationStatus.getNumModifiedRows(); resp.setNumModifiedRows(numModifiedRows); @@ -810,7 +810,7 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th } JobProgressUpdate progressUpdate = operationStatus.jobProgressUpdate(); ProgressMonitorStatusMapper mapper = ProgressMonitorStatusMapper.DEFAULT; - if ("tez".equals(hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) { + if ("tez".equals(sessionConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) { mapper = new TezProgressMonitorStatusMapper(); } TJobExecutionStatus executionStatus = From 276b37830f18cd34d89db27bc549594aecad3d56 Mon Sep 17 00:00:00 2001 From: Raghav Aggarwal Date: Mon, 4 Dec 2023 20:29:33 +0530 Subject: [PATCH 078/179] HIVE-27662: Incorrect parsing of nested complex types containing map during vectorized text processing (Raghav Aggarwal, Reviewed by Denys Kuzmenko, Chinna Rao Lalam) --- .../resources/testconfiguration.properties | 1 + .../queries/clientpositive/complex_datatype.q | 232 ++++++ .../llap/complex_datatype.q.out | 668 ++++++++++++++++++ .../lazy/fast/LazySimpleDeserializeRead.java | 14 +- 4 files changed, 906 insertions(+), 9 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/complex_datatype.q create mode 100644 ql/src/test/results/clientpositive/llap/complex_datatype.q.out diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 367b922d1300..7a76db86d7c5 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -56,6 +56,7 @@ minillap.query.files=\ cmv_direct.q,\ cmv_direct_with_specified_locations.q,\ cmv_direct_with_suffixed_locations.q,\ + complex_datatype.q,\ create_genericudaf.q,\ create_table.q,\ create_udaf.q,\ diff --git a/ql/src/test/queries/clientpositive/complex_datatype.q b/ql/src/test/queries/clientpositive/complex_datatype.q new file mode 100644 index 000000000000..49cc35f6a1be --- /dev/null +++ b/ql/src/test/queries/clientpositive/complex_datatype.q @@ -0,0 +1,232 @@ +set hive.default.fileformat=TEXTFILE; +set hive.fetch.task.conversion=none; +set hive.llap.io.enabled=false; +set hive.vectorized.execution.enabled=true; + +create EXTERNAL table `complex_map_array_table` as +select +'bob' as name, + MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column2; + +create EXTERNAL table `complex_map_struct_table` as +select +'bob' as name, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column2; + + +create EXTERNAL table `complex_table1` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +'bob' as name, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column3; + +create EXTERNAL table `complex_table2` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column2; + +create EXTERNAL table `complex_table3` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +MAP( + "Key3", + ARRAY( + 7, + 8, + 9 + ), + "Key4", + ARRAY( + 10, + 11, + 12 + ) + ) as column2; + +-- The below scenario's was working before fix +create EXTERNAL table `complex_array_map_table` as +select +'bob' as name, +ARRAY( + MAP( + "Key1", + "Value1" + ), + MAP( + "Key2", + "Value2" + ) + ) as column2; + +create EXTERNAL table `complex_map_map_table` as +select + 'bob' as name, + MAP( + "Key1", + MAP( + 1, + 2 + ), + "Key2", + MAP( + 3, + 4 + ) + ) as column2; + +create EXTERNAL table `complex_combined_table` as +select + ARRAY('arr_val1', 'arr_val2', 'arr_val3') as column1, + 'bob' as column2, + MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column3, + NAMED_STRUCT('abc', '7', 'def', '8') as column4, + MAP( + "Key3", + "Value3", + "Key4", + "Value4" + ) as column5; + +-- with vectorization set as "true" +select * from complex_map_array_table; +select * from complex_map_struct_table; +select * from complex_table1; +select * from complex_table2; +select * from complex_table3; +select * from complex_array_map_table; +select * from complex_map_map_table; +select * from complex_combined_table; + +-- with fetch task conversion set as "more" +set hive.fetch.task.conversion=more; + +select * from complex_map_array_table; +select * from complex_map_struct_table; +select * from complex_table1; +select * from complex_table2; +select * from complex_table3; +select * from complex_array_map_table; +select * from complex_map_map_table; +select * from complex_combined_table; + +-- with vectorization set as "false" +set hive.vectorized.execution.enabled=false; + +select * from complex_map_array_table; +select * from complex_map_struct_table; +select * from complex_table1; +select * from complex_table2; +select * from complex_table3; +select * from complex_array_map_table; +select * from complex_map_map_table; +select * from complex_combined_table; diff --git a/ql/src/test/results/clientpositive/llap/complex_datatype.q.out b/ql/src/test/results/clientpositive/llap/complex_datatype.q.out new file mode 100644 index 000000000000..727e185642c1 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/complex_datatype.q.out @@ -0,0 +1,668 @@ +PREHOOK: query: create EXTERNAL table `complex_map_array_table` as +select +'bob' as name, + MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_map_array_table +POSTHOOK: query: create EXTERNAL table `complex_map_array_table` as +select +'bob' as name, + MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_map_array_table +POSTHOOK: Lineage: complex_map_array_table.column2 EXPRESSION [] +POSTHOOK: Lineage: complex_map_array_table.name SIMPLE [] +PREHOOK: query: create EXTERNAL table `complex_map_struct_table` as +select +'bob' as name, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_map_struct_table +POSTHOOK: query: create EXTERNAL table `complex_map_struct_table` as +select +'bob' as name, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_map_struct_table +POSTHOOK: Lineage: complex_map_struct_table.column2 EXPRESSION [] +POSTHOOK: Lineage: complex_map_struct_table.name SIMPLE [] +PREHOOK: query: create EXTERNAL table `complex_table1` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +'bob' as name, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column3 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_table1 +POSTHOOK: query: create EXTERNAL table `complex_table1` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +'bob' as name, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column3 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_table1 +POSTHOOK: Lineage: complex_table1.column1 EXPRESSION [] +POSTHOOK: Lineage: complex_table1.column3 EXPRESSION [] +POSTHOOK: Lineage: complex_table1.name SIMPLE [] +PREHOOK: query: create EXTERNAL table `complex_table2` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_table2 +POSTHOOK: query: create EXTERNAL table `complex_table2` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +MAP( + "Map_Key1", + named_struct( + 'Id', + 'Id_Value1', + 'Name', + 'Name_Value1' + ), + "Map_Key2", + named_struct( + 'Id', + 'Id_Value2', + 'Name', + 'Name_Value2' + ) + ) as column2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_table2 +POSTHOOK: Lineage: complex_table2.column1 EXPRESSION [] +POSTHOOK: Lineage: complex_table2.column2 EXPRESSION [] +PREHOOK: query: create EXTERNAL table `complex_table3` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +MAP( + "Key3", + ARRAY( + 7, + 8, + 9 + ), + "Key4", + ARRAY( + 10, + 11, + 12 + ) + ) as column2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_table3 +POSTHOOK: query: create EXTERNAL table `complex_table3` as +select +MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column1, +MAP( + "Key3", + ARRAY( + 7, + 8, + 9 + ), + "Key4", + ARRAY( + 10, + 11, + 12 + ) + ) as column2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_table3 +POSTHOOK: Lineage: complex_table3.column1 EXPRESSION [] +POSTHOOK: Lineage: complex_table3.column2 EXPRESSION [] +PREHOOK: query: create EXTERNAL table `complex_array_map_table` as +select +'bob' as name, +ARRAY( + MAP( + "Key1", + "Value1" + ), + MAP( + "Key2", + "Value2" + ) + ) as column2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_array_map_table +POSTHOOK: query: create EXTERNAL table `complex_array_map_table` as +select +'bob' as name, +ARRAY( + MAP( + "Key1", + "Value1" + ), + MAP( + "Key2", + "Value2" + ) + ) as column2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_array_map_table +POSTHOOK: Lineage: complex_array_map_table.column2 EXPRESSION [] +POSTHOOK: Lineage: complex_array_map_table.name SIMPLE [] +PREHOOK: query: create EXTERNAL table `complex_map_map_table` as +select + 'bob' as name, + MAP( + "Key1", + MAP( + 1, + 2 + ), + "Key2", + MAP( + 3, + 4 + ) + ) as column2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_map_map_table +POSTHOOK: query: create EXTERNAL table `complex_map_map_table` as +select + 'bob' as name, + MAP( + "Key1", + MAP( + 1, + 2 + ), + "Key2", + MAP( + 3, + 4 + ) + ) as column2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_map_map_table +POSTHOOK: Lineage: complex_map_map_table.column2 EXPRESSION [] +POSTHOOK: Lineage: complex_map_map_table.name SIMPLE [] +PREHOOK: query: create EXTERNAL table `complex_combined_table` as +select + ARRAY('arr_val1', 'arr_val2', 'arr_val3') as column1, + 'bob' as column2, + MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column3, + NAMED_STRUCT('abc', '7', 'def', '8') as column4, + MAP( + "Key3", + "Value3", + "Key4", + "Value4" + ) as column5 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@complex_combined_table +POSTHOOK: query: create EXTERNAL table `complex_combined_table` as +select + ARRAY('arr_val1', 'arr_val2', 'arr_val3') as column1, + 'bob' as column2, + MAP( + "Key1", + ARRAY( + 1, + 2, + 3 + ), + "Key2", + ARRAY( + 4, + 5, + 6 + ) + ) as column3, + NAMED_STRUCT('abc', '7', 'def', '8') as column4, + MAP( + "Key3", + "Value3", + "Key4", + "Value4" + ) as column5 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@complex_combined_table +POSTHOOK: Lineage: complex_combined_table.column1 EXPRESSION [] +POSTHOOK: Lineage: complex_combined_table.column2 SIMPLE [] +POSTHOOK: Lineage: complex_combined_table.column3 EXPRESSION [] +POSTHOOK: Lineage: complex_combined_table.column4 EXPRESSION [] +POSTHOOK: Lineage: complex_combined_table.column5 EXPRESSION [] +PREHOOK: query: select * from complex_map_array_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_array_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_array_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_array_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Key1":[1,2,3],"Key2":[4,5,6]} +PREHOOK: query: select * from complex_map_struct_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_struct_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_struct_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_struct_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table1 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table1 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table1 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} bob {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table2 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table2 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table2 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table3 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} {"Key3":[7,8,9],"Key4":[10,11,12]} +PREHOOK: query: select * from complex_array_map_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_array_map_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_array_map_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_array_map_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob [{"Key1":"Value1"},{"Key2":"Value2"}] +PREHOOK: query: select * from complex_map_map_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_map_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_map_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_map_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Key1":{1:2},"Key2":{3:4}} +PREHOOK: query: select * from complex_combined_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_combined_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_combined_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_combined_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +["arr_val1","arr_val2","arr_val3"] bob {"Key1":[1,2,3],"Key2":[4,5,6]} {"abc":"7","def":"8"} {"Key3":"Value3","Key4":"Value4"} +PREHOOK: query: select * from complex_map_array_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_array_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_array_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_array_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Key1":[1,2,3],"Key2":[4,5,6]} +PREHOOK: query: select * from complex_map_struct_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_struct_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_struct_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_struct_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table1 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table1 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table1 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} bob {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table2 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table2 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table2 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table3 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} {"Key3":[7,8,9],"Key4":[10,11,12]} +PREHOOK: query: select * from complex_array_map_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_array_map_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_array_map_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_array_map_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob [{"Key1":"Value1"},{"Key2":"Value2"}] +PREHOOK: query: select * from complex_map_map_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_map_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_map_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_map_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Key1":{1:2},"Key2":{3:4}} +PREHOOK: query: select * from complex_combined_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_combined_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_combined_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_combined_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +["arr_val1","arr_val2","arr_val3"] bob {"Key1":[1,2,3],"Key2":[4,5,6]} {"abc":"7","def":"8"} {"Key3":"Value3","Key4":"Value4"} +PREHOOK: query: select * from complex_map_array_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_array_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_array_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_array_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Key1":[1,2,3],"Key2":[4,5,6]} +PREHOOK: query: select * from complex_map_struct_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_struct_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_struct_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_struct_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table1 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table1 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table1 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} bob {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table2 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table2 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table2 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} {"Map_Key1":{"id":"Id_Value1","name":"Name_Value1"},"Map_Key2":{"id":"Id_Value2","name":"Name_Value2"}} +PREHOOK: query: select * from complex_table3 +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_table3 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_table3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_table3 +POSTHOOK: Output: hdfs://### HDFS PATH ### +{"Key1":[1,2,3],"Key2":[4,5,6]} {"Key3":[7,8,9],"Key4":[10,11,12]} +PREHOOK: query: select * from complex_array_map_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_array_map_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_array_map_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_array_map_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob [{"Key1":"Value1"},{"Key2":"Value2"}] +PREHOOK: query: select * from complex_map_map_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_map_map_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_map_map_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_map_map_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +bob {"Key1":{1:2},"Key2":{3:4}} +PREHOOK: query: select * from complex_combined_table +PREHOOK: type: QUERY +PREHOOK: Input: default@complex_combined_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from complex_combined_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@complex_combined_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +["arr_val1","arr_val2","arr_val3"] bob {"Key1":[1,2,3],"Key2":[4,5,6]} {"abc":"7","def":"8"} {"Key3":"Value3","Key4":"Value4"} diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java index 34e5a4a9bafe..12fdfd66304d 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java @@ -875,15 +875,11 @@ private boolean doReadField(Field field) { case STRUCT: case UNION: { - if (currentLevel > 0) { - - // Check for Map which occupies 2 levels (key separator and key/value pair separator). - if (currentComplexTypeHelpers[currentLevel - 1] == null) { - Preconditions.checkState(currentLevel > 1); - Preconditions.checkState( - currentComplexTypeHelpers[currentLevel - 2] instanceof MapComplexTypeHelper); - currentLevel++; - } + // Check for Map which occupies 2 levels (key separator and key/value pair separator). + if (currentLevel > 0 + && currentComplexTypeHelpers[currentLevel] == null + && currentComplexTypeHelpers[currentLevel - 1] instanceof MapComplexTypeHelper) { + currentLevel++; } ComplexTypeHelper complexTypeHelper = field.complexTypeHelper; currentComplexTypeHelpers[currentLevel++] = complexTypeHelper; From 66b51d6aa5211c46fc0d727a66c5e0de333cd377 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 5 Dec 2023 10:43:53 +0530 Subject: [PATCH 079/179] HIVE-27926: Iceberg: Allow restricting Iceberg data file reads to table location. (#4910). (Ayush Saxena, reviewed by Denys Kuzmenko) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 5 +- .../mr/hive/HiveIcebergStorageHandler.java | 7 ++ .../mr/mapreduce/IcebergInputFormat.java | 20 +++++ .../TestHiveIcebergRestrictDataFiles.java | 74 +++++++++++++++++++ .../apache/hive/jdbc/TestRestrictedList.java | 1 + 5 files changed, 106 insertions(+), 1 deletion(-) create mode 100644 iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index ad807386f360..991c97e250a0 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2229,6 +2229,8 @@ public static enum ConfVars { HIVE_ICEBERG_MASK_DEFAULT_LOCATION("hive.iceberg.mask.default.location", false, "If this is set to true the URI for auth will have the default location masked with DEFAULT_TABLE_LOCATION"), + HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY("hive.iceberg.allow.datafiles.in.table.location.only", false, + "If this is set to true, then all the data files being read should be withing the table location"), HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true, "If this is set the header for RCFiles will simply be RCF. If this is not\n" + @@ -5575,7 +5577,8 @@ public static enum ConfVars { "hive.zookeeper.ssl.keystore.type," + "hive.zookeeper.ssl.truststore.location," + "hive.zookeeper.ssl.truststore.password," + - "hive.zookeeper.ssl.truststore.type", + "hive.zookeeper.ssl.truststore.type," + + "hive.iceberg.allow.datafiles.in.table.location.only", "Comma separated list of configuration options which are immutable at runtime"), HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list", METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 5f4f97b9f72b..e3336437c778 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -278,6 +278,13 @@ public void configureInputJobProperties(TableDesc tableDesc, Map overlayTableProperties(conf, tableDesc, map); // Until the vectorized reader can handle delete files, let's fall back to non-vector mode for V2 tables fallbackToNonVectorizedModeBasedOnProperties(tableDesc.getProperties()); + + boolean allowDataFilesWithinTableLocationOnly = + conf.getBoolean(HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, + HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.defaultBoolVal); + + map.put(HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, + String.valueOf(allowDataFilesWithinTableLocationOnly)); } @Override diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index dc50a1e34010..3ec1a3b3b7a5 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -33,8 +33,11 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.llap.LlapHiveUtils; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.AuthorizationException; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.mapred.JobConf; @@ -218,6 +221,12 @@ public List getSplits(JobContext context) { scan = applyConfig(conf, createTableScan(table, conf)); } + boolean allowDataFilesWithinTableLocationOnly = + conf.getBoolean(HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, + HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.defaultBoolVal); + Path tableLocation = new Path(conf.get(InputFormatConfig.TABLE_LOCATION)); + + try (CloseableIterable tasksIterable = scan.planTasks()) { tasksIterable.forEach(task -> { if (applyResidual && (model == InputFormatConfig.InMemoryDataModel.HIVE || @@ -225,6 +234,9 @@ public List getSplits(JobContext context) { // TODO: We do not support residual evaluation for HIVE and PIG in memory data model yet checkResiduals(task); } + if (allowDataFilesWithinTableLocationOnly) { + validateFileLocations(task, tableLocation); + } splits.add(new IcebergSplit(conf, task)); }); } catch (IOException e) { @@ -241,6 +253,14 @@ public List getSplits(JobContext context) { return splits; } + private static void validateFileLocations(CombinedScanTask split, Path tableLocation) { + for (FileScanTask fileScanTask : split.files()) { + if (!FileUtils.isPathWithinSubtree(new Path(fileScanTask.file().path().toString()), tableLocation)) { + throw new AuthorizationException("The table contains paths which are outside the table location"); + } + } + } + private static void checkResiduals(CombinedScanTask task) { task.files().forEach(fileScanTask -> { Expression residual = fileScanTask.residual(); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java new file mode 100644 index 000000000000..e9d6950ef460 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import org.apache.commons.collections4.ListUtils; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY; + +public class TestHiveIcebergRestrictDataFiles extends HiveIcebergStorageHandlerWithEngineBase { + + @BeforeClass + public static void beforeClass() { + shell = HiveIcebergStorageHandlerTestUtils.shell( + Collections.singletonMap(HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, "true")); + } + + @Test + public void testRestrictDataFiles() throws IOException, InterruptedException { + TableIdentifier table1 = TableIdentifier.of("default", "tab1"); + testTables.createTableWithVersions(shell, table1.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + + AssertHelpers.assertThrows("Should throw exception since there are files outside the table directory", + IllegalArgumentException.class, "The table contains paths which are outside the table location", + () -> shell.executeStatement("SELECT * FROM " + table1.name())); + + // Create another table with files within the table location + TableIdentifier table2 = TableIdentifier.of("default", "tab2"); + testTables.createTableWithVersions(shell, table2.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, null, 0); + + shell.executeStatement( + testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, table2, false)); + + List result = shell.executeStatement("SELECT * FROM " + table2.name()); + + HiveIcebergTestUtils.validateData(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, + HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, result), 0); + + // Insert some more records to generate new Data file + shell.executeStatement( + testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, table2, false)); + + result = shell.executeStatement("SELECT * FROM " + table2.name()); + + HiveIcebergTestUtils.validateData(ListUtils.union(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, + HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1), + HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, result), 0); + } +} diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java index aeec57757c21..52be546dca8d 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java @@ -109,6 +109,7 @@ public static void startServices() throws Exception { addToExpectedRestrictedMap("hive.zookeeper.ssl.truststore.location"); addToExpectedRestrictedMap("hive.zookeeper.ssl.truststore.password"); addToExpectedRestrictedMap("hive.zookeeper.ssl.truststore.type"); + addToExpectedRestrictedMap("hive.iceberg.allow.datafiles.in.table.location.only"); checkRestrictedListMatch(); } From 8d0a38b82315aa412a9f3a1f60046d9259346819 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Tue, 5 Dec 2023 13:57:21 +0530 Subject: [PATCH 080/179] HIVE-27918: Iceberg: Push transforms for clustering during table writes (#4908) (Sourabh Badhya reviewed by Denys Kuzmenko) --- .../mr/hive/HiveIcebergStorageHandler.java | 34 +- .../IcebergTransformSortFunctionUtil.java | 158 +++ .../{ => udf}/GenericUDFIcebergBucket.java | 2 +- .../mr/hive/udf/GenericUDFIcebergDay.java | 129 +++ .../mr/hive/udf/GenericUDFIcebergHour.java | 118 ++ .../mr/hive/udf/GenericUDFIcebergMonth.java | 129 +++ .../hive/udf/GenericUDFIcebergTruncate.java | 186 +++ .../mr/hive/udf/GenericUDFIcebergYear.java | 129 +++ .../positive/dynamic_partition_writes.q | 100 +- ...te_iceberg_copy_on_write_partitioned.q.out | 36 +- .../positive/dynamic_partition_writes.q.out | 1031 +++++++++++++++++ ...ge_iceberg_copy_on_write_partitioned.q.out | 24 +- .../merge_iceberg_partitioned_orc.q.out | 16 +- ...te_iceberg_copy_on_write_partitioned.q.out | 64 +- .../hadoop/hive/ql/exec/FunctionRegistry.java | 12 +- 15 files changed, 2064 insertions(+), 104 deletions(-) create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java rename iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/{ => udf}/GenericUDFIcebergBucket.java (99%) create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index e3336437c778..40e4cb1cc934 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -38,7 +38,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.commons.collections.MapUtils; @@ -76,7 +75,6 @@ import org.apache.hadoop.hive.ql.ddl.table.misc.properties.AlterTableSetPropertiesDesc; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.FetchOperator; -import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.IOConstants; @@ -120,7 +118,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.InputFormat; @@ -217,26 +214,6 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H public static final String TABLE_DEFAULT_LOCATION = "TABLE_DEFAULT_LOCATION"; - /** - * Function template for producing a custom sort expression function: - * Takes the source column index and the bucket count to creat a function where Iceberg bucket UDF is used to build - * the sort expression, e.g. iceberg_bucket(_col2, 5) - */ - private static final transient BiFunction, ExprNodeDesc>> - BUCKET_SORT_EXPR = - (idx, bucket) -> cols -> { - try { - ExprNodeDesc icebergBucketSourceCol = cols.get(idx); - return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergBucket(), "iceberg_bucket", - Lists.newArrayList( - icebergBucketSourceCol, - new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, bucket) - )); - } catch (UDFArgumentException e) { - throw new RuntimeException(e); - } - }; - private static final List ACID_VIRTUAL_COLS = ImmutableList.of(VirtualColumn.PARTITION_SPEC_ID, VirtualColumn.PARTITION_HASH, VirtualColumn.FILE_PATH, VirtualColumn.ROW_POSITION); private static final List ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA = ACID_VIRTUAL_COLS.stream() @@ -738,14 +715,9 @@ private void addCustomSortExpr(Table table, org.apache.hadoop.hive.ql.metadata. int offset = (shouldOverwrite(hmsTable, writeOperation) ? ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA : acidSelectColumns(hmsTable, writeOperation)).size(); - for (TransformSpec spec : transformSpecs) { - int order = fieldOrderMap.get(spec.getColumnName()); - if (TransformSpec.TransformType.BUCKET.equals(spec.getTransformType())) { - customSortExprs.add(BUCKET_SORT_EXPR.apply(order + offset, spec.getTransformParam().get())); - } else { - customSortExprs.add(cols -> cols.get(order + offset).clone()); - } - } + customSortExprs.addAll(transformSpecs.stream().map(spec -> + IcebergTransformSortFunctionUtil.getCustomSortExprs(spec, fieldOrderMap.get(spec.getColumnName()) + offset) + ).collect(Collectors.toList())); } @Override diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java new file mode 100644 index 000000000000..5373a4d903da --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTransformSortFunctionUtil.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive; + +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.parse.TransformSpec; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergBucket; +import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergDay; +import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergHour; +import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergMonth; +import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergTruncate; +import org.apache.iceberg.mr.hive.udf.GenericUDFIcebergYear; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * A utility class which provides Iceberg transform sort functions. + */ +public final class IcebergTransformSortFunctionUtil { + + private IcebergTransformSortFunctionUtil() { + // not called + } + + /** + * Function template for producing a custom sort expression function: + * Takes the source column index and the bucket count to create a function where Iceberg transform UDF is used to + * build the sort expression, e.g. iceberg_bucket(_col2, 5) + */ + private static final transient BiFunction, ExprNodeDesc>> + BUCKET_SORT_EXPR = + (idx, bucket) -> cols -> { + try { + ExprNodeDesc icebergBucketSourceCol = cols.get(idx); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergBucket(), "iceberg_bucket", + Lists.newArrayList( + icebergBucketSourceCol, + new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, bucket) + )); + } catch (UDFArgumentException e) { + throw new RuntimeException(e); + } + }; + + private static final transient BiFunction, ExprNodeDesc>> + TRUNCATE_SORT_EXPR = + (idx, truncateLength) -> cols -> { + try { + ExprNodeDesc icebergTruncateSourceCol = cols.get(idx); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergTruncate(), "iceberg_truncate", + Lists.newArrayList( + icebergTruncateSourceCol, + new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, truncateLength) + )); + } catch (UDFArgumentException e) { + throw new RuntimeException(e); + } + }; + + private static final transient Function, ExprNodeDesc>> + YEAR_SORT_EXPR = + idx -> cols -> { + try { + ExprNodeDesc icebergYearSourceCol = cols.get(idx); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergYear(), "iceberg_year", + Lists.newArrayList( + icebergYearSourceCol + )); + } catch (UDFArgumentException e) { + throw new RuntimeException(e); + } + }; + + private static final transient Function, ExprNodeDesc>> + MONTH_SORT_EXPR = + idx -> cols -> { + try { + ExprNodeDesc icebergMonthSourceCol = cols.get(idx); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergMonth(), "iceberg_month", + Lists.newArrayList( + icebergMonthSourceCol + )); + } catch (UDFArgumentException e) { + throw new RuntimeException(e); + } + }; + + private static final transient Function, ExprNodeDesc>> + DAY_SORT_EXPR = + idx -> cols -> { + try { + ExprNodeDesc icebergDaySourceCol = cols.get(idx); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergDay(), "iceberg_day", + Lists.newArrayList( + icebergDaySourceCol + )); + } catch (UDFArgumentException e) { + throw new RuntimeException(e); + } + }; + + private static final transient Function, ExprNodeDesc>> + HOUR_SORT_EXPR = + idx -> cols -> { + try { + ExprNodeDesc icebergHourSourceCol = cols.get(idx); + return ExprNodeGenericFuncDesc.newInstance(new GenericUDFIcebergHour(), "iceberg_hour", + Lists.newArrayList( + icebergHourSourceCol + )); + } catch (UDFArgumentException e) { + throw new RuntimeException(e); + } + }; + + public static Function, ExprNodeDesc> getCustomSortExprs(TransformSpec spec, int index) { + switch (spec.getTransformType()) { + case BUCKET: + return BUCKET_SORT_EXPR.apply(index, spec.getTransformParam().get()); + case TRUNCATE: + return TRUNCATE_SORT_EXPR.apply(index, spec.getTransformParam().get()); + case YEAR: + return YEAR_SORT_EXPR.apply(index); + case MONTH: + return MONTH_SORT_EXPR.apply(index); + case DAY: + return DAY_SORT_EXPR.apply(index); + case HOUR: + return HOUR_SORT_EXPR.apply(index); + default: + return cols -> cols.get(index).clone(); + } + } + +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergBucket.java similarity index 99% rename from iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java rename to iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergBucket.java index 5f27319d12f8..0077e6706fd3 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/GenericUDFIcebergBucket.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergBucket.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.iceberg.mr.hive; +package org.apache.iceberg.mr.hive.udf; import java.nio.ByteBuffer; import java.util.function.Function; diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java new file mode 100644 index 000000000000..344961222a2d --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergDay.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.udf; + +import java.util.function.Function; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; +import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.IntWritable; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; + +/** + * GenericUDFIcebergDay - UDF that wraps around Iceberg's day transform function + */ +@Description(name = "iceberg_day", + value = "_FUNC_(value) - " + + "Returns the bucket value calculated by Iceberg bucket transform function ", + extended = "Example:\n > SELECT _FUNC_('2023-01-02', 5);\n 2") +public class GenericUDFIcebergDay extends GenericUDF { + private final IntWritable result = new IntWritable(); + private transient PrimitiveObjectInspector argumentOI; + private transient ObjectInspectorConverters.Converter converter; + + @FunctionalInterface + private interface UDFEvalFunction { + void apply(T argument) throws HiveException; + } + + private transient UDFEvalFunction evaluator; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 1) { + throw new UDFArgumentLengthException( + "ICEBERG_DAY requires 1 arguments (value), but got " + arguments.length); + } + + if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw new UDFArgumentException( + "ICEBERG_DAY first argument takes primitive types, got " + argumentOI.getTypeName()); + } + argumentOI = (PrimitiveObjectInspector) arguments[0]; + + PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory(); + ObjectInspector outputOI; + switch (inputType) { + case DATE: + converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableDateObjectInspector); + Function dateTransform = Transforms.day().bind(Types.DateType.get()); + evaluator = arg -> { + DateWritableV2 val = (DateWritableV2) converter.convert(arg.get()); + result.set(dateTransform.apply(val.getDays())); + }; + break; + + case TIMESTAMP: + converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampObjectInspector); + Function timestampTransform = Transforms.day().bind(Types.TimestampType.withoutZone()); + evaluator = arg -> { + TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get()); + result.set(timestampTransform.apply(val.getNanos() / 1000L)); + }; + break; + + case TIMESTAMPLOCALTZ: + converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector); + Function timestampLocalTzTransform = Transforms.day().bind(Types.TimestampType.withZone()); + evaluator = arg -> { + TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get()); + result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L)); + }; + break; + + default: + throw new UDFArgumentException( + " ICEBERG_DAY() only takes DATE/TIMESTAMP/TIMESTAMPLOCALTZ" + + " types as first argument, got " + inputType); + } + outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector; + return outputOI; + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + DeferredObject argument = arguments[0]; + if (argument == null || argument.get() == null) { + return null; + } else { + evaluator.apply(argument); + } + return result; + } + + @Override + public String getDisplayString(String[] children) { + return getStandardDisplayString("iceberg_day", children); + } + +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java new file mode 100644 index 000000000000..9457d56e22cf --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergHour.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.udf; + +import java.util.function.Function; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.IntWritable; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; + +/** + * GenericUDFIcebergHour - UDF that wraps around Iceberg's hour transform function + */ +@Description(name = "iceberg_hour", + value = "_FUNC_(value) - " + + "Returns the bucket value calculated by Iceberg hour transform function ", + extended = "Example:\n > SELECT _FUNC_('2023-01-01 11:00:57');\n 11") +public class GenericUDFIcebergHour extends GenericUDF { + private final IntWritable result = new IntWritable(); + private transient PrimitiveObjectInspector argumentOI; + private transient ObjectInspectorConverters.Converter converter; + + @FunctionalInterface + private interface UDFEvalFunction { + void apply(T argument) throws HiveException; + } + + private transient UDFEvalFunction evaluator; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 1) { + throw new UDFArgumentLengthException( + "ICEBERG_YEAR requires 1 arguments (value), but got " + arguments.length); + } + + if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw new UDFArgumentException( + "ICEBERG_YEAR first argument takes primitive types, got " + argumentOI.getTypeName()); + } + argumentOI = (PrimitiveObjectInspector) arguments[0]; + + PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory(); + ObjectInspector outputOI; + switch (inputType) { + case TIMESTAMP: + converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampObjectInspector); + Function timestampTransform = Transforms.hour().bind(Types.TimestampType.withoutZone()); + evaluator = arg -> { + TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get()); + result.set(timestampTransform.apply(val.getNanos() / 1000L)); + }; + break; + + case TIMESTAMPLOCALTZ: + converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector); + Function timestampLocalTzTransform = Transforms.hour().bind(Types.TimestampType.withZone()); + evaluator = arg -> { + TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get()); + result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L)); + }; + break; + + default: + throw new UDFArgumentException( + " ICEBERG_HOUR() only takes TIMESTAMP/TIMESTAMPLOCALTZ" + + " types as first argument, got " + inputType); + } + outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector; + return outputOI; + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + DeferredObject argument = arguments[0]; + if (argument == null || argument.get() == null) { + return null; + } else { + evaluator.apply(argument); + } + return result; + } + + @Override + public String getDisplayString(String[] children) { + return getStandardDisplayString("iceberg_hour", children); + } + +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java new file mode 100644 index 000000000000..196f5dc6d55d --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergMonth.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.udf; + +import java.util.function.Function; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; +import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.IntWritable; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; + +/** + * GenericUDFIcebergMonth - UDF that wraps around Iceberg's month transform function + */ +@Description(name = "iceberg_month", + value = "_FUNC_(value) - " + + "Returns the bucket value calculated by Iceberg month transform function ", + extended = "Example:\n > SELECT _FUNC_('2023-01-01');\n 1") +public class GenericUDFIcebergMonth extends GenericUDF { + private final IntWritable result = new IntWritable(); + private transient PrimitiveObjectInspector argumentOI; + private transient ObjectInspectorConverters.Converter converter; + + @FunctionalInterface + private interface UDFEvalFunction { + void apply(T argument) throws HiveException; + } + + private transient UDFEvalFunction evaluator; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 1) { + throw new UDFArgumentLengthException( + "ICEBERG_MONTH requires 1 arguments (value), but got " + arguments.length); + } + + if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw new UDFArgumentException( + "ICEBERG_MONTH first argument takes primitive types, got " + argumentOI.getTypeName()); + } + argumentOI = (PrimitiveObjectInspector) arguments[0]; + + PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory(); + ObjectInspector outputOI; + switch (inputType) { + case DATE: + converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableDateObjectInspector); + Function dateTransform = Transforms.month().bind(Types.DateType.get()); + evaluator = arg -> { + DateWritableV2 val = (DateWritableV2) converter.convert(arg.get()); + result.set(dateTransform.apply(val.getDays())); + }; + break; + + case TIMESTAMP: + converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampObjectInspector); + Function timestampTransform = Transforms.month().bind(Types.TimestampType.withoutZone()); + evaluator = arg -> { + TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get()); + result.set(timestampTransform.apply(val.getNanos() / 1000L)); + }; + break; + + case TIMESTAMPLOCALTZ: + converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector); + Function timestampLocalTzTransform = Transforms.month().bind(Types.TimestampType.withZone()); + evaluator = arg -> { + TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get()); + result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L)); + }; + break; + + default: + throw new UDFArgumentException( + " ICEBERG_MONTH() only takes DATE/TIMESTAMP/TIMESTAMPLOCALTZ" + + " types as first argument, got " + inputType); + } + outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector; + return outputOI; + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + DeferredObject argument = arguments[0]; + if (argument == null || argument.get() == null) { + return null; + } else { + evaluator.apply(argument); + } + return result; + } + + @Override + public String getDisplayString(String[] children) { + return getStandardDisplayString("iceberg_month", children); + } + +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java new file mode 100644 index 000000000000..bc0b63fefe8d --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergTruncate.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.udf; + +import java.math.BigDecimal; +import java.util.function.Function; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantIntObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +/** + * GenericUDFIcebergTruncate - UDF that wraps around Iceberg's truncate transform function + */ +@Description(name = "iceberg_truncate", + value = "_FUNC_(value, truncateLength) - " + + "Returns the bucket value calculated by Iceberg bucket transform function ", + extended = "Example:\n > SELECT _FUNC_('abcdefgh', 5);\n abcde") +public class GenericUDFIcebergTruncate extends GenericUDF { + private final Text result = new Text(); + private int truncateLength = 0; + private transient PrimitiveObjectInspector argumentOI; + private transient ObjectInspectorConverters.Converter converter; + + @FunctionalInterface + private interface UDFEvalFunction { + void apply(T argument) throws HiveException; + } + + private transient UDFEvalFunction evaluator; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 2) { + throw new UDFArgumentLengthException( + "ICEBERG_BUCKET requires 2 arguments (value, bucketCount), but got " + arguments.length); + } + + truncateLength = getTruncateLength(arguments[1]); + + if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw new UDFArgumentException( + "ICEBERG_BUCKET first argument takes primitive types, got " + argumentOI.getTypeName()); + } + argumentOI = (PrimitiveObjectInspector) arguments[0]; + + PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory(); + ObjectInspector outputOI; + switch (inputType) { + case CHAR: + case VARCHAR: + case STRING: + converter = new PrimitiveObjectInspectorConverter.StringConverter(argumentOI); + Function stringTransform = Transforms.truncate(truncateLength).bind(Types.StringType.get()); + evaluator = arg -> { + String val = (String) converter.convert(arg.get()); + result.set(String.valueOf(stringTransform.apply(val))); + }; + break; + + case INT: + converter = new PrimitiveObjectInspectorConverter.IntConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableIntObjectInspector); + Function intTransform = Transforms.truncate(truncateLength).bind(Types.IntegerType.get()); + evaluator = arg -> { + IntWritable val = (IntWritable) converter.convert(arg.get()); + result.set(String.valueOf(intTransform.apply(val.get()))); + }; + break; + + case LONG: + converter = new PrimitiveObjectInspectorConverter.LongConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableLongObjectInspector); + Function longTransform = Transforms.truncate(truncateLength).bind(Types.LongType.get()); + evaluator = arg -> { + LongWritable val = (LongWritable) converter.convert(arg.get()); + result.set(String.valueOf(longTransform.apply(val.get()))); + }; + break; + + case DECIMAL: + DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) TypeInfoUtils.getTypeInfoFromObjectInspector(argumentOI); + Type.PrimitiveType decimalIcebergType = Types.DecimalType.of(decimalTypeInfo.getPrecision(), + decimalTypeInfo.getScale()); + + converter = new PrimitiveObjectInspectorConverter.HiveDecimalConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector); + Function bigDecimalTransform = Transforms.truncate(truncateLength).bind(decimalIcebergType); + evaluator = arg -> { + HiveDecimalWritable val = (HiveDecimalWritable) converter.convert(arg.get()); + result.set(((BigDecimal) bigDecimalTransform.apply(val.getHiveDecimal().bigDecimalValue())).toPlainString()); + }; + break; + + case FLOAT: + converter = new PrimitiveObjectInspectorConverter.FloatConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableFloatObjectInspector); + Function floatTransform = Transforms.truncate(truncateLength).bind(Types.FloatType.get()); + evaluator = arg -> { + FloatWritable val = (FloatWritable) converter.convert(arg.get()); + result.set(String.valueOf(floatTransform.apply(val.get()))); + }; + break; + + case DOUBLE: + converter = new PrimitiveObjectInspectorConverter.DoubleConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableDoubleObjectInspector); + Function doubleTransform = Transforms.truncate(truncateLength).bind(Types.DoubleType.get()); + evaluator = arg -> { + DoubleWritable val = (DoubleWritable) converter.convert(arg.get()); + result.set(String.valueOf(doubleTransform.apply(val.get()))); + }; + break; + + default: + throw new UDFArgumentException( + " ICEBERG_TRUNCATE() only takes STRING/CHAR/VARCHAR/INT/LONG/DECIMAL/FLOAT/DOUBLE" + + " types as first argument, got " + inputType); + } + outputOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector; + return outputOI; + } + + private static int getTruncateLength(ObjectInspector arg) throws UDFArgumentException { + UDFArgumentException udfArgumentException = new UDFArgumentException("ICEBERG_TRUNCATE() second argument can " + + " only take an int type, but got " + arg.getTypeName()); + if (arg.getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw udfArgumentException; + } + PrimitiveObjectInspector.PrimitiveCategory inputType = ((PrimitiveObjectInspector) arg).getPrimitiveCategory(); + if (inputType != PrimitiveObjectInspector.PrimitiveCategory.INT) { + throw udfArgumentException; + } + return ((WritableConstantIntObjectInspector) arg).getWritableConstantValue().get(); + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + DeferredObject argument = arguments[0]; + if (argument == null || argument.get() == null) { + return null; + } else { + evaluator.apply(argument); + } + return result; + } + + @Override + public String getDisplayString(String[] children) { + return getStandardDisplayString("iceberg_truncate", children); + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java new file mode 100644 index 000000000000..19754bfc01d6 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/udf/GenericUDFIcebergYear.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.udf; + +import java.util.function.Function; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.io.DateWritableV2; +import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorConverter; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.IntWritable; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; + +/** + * GenericUDFIcebergYear - UDF that wraps around Iceberg's year transform function + */ +@Description(name = "iceberg_year", + value = "_FUNC_(value) - " + + "Returns the bucket value calculated by Iceberg year transform function ", + extended = "Example:\n > SELECT _FUNC_('2023-01-01');\n 2023") +public class GenericUDFIcebergYear extends GenericUDF { + private final IntWritable result = new IntWritable(); + private transient PrimitiveObjectInspector argumentOI; + private transient ObjectInspectorConverters.Converter converter; + + @FunctionalInterface + private interface UDFEvalFunction { + void apply(T argument) throws HiveException; + } + + private transient UDFEvalFunction evaluator; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + if (arguments.length != 1) { + throw new UDFArgumentLengthException( + "ICEBERG_YEAR requires 1 arguments (value), but got " + arguments.length); + } + + if (arguments[0].getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw new UDFArgumentException( + "ICEBERG_YEAR first argument takes primitive types, got " + argumentOI.getTypeName()); + } + argumentOI = (PrimitiveObjectInspector) arguments[0]; + + PrimitiveObjectInspector.PrimitiveCategory inputType = argumentOI.getPrimitiveCategory(); + ObjectInspector outputOI; + switch (inputType) { + case DATE: + converter = new PrimitiveObjectInspectorConverter.DateConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableDateObjectInspector); + Function dateTransform = Transforms.year().bind(Types.DateType.get()); + evaluator = arg -> { + DateWritableV2 val = (DateWritableV2) converter.convert(arg.get()); + result.set(dateTransform.apply(val.getDays())); + }; + break; + + case TIMESTAMP: + converter = new PrimitiveObjectInspectorConverter.TimestampConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampObjectInspector); + Function timestampTransform = Transforms.year().bind(Types.TimestampType.withoutZone()); + evaluator = arg -> { + TimestampWritableV2 val = (TimestampWritableV2) converter.convert(arg.get()); + result.set(timestampTransform.apply(val.getNanos() / 1000L)); + }; + break; + + case TIMESTAMPLOCALTZ: + converter = new PrimitiveObjectInspectorConverter.TimestampLocalTZConverter(argumentOI, + PrimitiveObjectInspectorFactory.writableTimestampTZObjectInspector); + Function timestampLocalTzTransform = Transforms.year().bind(Types.TimestampType.withZone()); + evaluator = arg -> { + TimestampLocalTZWritable val = (TimestampLocalTZWritable) converter.convert(arg.get()); + result.set(timestampLocalTzTransform.apply(val.getNanos() / 1000L)); + }; + break; + + default: + throw new UDFArgumentException( + " ICEBERG_YEAR() only takes DATE/TIMESTAMP/TIMESTAMPLOCALTZ" + + " types as first argument, got " + inputType); + } + outputOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector; + return outputOI; + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + DeferredObject argument = arguments[0]; + if (argument == null || argument.get() == null) { + return null; + } else { + evaluator.apply(argument); + } + return result; + } + + @Override + public String getDisplayString(String[] children) { + return getStandardDisplayString("iceberg_year", children); + } + +} diff --git a/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q b/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q index 85063e2b095a..54e46a2a2609 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/dynamic_partition_writes.q @@ -8,6 +8,16 @@ drop table if exists tbl_target_identity; drop table if exists tbl_target_bucket; drop table if exists tbl_target_mixed; drop table if exists tbl_bucket_date; +drop table if exists tbl_target_truncate_str; +drop table if exists tbl_target_truncate_int; +drop table if exists tbl_target_truncate_bigint; +drop table if exists tbl_year_date; +drop table if exists tbl_year_timestamp; +drop table if exists tbl_month_date; +drop table if exists tbl_month_timestamp; +drop table if exists tbl_day_date; +drop table if exists tbl_day_timestamp; +drop table if exists tbl_hour_timestamp; create external table tbl_src (a int, b string, c bigint) stored by iceberg stored as orc; insert into tbl_src values (1, 'EUR', 10), (2, 'EUR', 10), (3, 'USD', 11), (4, 'EUR', 12), (5, 'HUF', 30), (6, 'USD', 10), (7, 'USD', 100), (8, 'PLN', 20), (9, 'PLN', 11), (10, 'CZK', 5), (12, NULL, NULL); @@ -53,4 +63,92 @@ tblproperties ('parquet.compression'='snappy','format-version'='2'); insert into tbl_bucket_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); update tbl_bucket_date set date_time_date = '2018-07-02' where date_time_date = '2018-07-03'; -select count(*) from tbl_bucket_date where date_time_date = '2018-07-02'; +select count(*) from tbl_bucket_date where date_time_date = '2018-07-02'; + +--truncate case - should invoke GenericUDFIcebergTruncate to truncate the column value and use for clustering and sorting +create external table tbl_target_truncate_str (a int, ccy string) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc; +explain insert into table tbl_target_truncate_str select a, b from tbl_src; +insert into table tbl_target_truncate_str select a, b from tbl_src; +select * from tbl_target_truncate_str order by a, ccy; + +create external table tbl_target_truncate_int (id int, ccy string) partitioned by spec (truncate(2, id)) stored by iceberg stored as orc; +explain insert into table tbl_target_truncate_int select a, b from tbl_src; +insert into table tbl_target_truncate_int select a, b from tbl_src; +select * from tbl_target_truncate_int order by id, ccy; + +create external table tbl_target_truncate_bigint (a int, ccy bigint) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc; +explain insert into table tbl_target_truncate_bigint select a, c from tbl_src; +insert into table tbl_target_truncate_bigint select a, c from tbl_src; +select * from tbl_target_truncate_bigint order by a, ccy; + +create external table tbl_target_truncate_decimal (a int, b string, ccy decimal(10,6)) partitioned by spec (truncate(2, b), truncate(3, ccy)) stored by iceberg stored as orc; +explain insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src; +insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src; +select * from tbl_target_truncate_decimal order by a, b; + +--year case - should invoke GenericUDFIcebergYear to convert the date/timestamp value to year and use for clustering and sorting +create external table tbl_year_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, year(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); +insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); +select * from tbl_year_date order by id, date_time_date; + +create external table tbl_year_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, year(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +select * from tbl_year_timestamp order by id, date_time_timestamp; + +--month case - should invoke GenericUDFIcebergMonth to convert the date/timestamp value to month and use for clustering and sorting +create external table tbl_month_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, month(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); +insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); +select * from tbl_month_date order by id, date_time_date; + +create external table tbl_month_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, month(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +select * from tbl_month_timestamp order by id, date_time_timestamp; + +--day case - should invoke GenericUDFIcebergMonth to convert the date/timestamp value to day and use for clustering and sorting +create external table tbl_day_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, day(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); +insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018); +select * from tbl_day_date order by id, date_time_date; + +create external table tbl_day_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, day(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +select * from tbl_day_timestamp order by id, date_time_timestamp; + +--hour case - should invoke GenericUDFIcebergMonth to convert the date/timestamp value to day and use for clustering and sorting +create external table tbl_hour_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, hour(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2'); + +explain insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018); +select * from tbl_hour_timestamp order by id, date_time_timestamp; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out index 6206f479aac6..e5fe8e90db7d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out @@ -432,10 +432,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 3 Data size: 903 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 19 @@ -663,18 +663,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 8 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -1034,18 +1034,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 6 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -1106,10 +1106,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 9 @@ -1486,18 +1486,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 6 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -1558,10 +1558,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 2 Data size: 602 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 9 diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out index a05ebf9af733..b7690c5579ff 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out @@ -28,6 +28,66 @@ PREHOOK: Output: database:default POSTHOOK: query: drop table if exists tbl_bucket_date POSTHOOK: type: DROPTABLE POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_target_truncate_str +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_target_truncate_str +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_target_truncate_int +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_target_truncate_int +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_target_truncate_bigint +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_target_truncate_bigint +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_year_date +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_year_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_year_timestamp +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_year_timestamp +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_month_date +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_month_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_month_timestamp +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_month_timestamp +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_day_date +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_day_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_day_timestamp +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_day_timestamp +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: drop table if exists tbl_hour_timestamp +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_hour_timestamp +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default PREHOOK: query: create external table tbl_src (a int, b string, c bigint) stored by iceberg stored as orc PREHOOK: type: CREATETABLE PREHOOK: Output: database:default @@ -606,3 +666,974 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_bucket_date POSTHOOK: Output: hdfs://### HDFS PATH ### 1 +PREHOOK: query: create external table tbl_target_truncate_str (a int, ccy string) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_target_truncate_str +POSTHOOK: query: create external table tbl_target_truncate_str (a int, ccy string) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_target_truncate_str +PREHOOK: query: explain insert into table tbl_target_truncate_str select a, b from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_str +POSTHOOK: query: explain insert into table tbl_target_truncate_str select a, b from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_str +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_target_truncate_str"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_18] + table:{"name:":"default.tbl_target_truncate_str"} + Select Operator [SEL_17] + Output:["_col0","_col1","iceberg_truncate(_col1, 2)"] + <-Map 1 [SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:iceberg_truncate(_col1, 2) + Select Operator [SEL_12] (rows=22 width=87) + Output:["_col0","_col1"] + TableScan [TS_0] (rows=22 width=87) + default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] + Reducer 3 vectorized + File Output Operator [FS_21] + Select Operator [SEL_20] (rows=1 width=530) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] + Group By Operator [GBY_19] (rows=1 width=332) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_16] + Group By Operator [GBY_15] (rows=1 width=400) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_14] (rows=22 width=87) + Output:["a","ccy"] + Please refer to the previous Select Operator [SEL_12] + +PREHOOK: query: insert into table tbl_target_truncate_str select a, b from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_str +POSTHOOK: query: insert into table tbl_target_truncate_str select a, b from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_str +PREHOOK: query: select * from tbl_target_truncate_str order by a, ccy +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_target_truncate_str +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_target_truncate_str order by a, ccy +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_target_truncate_str +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 EUR +2 EUR +3 USD +4 EUR +5 HUF +6 USD +7 USD +8 PLN +9 PLN +10 CZK +10 EUR +12 NULL +20 EUR +30 USD +40 EUR +50 HUF +60 USD +70 USD +80 PLN +90 PLN +100 CZK +110 NULL +PREHOOK: query: create external table tbl_target_truncate_int (id int, ccy string) partitioned by spec (truncate(2, id)) stored by iceberg stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_target_truncate_int +POSTHOOK: query: create external table tbl_target_truncate_int (id int, ccy string) partitioned by spec (truncate(2, id)) stored by iceberg stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_target_truncate_int +PREHOOK: query: explain insert into table tbl_target_truncate_int select a, b from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_int +POSTHOOK: query: explain insert into table tbl_target_truncate_int select a, b from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_int +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_target_truncate_int"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_18] + table:{"name:":"default.tbl_target_truncate_int"} + Select Operator [SEL_17] + Output:["_col0","_col1","iceberg_truncate(_col0, 2)"] + <-Map 1 [SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:iceberg_truncate(_col0, 2) + Select Operator [SEL_12] (rows=22 width=87) + Output:["_col0","_col1"] + TableScan [TS_0] (rows=22 width=87) + default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] + Reducer 3 vectorized + File Output Operator [FS_21] + Select Operator [SEL_20] (rows=1 width=530) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] + Group By Operator [GBY_19] (rows=1 width=332) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_16] + Group By Operator [GBY_15] (rows=1 width=400) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(id)","max(id)","count(1)","count(id)","compute_bit_vector_hll(id)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_14] (rows=22 width=87) + Output:["id","ccy"] + Please refer to the previous Select Operator [SEL_12] + +PREHOOK: query: insert into table tbl_target_truncate_int select a, b from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_int +POSTHOOK: query: insert into table tbl_target_truncate_int select a, b from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_int +PREHOOK: query: select * from tbl_target_truncate_int order by id, ccy +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_target_truncate_int +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_target_truncate_int order by id, ccy +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_target_truncate_int +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 EUR +2 EUR +3 USD +4 EUR +5 HUF +6 USD +7 USD +8 PLN +9 PLN +10 CZK +10 EUR +12 NULL +20 EUR +30 USD +40 EUR +50 HUF +60 USD +70 USD +80 PLN +90 PLN +100 CZK +110 NULL +PREHOOK: query: create external table tbl_target_truncate_bigint (a int, ccy bigint) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_target_truncate_bigint +POSTHOOK: query: create external table tbl_target_truncate_bigint (a int, ccy bigint) partitioned by spec (truncate(2, ccy)) stored by iceberg stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_target_truncate_bigint +PREHOOK: query: explain insert into table tbl_target_truncate_bigint select a, c from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_bigint +POSTHOOK: query: explain insert into table tbl_target_truncate_bigint select a, c from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_bigint +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_target_truncate_bigint"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_18] + table:{"name:":"default.tbl_target_truncate_bigint"} + Select Operator [SEL_17] + Output:["_col0","_col1","iceberg_truncate(_col1, 2)"] + <-Map 1 [SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:iceberg_truncate(_col1, 2) + Select Operator [SEL_12] (rows=22 width=11) + Output:["_col0","_col1"] + TableScan [TS_0] (rows=22 width=11) + default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","c"] + Reducer 3 vectorized + File Output Operator [FS_21] + Select Operator [SEL_20] (rows=1 width=528) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] + Group By Operator [GBY_19] (rows=1 width=336) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_16] + Group By Operator [GBY_15] (rows=1 width=336) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_14] (rows=22 width=11) + Output:["a","ccy"] + Please refer to the previous Select Operator [SEL_12] + +PREHOOK: query: insert into table tbl_target_truncate_bigint select a, c from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_bigint +POSTHOOK: query: insert into table tbl_target_truncate_bigint select a, c from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_bigint +PREHOOK: query: select * from tbl_target_truncate_bigint order by a, ccy +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_target_truncate_bigint +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_target_truncate_bigint order by a, ccy +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_target_truncate_bigint +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 10 +2 10 +3 11 +4 12 +5 30 +6 10 +7 100 +8 20 +9 11 +10 5 +10 12 +12 NULL +20 11 +30 100 +40 10 +50 30 +60 12 +70 20 +80 100 +90 18 +100 12 +110 NULL +PREHOOK: query: create external table tbl_target_truncate_decimal (a int, b string, ccy decimal(10,6)) partitioned by spec (truncate(2, b), truncate(3, ccy)) stored by iceberg stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_target_truncate_decimal +POSTHOOK: query: create external table tbl_target_truncate_decimal (a int, b string, ccy decimal(10,6)) partitioned by spec (truncate(2, b), truncate(3, ccy)) stored by iceberg stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_target_truncate_decimal +PREHOOK: query: explain insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_decimal +POSTHOOK: query: explain insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_decimal +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_target_truncate_decimal"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_19] + table:{"name:":"default.tbl_target_truncate_decimal"} + Select Operator [SEL_18] + Output:["_col0","_col1","_col2","iceberg_truncate(_col1, 2)","iceberg_truncate(_col2, 3)"] + <-Map 1 [SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_14] + PartitionCols:iceberg_truncate(_col1, 2), iceberg_truncate(_col2, 3) + Select Operator [SEL_13] (rows=22 width=199) + Output:["_col0","_col1","_col2"] + TableScan [TS_0] (rows=22 width=87) + default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] + Reducer 3 vectorized + File Output Operator [FS_22] + Select Operator [SEL_21] (rows=1 width=1005) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_20] (rows=1 width=708) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized + PARTITION_ONLY_SHUFFLE [RS_17] + Group By Operator [GBY_16] (rows=1 width=776) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(b))","avg(COALESCE(length(b),0))","count(b)","compute_bit_vector_hll(b)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_15] (rows=22 width=199) + Output:["a","b","ccy"] + Please refer to the previous Select Operator [SEL_13] + +PREHOOK: query: insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_src +PREHOOK: Output: default@tbl_target_truncate_decimal +POSTHOOK: query: insert into table tbl_target_truncate_decimal select a, b, 1.567894 from tbl_src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_src +POSTHOOK: Output: default@tbl_target_truncate_decimal +PREHOOK: query: select * from tbl_target_truncate_decimal order by a, b +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_target_truncate_decimal +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_target_truncate_decimal order by a, b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_target_truncate_decimal +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 EUR 1.567894 +2 EUR 1.567894 +3 USD 1.567894 +4 EUR 1.567894 +5 HUF 1.567894 +6 USD 1.567894 +7 USD 1.567894 +8 PLN 1.567894 +9 PLN 1.567894 +10 CZK 1.567894 +10 EUR 1.567894 +12 NULL 1.567894 +20 EUR 1.567894 +30 USD 1.567894 +40 EUR 1.567894 +50 HUF 1.567894 +60 USD 1.567894 +70 USD 1.567894 +80 PLN 1.567894 +90 PLN 1.567894 +100 CZK 1.567894 +110 NULL 1.567894 +PREHOOK: query: create external table tbl_year_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, year(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_year_date +POSTHOOK: query: create external table tbl_year_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, year(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_year_date +PREHOOK: query: explain insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_year_date +POSTHOOK: query: explain insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_year_date +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_year_date"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_year_date"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_year(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_year(_col1) + Select Operator [SEL_3] (rows=1 width=240) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=890) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=596) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=664) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=240) + Output:["id","date_time_date","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_year_date +POSTHOOK: query: insert into tbl_year_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_year_date +PREHOOK: query: select * from tbl_year_date order by id, date_time_date +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_year_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_year_date order by id, date_time_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_year_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 2018 +40568 2018-07-03 2018 +88669 2018-05-27 2018 +PREHOOK: query: create external table tbl_year_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, year(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_year_timestamp +POSTHOOK: query: create external table tbl_year_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, year(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_year_timestamp +PREHOOK: query: explain insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_year_timestamp +POSTHOOK: query: explain insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_year_timestamp +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_year_timestamp"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_year_timestamp"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_year(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_year(_col1) + Select Operator [SEL_3] (rows=1 width=224) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=863) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=564) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=632) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=224) + Output:["id","date_time_timestamp","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_year_timestamp +POSTHOOK: query: insert into tbl_year_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_year_timestamp +PREHOOK: query: select * from tbl_year_timestamp order by id, date_time_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_year_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_year_timestamp order by id, date_time_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_year_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 12:45:56 2018 +40568 2018-07-03 06:07:56 2018 +88669 2018-05-27 11:12:00 2018 +PREHOOK: query: create external table tbl_month_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, month(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_month_date +POSTHOOK: query: create external table tbl_month_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, month(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_month_date +PREHOOK: query: explain insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_month_date +POSTHOOK: query: explain insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_month_date +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_month_date"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_month_date"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_month(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_month(_col1) + Select Operator [SEL_3] (rows=1 width=240) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=890) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=596) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=664) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=240) + Output:["id","date_time_date","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_month_date +POSTHOOK: query: insert into tbl_month_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_month_date +PREHOOK: query: select * from tbl_month_date order by id, date_time_date +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_month_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_month_date order by id, date_time_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_month_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 2018 +40568 2018-07-03 2018 +88669 2018-05-27 2018 +PREHOOK: query: create external table tbl_month_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, month(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_month_timestamp +POSTHOOK: query: create external table tbl_month_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, month(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_month_timestamp +PREHOOK: query: explain insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_month_timestamp +POSTHOOK: query: explain insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_month_timestamp +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_month_timestamp"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_month_timestamp"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_month(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_month(_col1) + Select Operator [SEL_3] (rows=1 width=224) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=863) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=564) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=632) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=224) + Output:["id","date_time_timestamp","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_month_timestamp +POSTHOOK: query: insert into tbl_month_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_month_timestamp +PREHOOK: query: select * from tbl_month_timestamp order by id, date_time_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_month_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_month_timestamp order by id, date_time_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_month_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 12:45:56 2018 +40568 2018-07-03 06:07:56 2018 +88669 2018-05-27 11:12:00 2018 +PREHOOK: query: create external table tbl_day_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, day(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_day_date +POSTHOOK: query: create external table tbl_day_date (id string, date_time_date date, year_partition int) + partitioned by spec (year_partition, day(date_time_date)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_day_date +PREHOOK: query: explain insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_day_date +POSTHOOK: query: explain insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_day_date +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_day_date"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_day_date"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_day(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_day(_col1) + Select Operator [SEL_3] (rows=1 width=240) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=890) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=596) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=664) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=240) + Output:["id","date_time_date","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_day_date +POSTHOOK: query: insert into tbl_day_date values (88669, '2018-05-27', 2018), (40568, '2018-02-12', 2018), (40568, '2018-07-03', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_day_date +PREHOOK: query: select * from tbl_day_date order by id, date_time_date +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_day_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_day_date order by id, date_time_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_day_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 2018 +40568 2018-07-03 2018 +88669 2018-05-27 2018 +PREHOOK: query: create external table tbl_day_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, day(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_day_timestamp +POSTHOOK: query: create external table tbl_day_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, day(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_day_timestamp +PREHOOK: query: explain insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_day_timestamp +POSTHOOK: query: explain insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_day_timestamp +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_day_timestamp"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_day_timestamp"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_day(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_day(_col1) + Select Operator [SEL_3] (rows=1 width=224) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=863) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=564) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=632) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=224) + Output:["id","date_time_timestamp","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_day_timestamp +POSTHOOK: query: insert into tbl_day_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_day_timestamp +PREHOOK: query: select * from tbl_day_timestamp order by id, date_time_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_day_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_day_timestamp order by id, date_time_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_day_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 12:45:56 2018 +40568 2018-07-03 06:07:56 2018 +88669 2018-05-27 11:12:00 2018 +PREHOOK: query: create external table tbl_hour_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, hour(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_hour_timestamp +POSTHOOK: query: create external table tbl_hour_timestamp (id string, date_time_timestamp timestamp, year_partition int) + partitioned by spec (year_partition, hour(date_time_timestamp)) +stored by iceberg stored as parquet +tblproperties ('parquet.compression'='snappy','format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_hour_timestamp +PREHOOK: query: explain insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_hour_timestamp +POSTHOOK: query: explain insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_hour_timestamp +Plan optimized by CBO. + +Vertex dependency in root stage +Reducer 2 <- Map 1 (SIMPLE_EDGE) +Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) + +Stage-3 + Stats Work{} + Stage-0 + Move Operator + table:{"name:":"default.tbl_hour_timestamp"} + Stage-2 + Dependency Collection{} + Stage-1 + Reducer 2 vectorized + File Output Operator [FS_16] + table:{"name:":"default.tbl_hour_timestamp"} + Select Operator [SEL_15] + Output:["_col0","_col1","_col2","_col2","iceberg_hour(_col1)"] + <-Map 1 [SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_13] + PartitionCols:_col2, iceberg_hour(_col1) + Select Operator [SEL_3] (rows=1 width=224) + Output:["_col0","_col1","_col2"] + UDTF Operator [UDTF_2] (rows=1 width=64) + function name:inline + Select Operator [SEL_1] (rows=1 width=64) + Output:["_col0"] + TableScan [TS_0] (rows=1 width=10) + _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE + Reducer 3 vectorized + File Output Operator [FS_19] + Select Operator [SEL_18] (rows=1 width=863) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] + Group By Operator [GBY_17] (rows=1 width=564) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] + <-Map 1 [CUSTOM_SIMPLE_EDGE] + PARTITION_ONLY_SHUFFLE [RS_9] + Group By Operator [GBY_8] (rows=1 width=632) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_7] (rows=1 width=224) + Output:["id","date_time_timestamp","year_partition"] + Please refer to the previous Select Operator [SEL_3] + +PREHOOK: query: insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_hour_timestamp +POSTHOOK: query: insert into tbl_hour_timestamp values (88669, '2018-05-27 11:12:00', 2018), (40568, '2018-02-12 12:45:56', 2018), (40568, '2018-07-03 06:07:56', 2018) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_hour_timestamp +PREHOOK: query: select * from tbl_hour_timestamp order by id, date_time_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_hour_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_hour_timestamp order by id, date_time_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_hour_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +40568 2018-02-12 12:45:56 2018 +40568 2018-07-03 06:07:56 2018 +88669 2018-05-27 11:12:00 2018 diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out index 1227db6101d0..308d2a8d62a4 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out @@ -228,18 +228,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 302 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 4 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -322,10 +322,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 8 @@ -379,10 +379,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 598 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 7 Data size: 2096 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Union 3 @@ -511,18 +511,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 9 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 3 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out index f8cc1fa15596..f20856fd291c 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out @@ -162,10 +162,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string) + key expressions: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string) + Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) Filter Operator @@ -176,10 +176,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string) + key expressions: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), _col1 (type: string) + Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) Filter Operator @@ -237,8 +237,8 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY._col1 (type: string) - outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), _col1 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY.iceberg_truncate(_col1, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), iceberg_truncate(_col1, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -252,8 +252,8 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY._col1 (type: string) - outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), _col1 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), KEY.iceberg_bucket(_col0, 16) (type: int), KEY.iceberg_truncate(_col1, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, iceberg_bucket(_col0, 16), iceberg_truncate(_col1, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out index 985c32f342ea..7a5b872fa115 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out @@ -60,10 +60,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 4 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Execution mode: vectorized @@ -118,8 +118,8 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -144,10 +144,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 7 @@ -184,10 +184,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 8 @@ -605,10 +605,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 14 @@ -809,10 +809,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 9 Data size: 2781 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 23 @@ -1085,18 +1085,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 8 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -1524,18 +1524,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 6 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -1596,10 +1596,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 9 @@ -1617,10 +1617,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Union 5 @@ -1946,10 +1946,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 2 @@ -2030,18 +2030,18 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 308 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 5 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY._col5 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), _col5 + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int), KEY.iceberg_bucket(_col4, 16) (type: int), KEY.iceberg_truncate(_col5, 3) (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, iceberg_bucket(_col4, 16), iceberg_truncate(_col5, 3) File Output Operator compressed: false Dp Sort State: PARTITION_SORTED @@ -2128,10 +2128,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 301 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) null sort order: aa sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), _col5 (type: string) + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Union 4 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java index 871980a64464..1f411971a806 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java @@ -780,7 +780,17 @@ public final class FunctionRegistry { try { system.registerGenericUDF("iceberg_bucket", - (Class) Class.forName("org.apache.iceberg.mr.hive.GenericUDFIcebergBucket")); + (Class) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergBucket")); + system.registerGenericUDF("iceberg_truncate", + (Class) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergTruncate")); + system.registerGenericUDF("iceberg_year", + (Class) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergYear")); + system.registerGenericUDF("iceberg_month", + (Class) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergMonth")); + system.registerGenericUDF("iceberg_day", + (Class) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergDay")); + system.registerGenericUDF("iceberg_hour", + (Class) Class.forName("org.apache.iceberg.mr.hive.udf.GenericUDFIcebergHour")); } catch (ClassNotFoundException e) { LOG.warn("iceberg_bucket function could not be registered"); } From 1e4f488394d19ea51766e0633a605e078d8558c3 Mon Sep 17 00:00:00 2001 From: Shivangi Jha Date: Wed, 6 Dec 2023 09:58:12 +0530 Subject: [PATCH 081/179] HIVE-27894: Enhance HMS Handler Logs for all 'get_partition' functions (Shivangi Jha, Reviewed by Chinna Rao Lalam) --- .../hadoop/hive/metastore/HMSHandler.java | 92 +++++++++++++++---- 1 file changed, 75 insertions(+), 17 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index dfabec77ac5f..1399553e40bb 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -151,6 +151,12 @@ public class HMSHandler extends FacebookBase implements IHMSHandler { static final String NO_FILTER_STRING = ""; static final int UNLIMITED_MAX_PARTITIONS = -1; + static final int LOG_SAMPLE_PARTITIONS_MAX_SIZE = 4; + + static final int LOG_SAMPLE_PARTITIONS_HALF_SIZE = 2; + + static final String LOG_SAMPLE_PARTITIONS_SEPARATOR = ","; + private Warehouse wh; // hdfs warehouse private static Striped tablelocks; @@ -815,7 +821,7 @@ private void startMultiTableFunction(String function, String db, List tb private void startPartitionFunction(String function, String cat, String db, String tbl, List partVals) { startFunction(function, " : tbl=" + - TableName.getQualified(cat, db, tbl) + "[" + join(partVals, ",") + "]"); + TableName.getQualified(cat, db, tbl) + samplePartitionValues(partVals)); } private void startPartitionFunction(String function, String catName, String db, String tbl, @@ -824,6 +830,52 @@ private void startPartitionFunction(String function, String catName, String db, TableName.getQualified(catName, db, tbl) + " partition=" + partName); } + private void startPartitionFunction(String function, String catName, String db, String tbl, int maxParts) { + startFunction(function, " : tbl=" + TableName.getQualified(catName, db, tbl) + ": Max partitions =" + maxParts); + } + + private void startPartitionFunction(String function, String catName, String db, String tbl, int maxParts, + List partVals) { + startFunction(function, " : tbl=" + TableName.getQualified(catName, db, tbl) + ": Max partitions =" + maxParts + + samplePartitionValues(partVals)); + } + + private void startPartitionFunction(String function, String catName, String db, String tbl, int maxParts, + String filter) { + startFunction(function, + " : tbl=" + TableName.getQualified(catName, db, tbl) + ": Filter=" + filter + ": Max partitions =" + + maxParts); + } + + private void startPartitionFunction(String function, String catName, String db, String tbl, int maxParts, + String expression, String defaultPartitionName) { + startFunction(function, " : tbl=" + TableName.getQualified(catName, db, tbl) + ": Expression=" + expression + + ": Default partition name=" + defaultPartitionName + ": Max partitions=" + maxParts); + } + + private String getGroupsCountAndUsername(final String user_name, final List group_names) { + return ". Number of groups= " + (group_names == null ? 0 : group_names.size()) + ", user name= " + user_name; + } + + private String samplePartitionValues(List partVals) { + if (CollectionUtils.isEmpty(partVals)) { + return ": Partitions = []"; + } + StringBuilder sb = new StringBuilder(": Number of Partitions = " + partVals.size()); + sb.append(": Partitions = ["); + if (partVals.size() > LOG_SAMPLE_PARTITIONS_MAX_SIZE) { + // extracting starting 2 values, and ending 2 values + sb.append(join(partVals.subList(0, LOG_SAMPLE_PARTITIONS_HALF_SIZE), LOG_SAMPLE_PARTITIONS_SEPARATOR)); + sb.append(" .... "); + sb.append(join(partVals.subList(partVals.size() - LOG_SAMPLE_PARTITIONS_HALF_SIZE, partVals.size()), + LOG_SAMPLE_PARTITIONS_SEPARATOR)); + } else { + sb.append(join(partVals, LOG_SAMPLE_PARTITIONS_SEPARATOR)); + } + sb.append("]"); + return sb.toString(); + } + private void endFunction(String function, boolean successful, Exception e) { endFunction(function, successful, e, null); } @@ -5453,8 +5505,9 @@ public Partition get_partition_with_auth(final String db_name, final String user_name, final List group_names) throws TException { String[] parsedDbName = parseDbName(db_name, conf); - startPartitionFunction("get_partition_with_auth", parsedDbName[CAT_NAME], - parsedDbName[DB_NAME], tbl_name, part_vals); + startFunction("get_partition_with_auth", + " : tbl=" + TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name) + + samplePartitionValues(part_vals) + getGroupsCountAndUsername(user_name,group_names)); fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); Partition ret = null; Exception ex = null; @@ -5592,7 +5645,7 @@ public List get_partitions_pspec(final String db_name, final Stri String dbName = parsedDbName[DB_NAME]; String tableName = tbl_name.toLowerCase(); - startTableFunction("get_partitions_pspec", catName, dbName, tableName); + startPartitionFunction("get_partitions_pspec", catName, dbName, tableName, max_parts); List partitionSpecs = null; try { @@ -5678,7 +5731,7 @@ private static boolean is_partition_spec_grouping_enabled(Table table) { public List get_partition_names(final String db_name, final String tbl_name, final short max_parts) throws NoSuchObjectException, MetaException { String[] parsedDbName = parseDbName(db_name, conf); - startTableFunction("get_partition_names", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); + startPartitionFunction("get_partition_names", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, max_parts); fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); List ret = null; Exception ex = null; @@ -5703,7 +5756,9 @@ public PartitionValuesResponse get_partition_values(PartitionValuesRequest reque String catName = request.isSetCatName() ? request.getCatName() : getDefaultCatalog(conf); String dbName = request.getDbName(); String tblName = request.getTblName(); - + long maxParts = request.getMaxParts(); + String filter = request.isSetFilter() ? request.getFilter() : ""; + startPartitionFunction("get_partition_values", catName, dbName, tblName, (int) maxParts, filter); try { authorizeTableForPartitionMetadata(catName, dbName, tblName); @@ -6562,8 +6617,8 @@ public List get_partitions_ps(final String db_name, final String tbl_name, final List part_vals, final short max_parts) throws TException { String[] parsedDbName = parseDbName(db_name, conf); - startPartitionFunction("get_partitions_ps", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], - tbl_name, part_vals); + startPartitionFunction("get_partitions_ps", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, max_parts, + part_vals); List ret = null; Exception ex = null; @@ -6655,8 +6710,8 @@ public List get_partition_names_ps(final String db_name, final String tbl_name, final List part_vals, final short max_parts) throws TException { String[] parsedDbName = parseDbName(db_name, conf); - startPartitionFunction("get_partitions_names_ps", parsedDbName[CAT_NAME], - parsedDbName[DB_NAME], tbl_name, part_vals); + startPartitionFunction("get_partitions_names_ps", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, + max_parts, part_vals); fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name); List ret = null; Exception ex = null; @@ -6847,8 +6902,7 @@ public PartitionsStatsResult get_partitions_statistics_req(PartitionsStatsReques String catName = request.isSetCatName() ? request.getCatName().toLowerCase() : getDefaultCatalog(conf); String dbName = request.getDbName().toLowerCase(); String tblName = request.getTblName().toLowerCase(); - startFunction("get_partitions_statistics_req", ": table=" + - TableName.getQualified(catName, dbName, tblName)); + startPartitionFunction("get_partitions_statistics_req", catName, dbName, tblName, request.getPartNames()); PartitionsStatsResult result = null; List lowerCaseColNames = new ArrayList<>(request.getColNames().size()); @@ -7264,8 +7318,8 @@ public List get_part_specs_by_filter(final String dbName, final S throws TException { String[] parsedDbName = parseDbName(dbName, conf); - startTableFunction("get_partitions_by_filter_pspec", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName); - + startPartitionFunction("get_partitions_by_filter_pspec", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName, + maxParts, filter); List partitionSpecs = null; try { Table table = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName); @@ -7330,7 +7384,10 @@ public PartitionsByExprResult get_partitions_by_expr( PartitionsByExprRequest req) throws TException { String dbName = req.getDbName(), tblName = req.getTblName(); String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf); - startTableFunction("get_partitions_by_expr", catName, dbName, tblName); + String expr = req.isSetExpr() ? Arrays.toString((req.getExpr())) : ""; + String defaultPartitionName = req.isSetDefaultPartitionName() ? req.getDefaultPartitionName() : ""; + int maxParts = req.getMaxParts(); + startPartitionFunction("get_partitions_by_expr", catName, dbName, tblName, maxParts, expr, defaultPartitionName); fireReadTablePreEvent(catName, dbName, tblName); PartitionsByExprResult ret = null; Exception ex = null; @@ -7363,8 +7420,9 @@ public int get_num_partitions_by_filter(final String dbName, if (parsedDbName[DB_NAME] == null || tblName == null) { throw new MetaException("The DB and table name cannot be null."); } - startTableFunction("get_num_partitions_by_filter", parsedDbName[CAT_NAME], - parsedDbName[DB_NAME], tblName); + startFunction("get_num_partitions_by_filter", + " : tbl=" + TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName) + " Filter=" + + filter); int ret = -1; Exception ex = null; From 6af511e53dfa71f3e5c80b4abab210da29033e57 Mon Sep 17 00:00:00 2001 From: tarak271 Date: Wed, 6 Dec 2023 18:37:21 +0530 Subject: [PATCH 082/179] HIVE-27797: Transactions that got timed out are not getting logged as 'ABORTED' in NOTIFICATION_LOG (Taraka Rama Rao Lethavadla, reviewed by Denys Kuzmenko) Closes #4805 --- .../TestTimedOutTxnNotificationLogging.java | 201 ++++++++++++++++++ .../hadoop/hive/metastore/txn/TxnHandler.java | 27 ++- 2 files changed, 220 insertions(+), 8 deletions(-) create mode 100644 itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java new file mode 100644 index 000000000000..acad300c6817 --- /dev/null +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.parse; + +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.common.repl.ReplScope; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetastoreTaskThread; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage; +import org.apache.hadoop.hive.metastore.messaging.MessageBuilder; +import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer; +import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage; +import org.apache.hadoop.hive.metastore.messaging.event.filters.AndFilter; +import org.apache.hadoop.hive.metastore.messaging.event.filters.CatalogFilter; +import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter; +import org.apache.hadoop.hive.metastore.messaging.event.filters.ReplEventFilter; +import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.AcidTxnCleanerService; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; +import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.events.EventUtils; +import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.util.StringUtils; +import org.apache.hive.hcatalog.listener.DbNotificationListener; +import org.apache.thrift.TException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static junit.framework.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class TestTimedOutTxnNotificationLogging { + + private HiveConf hiveConf; + + private static IMetaStoreClient hive; + + @Parameterized.Parameter + public int numberOfTxns; + + @Parameterized.Parameter(1) + public TxnType txnType; + + @Parameterized.Parameter(2) + public int expectedNotifications; + + @Parameterized.Parameters(name = "{index}: numberOfTxns={0},txnType={1},expectedNotifications={2}") + public static Collection data() { + return Arrays.asList( + new Object[][] { { 3, TxnType.REPL_CREATED, 3 }, { 3, TxnType.DEFAULT, 3 }, { 3, TxnType.READ_ONLY, 0 } }); + } + + @Before + public void setUp() throws Exception { + setConf(); + TestTxnDbUtil.prepDb(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + hive = new HiveMetaStoreClient(hiveConf); + } + + private void setConf() { + hiveConf = new HiveConf(); + MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.HIVE_IN_TEST, true); + MetastoreConf.setVar(hiveConf, MetastoreConf.ConfVars.WAREHOUSE, "/tmp"); + MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.TXN_TIMEOUT, 1000, TimeUnit.MILLISECONDS); + MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.EVENT_DB_LISTENER_TTL, 1, TimeUnit.SECONDS); + HiveConf.setVar(hiveConf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, + SQLStdHiveAuthorizerFactory.class.getName()); + MetastoreConf.setVar(hiveConf, MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS, + DbNotificationListener.class.getName()); + MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.EVENT_DB_LISTENER_CLEAN_INTERVAL, 10, + TimeUnit.MILLISECONDS); + MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.EVENT_DB_LISTENER_CLEAN_STARTUP_WAIT_INTERVAL, 0, + TimeUnit.SECONDS); + } + + @After + public void tearDown() throws Exception { + TestTxnDbUtil.cleanDb(hiveConf); + if (hive != null) { + hive.close(); + } + SessionState.get().close(); + hiveConf = null; + } + + @Test + public void testTxnNotificationLogging() throws Exception { + try { + List txnIds = openTxns(numberOfTxns, txnType); + assertEquals(txnIds.size(), getNumberOfTxns(txnIds, TxnState.OPEN)); + assertEquals(expectedNotifications, getNumNotifications(txnIds, MessageBuilder.OPEN_TXN_EVENT)); + Thread.sleep(1000); + runHouseKeeperService(); + if (txnType != TxnType.REPL_CREATED) { + assertEquals(txnIds.size(), getNumberOfTxns(txnIds, TxnState.ABORTED)); + assertEquals(expectedNotifications, getNumNotifications(txnIds, MessageBuilder.ABORT_TXN_EVENT)); + } + } finally { + runTxnHouseKeeperService(); + } + } + + private int getNumNotifications(List txnIds, String eventType) throws IOException, TException { + int numNotifications = 0; + IMetaStoreClient.NotificationFilter evFilter = new AndFilter(new ReplEventFilter(new ReplScope()), + new CatalogFilter(MetaStoreUtils.getDefaultCatalog(hiveConf)), new EventBoundaryFilter(0, 100)); + NotificationEventResponse rsp = hive.getNextNotification(new NotificationEventRequest(), true, evFilter); + if (rsp.getEvents() == null) { + return numNotifications; + } + Iterator eventIterator = rsp.getEvents().iterator(); + MessageDeserializer deserializer = null; + while (eventIterator.hasNext()) { + NotificationEvent ev = eventIterator.next(); + if (eventType.equals(ev.getEventType())) { + deserializer = ReplUtils.getEventDeserializer(ev); + switch (ev.getEventType()) { + case MessageBuilder.OPEN_TXN_EVENT: + OpenTxnMessage openTxnMessage = deserializer.getOpenTxnMessage(ev.getMessage()); + if (txnIds.contains(openTxnMessage.getTxnIds().get(0))) { + numNotifications++; + } + break; + case MessageBuilder.ABORT_TXN_EVENT: + AbortTxnMessage abortTxnMessage = deserializer.getAbortTxnMessage(ev.getMessage()); + if (txnIds.contains(abortTxnMessage.getTxnId())) { + numNotifications++; + } + } + } + } + return numNotifications; + } + + private List openTxns(int txnCounter, TxnType txnType) throws TException { + List txnIds = new LinkedList<>(); + for (; txnCounter > 0; txnCounter--) { + if (txnType == TxnType.REPL_CREATED) { + Long srcTxn = (long) (11 + txnCounter); + List srcTxns = Arrays.asList(new Long[] { srcTxn }); + txnIds.addAll(hive.replOpenTxn("testPolicy", srcTxns, "hive", txnType)); + } else { + txnIds.add(hive.openTxn("hive", txnType)); + } + } + return txnIds; + } + + private int getNumberOfTxns(List txnIds, TxnState txnState) throws TException { + AtomicInteger numTxns = new AtomicInteger(); + hive.showTxns().getOpen_txns().forEach(txnInfo -> { + if (txnInfo.getState() == txnState && txnIds.contains(txnInfo.getId())) { + numTxns.incrementAndGet(); + } + }); + return numTxns.get(); + } + + private void runHouseKeeperService() { + MetastoreTaskThread acidHouseKeeperService = new AcidHouseKeeperService(); + acidHouseKeeperService.setConf(hiveConf); + acidHouseKeeperService.run(); //this will abort timedout txns + } + + private void runTxnHouseKeeperService() { + MetastoreTaskThread acidTxnCleanerService = new AcidTxnCleanerService(); + acidTxnCleanerService.setConf(hiveConf); + acidTxnCleanerService.run(); //this will remove empty aborted txns + } +} \ No newline at end of file diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java index c661e34b0739..8905a01490e0 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -5799,7 +5799,7 @@ public void performTimeOuts() { //timely way. timeOutLocks(); while (true) { - String s = " \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN + + String s = " \"TXN_ID\", \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN + " AND (" + "\"TXN_TYPE\" != " + TxnType.REPL_CREATED.getValue() + " AND \"TXN_LAST_HEARTBEAT\" < " + getEpochFn(dbProduct) + "-" + timeout + @@ -5811,14 +5811,14 @@ public void performTimeOuts() { s = sqlGenerator.addLimitClause(10 * TIMED_OUT_TXN_ABORT_BATCH_SIZE, s); LOG.debug("Going to execute query <{}>", s); - List> timedOutTxns = jdbcResource.getJdbcTemplate().query(s, rs -> { - List> txnbatch = new ArrayList<>(); - List currentBatch = new ArrayList<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE); + List> timedOutTxns = jdbcResource.getJdbcTemplate().query(s, rs -> { + List> txnbatch = new ArrayList<>(); + Map currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE); while (rs.next()) { - currentBatch.add(rs.getLong(1)); + currentBatch.put(rs.getLong(1),TxnType.findByValue(rs.getInt(2))); if (currentBatch.size() == TIMED_OUT_TXN_ABORT_BATCH_SIZE) { txnbatch.add(currentBatch); - currentBatch = new ArrayList<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE); + currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE); } } if (currentBatch.size() > 0) { @@ -5835,12 +5835,23 @@ public void performTimeOuts() { Object savePoint = context.getTransactionStatus().createSavepoint(); int numTxnsAborted = 0; - for (List batchToAbort : timedOutTxns) { + for (Map batchToAbort : timedOutTxns) { context.getTransactionStatus().releaseSavepoint(savePoint); savePoint = context.getTransactionStatus().createSavepoint(); - if (abortTxns(jdbcResource.getConnection(), batchToAbort, true, false, false, TxnErrorMsg.ABORT_TIMEOUT) == batchToAbort.size()) { + if (abortTxns(jdbcResource.getConnection(), new ArrayList<>(batchToAbort.keySet()), true, false, false, TxnErrorMsg.ABORT_TIMEOUT) == batchToAbort.size()) { numTxnsAborted += batchToAbort.size(); //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout' + LOG.info("Aborted the following transactions due to timeout: {}", batchToAbort); + if (transactionalListeners != null) { + for (Map.Entry txnEntry : batchToAbort.entrySet()) { + List dbsUpdated = getTxnDbsUpdated(txnEntry.getKey(), jdbcResource.getConnection()); + MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, + EventMessage.EventType.ABORT_TXN, + new AbortTxnEvent(txnEntry.getKey(), txnEntry.getValue(), null, dbsUpdated), + jdbcResource.getConnection(), sqlGenerator); + } + LOG.debug("Added Notifications for the transactions that are aborted due to timeout: {}", batchToAbort); + } } else { //could not abort all txns in this batch - this may happen because in parallel with this //operation there was activity on one of the txns in this batch (commit/abort/heartbeat) From d6158aa4d9aa11da32981d97e53e537a227c3c58 Mon Sep 17 00:00:00 2001 From: okumin Date: Wed, 6 Dec 2023 23:41:36 +0900 Subject: [PATCH 083/179] HIVE-27905: Some GenericUDFs wrongly cast ObjectInspectors (#4896). (okumin, reviewed by Ayush Saxena, Attila Turoczy) --- .../generic/GenericUDFBaseNwayCompare.java | 1 + .../generic/GenericUDFEnforceConstraint.java | 5 +++++ .../ql/udf/generic/GenericUDFEpochMilli.java | 6 ++++- .../generic/GenericUDFExceptionInVertex.java | 22 +++++++++++++------ .../ql/udf/generic/GenericUDFGrouping.java | 1 + .../ql/udf/generic/GenericUDFOPFalse.java | 1 + .../ql/udf/generic/GenericUDFOPNotFalse.java | 1 + .../ql/udf/generic/GenericUDFOPNotTrue.java | 1 + .../hive/ql/udf/generic/GenericUDFOPTrue.java | 1 + .../udf/generic/GenericUDFSQCountCheck.java | 1 + .../hive/ql/udf/generic/GenericUDFSplit.java | 1 + .../hive/ql/udf/generic/GenericUDFTrunc.java | 3 +++ .../udf/generic/GenericUDFTumbledWindow.java | 3 +++ .../udf_enforce_constraint_wrong_type1.q | 1 + .../udf_exception_in_vertex_udf_wrong_type1.q | 2 ++ .../udf_exception_in_vertex_udf_wrong_type2.q | 2 ++ .../udf_exception_in_vertex_udf_wrong_type3.q | 2 ++ .../udf_exception_in_vertex_udf_wrong_type4.q | 2 ++ .../clientnegative/udf_greatest_error_3.q | 2 ++ .../clientnegative/udf_grouping_wrong_type1.q | 1 + .../clientnegative/udf_isfalse_wrong_type1.q | 1 + .../udf_isnotfalse_wrong_type1.q | 1 + .../udf_isnottrue_wrong_type1.q | 1 + .../clientnegative/udf_istrue_wrong_type1.q | 1 + .../clientnegative/udf_split_wrong_type1.q | 1 + .../clientnegative/udf_split_wrong_type2.q | 1 + .../udf_sq_count_check_wrong_type1.q | 1 + .../udf_to_epoch_milli_wrong_type1.q | 2 ++ .../clientnegative/udf_trunc_wrong_type1.q | 1 + .../clientnegative/udf_trunc_wrong_type2.q | 1 + .../udf_tumbling_window_check_wrong_type1.q | 1 + .../udf_tumbling_window_check_wrong_type2.q | 1 + .../udf_enforce_constraint_wrong_type1.q.out | 1 + ..._exception_in_vertex_udf_wrong_type1.q.out | 1 + ..._exception_in_vertex_udf_wrong_type2.q.out | 1 + ..._exception_in_vertex_udf_wrong_type3.q.out | 1 + ..._exception_in_vertex_udf_wrong_type4.q.out | 1 + .../clientnegative/udf_greatest_error_3.q.out | 1 + .../udf_grouping_wrong_type1.q.out | 1 + .../udf_isfalse_wrong_type1.q.out | 1 + .../udf_isnotfalse_wrong_type1.q.out | 1 + .../udf_isnottrue_wrong_type1.q.out | 1 + .../udf_istrue_wrong_type1.q.out | 1 + .../udf_split_wrong_type1.q.out | 1 + .../udf_split_wrong_type2.q.out | 1 + .../udf_sq_count_check_wrong_type1.q.out | 1 + .../udf_to_epoch_milli_wrong_type1.q.out | 1 + .../udf_trunc_wrong_type1.q.out | 1 + .../udf_trunc_wrong_type2.q.out | 1 + ...df_tumbling_window_check_wrong_type1.q.out | 1 + ...df_tumbling_window_check_wrong_type2.q.out | 1 + 51 files changed, 83 insertions(+), 8 deletions(-) create mode 100644 ql/src/test/queries/clientnegative/udf_enforce_constraint_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q create mode 100644 ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q create mode 100644 ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q create mode 100644 ql/src/test/queries/clientnegative/udf_greatest_error_3.q create mode 100644 ql/src/test/queries/clientnegative/udf_grouping_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_isfalse_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_isnotfalse_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_isnottrue_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_istrue_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_split_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_split_wrong_type2.q create mode 100644 ql/src/test/queries/clientnegative/udf_sq_count_check_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_to_epoch_milli_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_trunc_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_trunc_wrong_type2.q create mode 100644 ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type1.q create mode 100644 ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type2.q create mode 100644 ql/src/test/results/clientnegative/udf_enforce_constraint_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q.out create mode 100644 ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q.out create mode 100644 ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q.out create mode 100644 ql/src/test/results/clientnegative/udf_greatest_error_3.q.out create mode 100644 ql/src/test/results/clientnegative/udf_grouping_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_isfalse_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_isnotfalse_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_isnottrue_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_istrue_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_split_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_split_wrong_type2.q.out create mode 100644 ql/src/test/results/clientnegative/udf_sq_count_check_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_to_epoch_milli_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_trunc_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_trunc_wrong_type2.q.out create mode 100644 ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type1.q.out create mode 100644 ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type2.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNwayCompare.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNwayCompare.java index d71f04f91b82..cdc14286333e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNwayCompare.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNwayCompare.java @@ -62,6 +62,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen TypeInfo commonInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(arguments[0]); for (int i = 1; i < arguments.length; i++) { + checkArgPrimitive(arguments, i); PrimitiveTypeInfo currInfo = (PrimitiveTypeInfo) TypeInfoUtils.getTypeInfoFromObjectInspector(arguments[i]); commonInfo = FunctionRegistry.getCommonClassForComparison( diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java index aa0059b1cf2b..82f15af47b4f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceConstraint.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; import org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -47,6 +48,10 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen + arguments.length); } + if (!(arguments[0] instanceof BooleanObjectInspector)) { + throw new UDFArgumentTypeException(0, + String.format("%s only takes BOOLEAN, got %s", getFuncName(), arguments[0].getTypeName())); + } boi = (BooleanObjectInspector) arguments[0]; return PrimitiveObjectInspectorFactory.writableBooleanObjectInspector; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEpochMilli.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEpochMilli.java index 58bd86d6456b..f17c909dc429 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEpochMilli.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEpochMilli.java @@ -20,6 +20,7 @@ import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; @@ -49,8 +50,11 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen } if (arguments[0] instanceof TimestampObjectInspector) { tsOi = (TimestampObjectInspector) arguments[0]; - } else { + } else if (arguments[0] instanceof TimestampLocalTZObjectInspector) { tsWithLocalTzOi = (TimestampLocalTZObjectInspector) arguments[0]; + } else { + throw new UDFArgumentTypeException(0, + String.format("%s only takes TIMESTAMP, got %s", getFuncName(), arguments[0].getTypeName())); } return PrimitiveObjectInspectorFactory.writableLongObjectInspector; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFExceptionInVertex.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFExceptionInVertex.java index 55ed806081a9..0f12db5a6fdf 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFExceptionInVertex.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFExceptionInVertex.java @@ -76,27 +76,35 @@ public ObjectInspector initialize(ObjectInspector[] parameters) throws UDFArgume return PrimitiveObjectInspectorFactory.javaLongObjectInspector; } - public static String getVertexName(ObjectInspector[] parameters, int index) { - return ((WritableConstantStringObjectInspector) parameters[index]).getWritableConstantValue() - .toString(); + public static String getVertexName(ObjectInspector[] parameters, int index) throws UDFArgumentTypeException { + if (parameters[index] instanceof WritableConstantStringObjectInspector) { + return ((WritableConstantStringObjectInspector) parameters[index]).getWritableConstantValue() + .toString(); + } else { + throw new UDFArgumentTypeException(index, String.format( + "This argument takes only constant STRING, got %s", parameters[index].getTypeName())); + } } - public static String getTaskNumber(ObjectInspector[] parameters, int index) { + public static String getTaskNumber(ObjectInspector[] parameters, int index) throws UDFArgumentTypeException { return getExpressionAtIndex(parameters, index); } - public static String getTaskAttemptNumber(ObjectInspector[] parameters, int index) { + public static String getTaskAttemptNumber(ObjectInspector[] parameters, int index) throws UDFArgumentTypeException { return getExpressionAtIndex(parameters, index); } - private static String getExpressionAtIndex(ObjectInspector[] parameters, int index) { + private static String getExpressionAtIndex(ObjectInspector[] parameters, int index) throws UDFArgumentTypeException { if (parameters.length > index) { if (parameters[index] instanceof WritableConstantStringObjectInspector) { return ((WritableConstantStringObjectInspector) parameters[index]) .getWritableConstantValue().toString(); - } else { + } else if (parameters[index] instanceof WritableConstantIntObjectInspector) { return ((WritableConstantIntObjectInspector) parameters[index]).getWritableConstantValue() .toString(); + } else { + throw new UDFArgumentTypeException(index, String.format( + "This argument takes only constant STRING or INT, got %s", parameters[index].getTypeName())); } } else { return "*"; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGrouping.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGrouping.java index d11e51b3cbf0..649c4788b3ac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGrouping.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFGrouping.java @@ -71,6 +71,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen indices = new int[arguments.length - 1]; for (int i = 1; i < arguments.length; i++) { + checkArgPrimitive(arguments, i); PrimitiveObjectInspector arg2OI = (PrimitiveObjectInspector) arguments[i]; if (!(arg2OI instanceof ConstantObjectInspector)) { throw new UDFArgumentTypeException(i, "Must be a constant. Got: " + arg2OI.getClass().getSimpleName()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPFalse.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPFalse.java index 3d0cb9687bed..433222429e7d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPFalse.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPFalse.java @@ -39,6 +39,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen if (arguments.length != 1) { throw new UDFArgumentLengthException("Invalid number of arguments"); } + checkArgPrimitive(arguments, 0); conditionConverter = ObjectInspectorConverters.getConverter(arguments[0], PrimitiveObjectInspectorFactory.writableBooleanObjectInspector); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotFalse.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotFalse.java index 4bfb8983327b..14301e4f5a07 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotFalse.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotFalse.java @@ -39,6 +39,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen if (arguments.length != 1) { throw new UDFArgumentLengthException("Invalid number of arguments"); } + checkArgPrimitive(arguments, 0); conditionConverter = ObjectInspectorConverters.getConverter(arguments[0], PrimitiveObjectInspectorFactory.writableBooleanObjectInspector); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotTrue.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotTrue.java index 65b80b577ed3..a92f29e27d14 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotTrue.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotTrue.java @@ -39,6 +39,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen if (arguments.length != 1) { throw new UDFArgumentLengthException("Invalid number of arguments"); } + checkArgPrimitive(arguments, 0); conditionConverter = ObjectInspectorConverters.getConverter(arguments[0], PrimitiveObjectInspectorFactory.writableBooleanObjectInspector); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPTrue.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPTrue.java index 30d33fb8bbc3..208289013320 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPTrue.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPTrue.java @@ -39,6 +39,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen if (arguments.length != 1) { throw new UDFArgumentLengthException("Invalid number of arguments"); } + checkArgPrimitive(arguments, 0); conditionConverter = ObjectInspectorConverters.getConverter(arguments[0], PrimitiveObjectInspectorFactory.writableBooleanObjectInspector); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java index b965410f9459..421c3fce70ad 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java @@ -48,6 +48,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen "Invalid scalar subquery expression. Subquery count check expected two argument but received: " + arguments.length); } + checkArgPrimitive(arguments, 0); converters[0] = ObjectInspectorConverters.getConverter(arguments[0], PrimitiveObjectInspectorFactory.writableLongObjectInspector); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSplit.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSplit.java index f95e75b04e05..e091965ce4af 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSplit.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSplit.java @@ -53,6 +53,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen converters = new ObjectInspectorConverters.Converter[arguments.length]; for (int i = 0; i < arguments.length; i++) { + checkArgPrimitive(arguments, i); converters[i] = ObjectInspectorConverters.getConverter(arguments[i], PrimitiveObjectInspectorFactory.writableStringObjectInspector); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTrunc.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTrunc.java index 3cebbe7af853..ffb553f78c53 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTrunc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTrunc.java @@ -106,6 +106,9 @@ public class GenericUDFTrunc extends GenericUDF { @Override public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + for (int i = 0; i < arguments.length; i++) { + checkArgPrimitive(arguments, i); + } if (arguments.length == 2) { inputType1 = ((PrimitiveObjectInspector) arguments[0]).getPrimitiveCategory(); inputType2 = ((PrimitiveObjectInspector) arguments[1]).getPrimitiveCategory(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTumbledWindow.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTumbledWindow.java index 17823c01ee55..7c6c3b24e970 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTumbledWindow.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTumbledWindow.java @@ -71,6 +71,9 @@ public class GenericUDFTumbledWindow extends GenericUDF { */ @Override public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { checkArgsSize(arguments, 2, 3); + for (int i = 0; i < arguments.length; i++) { + checkArgPrimitive(arguments, i); + } //arg 1 has to be of timestamp type //arg 2 has to be an interval //arg 3 has to be absent or timestamp type diff --git a/ql/src/test/queries/clientnegative/udf_enforce_constraint_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_enforce_constraint_wrong_type1.q new file mode 100644 index 000000000000..51d6271d57ff --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_enforce_constraint_wrong_type1.q @@ -0,0 +1 @@ +SELECT ENFORCE_CONSTRAINT(1); diff --git a/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q new file mode 100644 index 000000000000..5fe8a9576421 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q @@ -0,0 +1,2 @@ +--! qt:dataset:src +SELECT EXCEPTION_IN_VERTEX_UDF(value, 1) FROM src diff --git a/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q new file mode 100644 index 000000000000..5fe8a9576421 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q @@ -0,0 +1,2 @@ +--! qt:dataset:src +SELECT EXCEPTION_IN_VERTEX_UDF(value, 1) FROM src diff --git a/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q new file mode 100644 index 000000000000..0cb863d42019 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q @@ -0,0 +1,2 @@ +--! qt:dataset:src +SELECT EXCEPTION_IN_VERTEX_UDF(value, 'Map 1', 99.9) FROM src diff --git a/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q new file mode 100644 index 000000000000..0a112ad44036 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q @@ -0,0 +1,2 @@ +--! qt:dataset:src +SELECT EXCEPTION_IN_VERTEX_UDF(value, 'Map 1', 1, 99.9) FROM src diff --git a/ql/src/test/queries/clientnegative/udf_greatest_error_3.q b/ql/src/test/queries/clientnegative/udf_greatest_error_3.q new file mode 100644 index 000000000000..29860dc2f0af --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_greatest_error_3.q @@ -0,0 +1,2 @@ +--! qt:dataset:src +SELECT GREATEST('2.0', array('a', 'b')) FROM src LIMIT 1; diff --git a/ql/src/test/queries/clientnegative/udf_grouping_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_grouping_wrong_type1.q new file mode 100644 index 000000000000..d985bc36c0e1 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_grouping_wrong_type1.q @@ -0,0 +1 @@ +SELECT GROUPING(1, ARRAY(2)); diff --git a/ql/src/test/queries/clientnegative/udf_isfalse_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_isfalse_wrong_type1.q new file mode 100644 index 000000000000..b15a05ded872 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_isfalse_wrong_type1.q @@ -0,0 +1 @@ +SELECT ISFALSE(ARRAY(1)) diff --git a/ql/src/test/queries/clientnegative/udf_isnotfalse_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_isnotfalse_wrong_type1.q new file mode 100644 index 000000000000..eb3de6996553 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_isnotfalse_wrong_type1.q @@ -0,0 +1 @@ +SELECT ISNOTFALSE(ARRAY(1)); diff --git a/ql/src/test/queries/clientnegative/udf_isnottrue_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_isnottrue_wrong_type1.q new file mode 100644 index 000000000000..aa36ac24721c --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_isnottrue_wrong_type1.q @@ -0,0 +1 @@ +SELECT ISNOTTRUE(ARRAY(1)); diff --git a/ql/src/test/queries/clientnegative/udf_istrue_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_istrue_wrong_type1.q new file mode 100644 index 000000000000..be6742ad1247 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_istrue_wrong_type1.q @@ -0,0 +1 @@ +SELECT ISTRUE(ARRAY(1)); diff --git a/ql/src/test/queries/clientnegative/udf_split_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_split_wrong_type1.q new file mode 100644 index 000000000000..f9cd7c5ac3ba --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_split_wrong_type1.q @@ -0,0 +1 @@ +SELECT SPLIT(ARRAY('1,2,3'), ','); diff --git a/ql/src/test/queries/clientnegative/udf_split_wrong_type2.q b/ql/src/test/queries/clientnegative/udf_split_wrong_type2.q new file mode 100644 index 000000000000..2ce53c86b9a7 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_split_wrong_type2.q @@ -0,0 +1 @@ +SELECT SPLIT('1,2,3', ARRAY(',', ':')); diff --git a/ql/src/test/queries/clientnegative/udf_sq_count_check_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_sq_count_check_wrong_type1.q new file mode 100644 index 000000000000..261f44b92dd3 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_sq_count_check_wrong_type1.q @@ -0,0 +1 @@ +SELECT SQ_COUNT_CHECK(ARRAY(1)); diff --git a/ql/src/test/queries/clientnegative/udf_to_epoch_milli_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_to_epoch_milli_wrong_type1.q new file mode 100644 index 000000000000..4438c00eae4d --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_to_epoch_milli_wrong_type1.q @@ -0,0 +1,2 @@ +--! qt:dataset:src +SELECT TO_EPOCH_MILLI(1) FROM src diff --git a/ql/src/test/queries/clientnegative/udf_trunc_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_trunc_wrong_type1.q new file mode 100644 index 000000000000..9ef1294366f2 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_trunc_wrong_type1.q @@ -0,0 +1 @@ +SELECT TRUNC(ARRAY('2023-12-04'), 'MM'); diff --git a/ql/src/test/queries/clientnegative/udf_trunc_wrong_type2.q b/ql/src/test/queries/clientnegative/udf_trunc_wrong_type2.q new file mode 100644 index 000000000000..8746bf4c9116 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_trunc_wrong_type2.q @@ -0,0 +1 @@ +SELECT TRUNC('2023-12-04', ARRAY('MM')); diff --git a/ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type1.q b/ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type1.q new file mode 100644 index 000000000000..afede1661592 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type1.q @@ -0,0 +1 @@ +SELECT TUMBLING_WINDOW(ARRAY('2020-03-01 06:03:00'), interval '5' MINUTES); diff --git a/ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type2.q b/ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type2.q new file mode 100644 index 000000000000..24837bf5c7bc --- /dev/null +++ b/ql/src/test/queries/clientnegative/udf_tumbling_window_check_wrong_type2.q @@ -0,0 +1 @@ +SELECT TUMBLING_WINDOW(CAST('2020-03-01 06:03:00' as timestamp), ARRAY(interval '5' MINUTES)); diff --git a/ql/src/test/results/clientnegative/udf_enforce_constraint_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_enforce_constraint_wrong_type1.q.out new file mode 100644 index 000000000000..67279fc6b42c --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_enforce_constraint_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:26 Argument type mismatch '1': enforce_constraint only takes BOOLEAN, got int diff --git a/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q.out new file mode 100644 index 000000000000..9f98fa8f9fe6 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 2:38 Argument type mismatch '1': This argument takes only constant STRING, got int diff --git a/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q.out b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q.out new file mode 100644 index 000000000000..9f98fa8f9fe6 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type2.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 2:38 Argument type mismatch '1': This argument takes only constant STRING, got int diff --git a/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q.out b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q.out new file mode 100644 index 000000000000..7b92a101cfea --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type3.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 2:47 Argument type mismatch '99.9': This argument takes only constant STRING or INT, got decimal(3,1) diff --git a/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q.out b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q.out new file mode 100644 index 000000000000..0c2f9310812c --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_exception_in_vertex_udf_wrong_type4.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 2:50 Argument type mismatch '99.9': This argument takes only constant STRING or INT, got decimal(3,1) diff --git a/ql/src/test/results/clientnegative/udf_greatest_error_3.q.out b/ql/src/test/results/clientnegative/udf_greatest_error_3.q.out new file mode 100644 index 000000000000..c6bb3413fc85 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_greatest_error_3.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 2:23 Argument type mismatch ''b'': greatest only takes primitive types as 2nd argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_grouping_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_grouping_wrong_type1.q.out new file mode 100644 index 000000000000..de4f379c3499 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_grouping_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:19 Argument type mismatch '2': grouping only takes primitive types as 2nd argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_isfalse_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_isfalse_wrong_type1.q.out new file mode 100644 index 000000000000..d03b7b33b73b --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_isfalse_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:15 Argument type mismatch '1': opfalse only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_isnotfalse_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_isnotfalse_wrong_type1.q.out new file mode 100644 index 000000000000..71712af520c8 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_isnotfalse_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:18 Argument type mismatch '1': opnotfalse only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_isnottrue_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_isnottrue_wrong_type1.q.out new file mode 100644 index 000000000000..61c08e9f91b4 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_isnottrue_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:17 Argument type mismatch '1': opnottrue only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_istrue_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_istrue_wrong_type1.q.out new file mode 100644 index 000000000000..6a235dad76e4 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_istrue_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:14 Argument type mismatch '1': optrue only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_split_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_split_wrong_type1.q.out new file mode 100644 index 000000000000..b10d0dc9e7b9 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_split_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:13 Argument type mismatch ''1,2,3'': split only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_split_wrong_type2.q.out b/ql/src/test/results/clientnegative/udf_split_wrong_type2.q.out new file mode 100644 index 000000000000..10d58d85ec60 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_split_wrong_type2.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:22 Argument type mismatch '':'': split only takes primitive types as 2nd argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_sq_count_check_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_sq_count_check_wrong_type1.q.out new file mode 100644 index 000000000000..8e82da5661da --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_sq_count_check_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:22 Argument type mismatch '1': sq_count_check only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_to_epoch_milli_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_to_epoch_milli_wrong_type1.q.out new file mode 100644 index 000000000000..ca8e47d82c93 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_to_epoch_milli_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 2:22 Argument type mismatch '1': epochmilli only takes TIMESTAMP, got int diff --git a/ql/src/test/results/clientnegative/udf_trunc_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_trunc_wrong_type1.q.out new file mode 100644 index 000000000000..571b97e3e902 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_trunc_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:13 Argument type mismatch ''2023-12-04'': trunc only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_trunc_wrong_type2.q.out b/ql/src/test/results/clientnegative/udf_trunc_wrong_type2.q.out new file mode 100644 index 000000000000..65097f1bf7db --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_trunc_wrong_type2.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:27 Argument type mismatch ''MM'': trunc only takes primitive types as 2nd argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type1.q.out b/ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type1.q.out new file mode 100644 index 000000000000..f22b44f7b435 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:23 Argument type mismatch ''2020-03-01 06:03:00'': tumbling_window only takes primitive types as 1st argument, got LIST diff --git a/ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type2.q.out b/ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type2.q.out new file mode 100644 index 000000000000..c13ec561a2a5 --- /dev/null +++ b/ql/src/test/results/clientnegative/udf_tumbling_window_check_wrong_type2.q.out @@ -0,0 +1 @@ +FAILED: SemanticException [Error 10016]: Line 1:65 Argument type mismatch ''5'': tumbling_window only takes primitive types as 2nd argument, got LIST From 60b1de1a05c18156f597ac36f2b702e930b26e40 Mon Sep 17 00:00:00 2001 From: Akshat Mathur Date: Wed, 6 Dec 2023 21:08:19 +0530 Subject: [PATCH 084/179] HIVE-27935: Add qtest for Avro invalid schema and field names (#4918) (Akshat Mathur, reviewed by Butao Zhang) --- .../avro_duplicate_field_name.q | 25 +++++++++++++++++ .../clientnegative/avro_invalid_field_name.q | 18 +++++++++++++ .../avro_duplicate_field_name.q.out | 27 +++++++++++++++++++ .../avro_invalid_field_name.q.out | 20 ++++++++++++++ 4 files changed, 90 insertions(+) create mode 100644 ql/src/test/queries/clientnegative/avro_duplicate_field_name.q create mode 100644 ql/src/test/queries/clientnegative/avro_invalid_field_name.q create mode 100644 ql/src/test/results/clientnegative/avro_duplicate_field_name.q.out create mode 100644 ql/src/test/results/clientnegative/avro_invalid_field_name.q.out diff --git a/ql/src/test/queries/clientnegative/avro_duplicate_field_name.q b/ql/src/test/queries/clientnegative/avro_duplicate_field_name.q new file mode 100644 index 000000000000..d10af6c15068 --- /dev/null +++ b/ql/src/test/queries/clientnegative/avro_duplicate_field_name.q @@ -0,0 +1,25 @@ +-- verify AVRO-3827: Disallow duplicate field names + +CREATE TABLE avroExternalDupField +STORED AS AVRO +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "org.apache.hive", + "name": "my_schema", + "type": "record", + "fields": [ + { + "name": "f1", + "type": { + "name": "a", + "type": "record", + "fields": [] + } + }, { + "name": "f1", + "type": { + "name": "b", + "type": "record", + "fields": [] + } + } + ] }'); diff --git a/ql/src/test/queries/clientnegative/avro_invalid_field_name.q b/ql/src/test/queries/clientnegative/avro_invalid_field_name.q new file mode 100644 index 000000000000..8ce42bc3dbf0 --- /dev/null +++ b/ql/src/test/queries/clientnegative/avro_invalid_field_name.q @@ -0,0 +1,18 @@ +-- verify AVRO-3820:Don't allow invalid field names, field name should match [A-Za-z_][A-Za-z0-9_]* + +CREATE TABLE avroExternalInvalidField +STORED AS AVRO +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "org.apache.hive", + "name": "my_record", + "type": "record", + "fields": [ + { + "name": "f1.x", + "type": { + "name": "my_enum", + "type": "enum", + "symbols": ["a"] + } + } + ] }'); diff --git a/ql/src/test/results/clientnegative/avro_duplicate_field_name.q.out b/ql/src/test/results/clientnegative/avro_duplicate_field_name.q.out new file mode 100644 index 000000000000..3d7527c33a05 --- /dev/null +++ b/ql/src/test/results/clientnegative/avro_duplicate_field_name.q.out @@ -0,0 +1,27 @@ +PREHOOK: query: CREATE TABLE avroExternalDupField +STORED AS AVRO +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "org.apache.hive", + "name": "my_schema", + "type": "record", + "fields": [ + { + "name": "f1", + "type": { + "name": "a", + "type": "record", + "fields": [] + } + }, { + "name": "f1", + "type": { + "name": "b", + "type": "record", + "fields": [] + } + } + ] }') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@avroExternalDupField +FAILED: Execution Error, return code 40000 from org.apache.hadoop.hive.ql.ddl.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException Invalid schema reported) diff --git a/ql/src/test/results/clientnegative/avro_invalid_field_name.q.out b/ql/src/test/results/clientnegative/avro_invalid_field_name.q.out new file mode 100644 index 000000000000..75ae3f4e8d30 --- /dev/null +++ b/ql/src/test/results/clientnegative/avro_invalid_field_name.q.out @@ -0,0 +1,20 @@ +PREHOOK: query: CREATE TABLE avroExternalInvalidField +STORED AS AVRO +TBLPROPERTIES ('avro.schema.literal'='{ + "namespace": "org.apache.hive", + "name": "my_record", + "type": "record", + "fields": [ + { + "name": "f1.x", + "type": { + "name": "my_enum", + "type": "enum", + "symbols": ["a"] + } + } + ] }') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@avroExternalInvalidField +FAILED: Execution Error, return code 40000 from org.apache.hadoop.hive.ql.ddl.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException Invalid schema reported) From 6495d3101de0a3e2577428a19da56de694a0d56d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Thu, 7 Dec 2023 10:46:59 +0530 Subject: [PATCH 085/179] HIVE-27714: Addendum: Iceberg: metadata location overrides can cause data breach - handling default locations. (#4921). (Ayush Saxena, reviewed by Sourabh Badhya) --- .../org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java | 2 +- .../java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java | 3 +-- .../org/apache/hadoop/hive/ql/session/SessionStateUtil.java | 1 + .../hadoop/hive/metastore/api/hive_metastoreConstants.java | 2 -- 4 files changed, 3 insertions(+), 5 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 40e4cb1cc934..9b430af805c6 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -1064,7 +1064,7 @@ static String encodeString(String rawString) { String getPathForAuth(String locationProperty) { return getPathForAuth(locationProperty, - SessionStateUtil.getProperty(conf, hive_metastoreConstants.DEFAULT_TABLE_LOCATION).orElse(null)); + SessionStateUtil.getProperty(conf, SessionStateUtil.DEFAULT_TABLE_LOCATION).orElse(null)); } String getPathForAuth(String locationProperty, String defaultTableLocation) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 8cdba7fd8dfe..0823b6d9ba4d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -25,7 +25,6 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVEARCHIVEENABLED; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_DEFAULT_STORAGE_HANDLER; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS; -import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.DEFAULT_TABLE_LOCATION; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.TABLE_IS_CTAS; @@ -13803,7 +13802,7 @@ private Map validateAndAddDefaultProperties( } if (isIcebergTable(retValue)) { - SessionStateUtil.addResourceOrThrow(conf, hive_metastoreConstants.DEFAULT_TABLE_LOCATION, + SessionStateUtil.addResourceOrThrow(conf, SessionStateUtil.DEFAULT_TABLE_LOCATION, getDefaultLocation(qualifiedTabName[0], qualifiedTabName[1], true)); } return retValue; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionStateUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionStateUtil.java index f831f0920b66..0009a54c3a3c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionStateUtil.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionStateUtil.java @@ -31,6 +31,7 @@ public class SessionStateUtil { private static final Logger LOG = LoggerFactory.getLogger(SessionStateUtil.class); private static final String COMMIT_INFO_PREFIX = "COMMIT_INFO."; + public static final String DEFAULT_TABLE_LOCATION = "defaultLocation"; private SessionStateUtil() { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java index 776683b882f8..f5a102ab9647 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java @@ -97,6 +97,4 @@ public static final java.lang.String EXPECTED_PARAMETER_VALUE = "expected_parameter_value"; - public static final java.lang.String DEFAULT_TABLE_LOCATION = "defaultLocation"; - } From fd6ced288dbf9ce7f3c3a2ca948d78f3b88f170f Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Thu, 7 Dec 2023 21:56:21 +0800 Subject: [PATCH 086/179] HIVE-27093: Fix NPE in initialize() of Partition class (#4073). (Wechar, reviewed by Ayush Saxena) --- ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java index 6b34a2b69729..884aa2016279 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java @@ -166,7 +166,7 @@ protected void initialize(Table table, return; } - if (table.isPartitioned()) { + if (table.isPartitioned() && tPartition.isSetSd()) { try { if (tPartition.getSd().getLocation() == null) { // set default if location is not set and this is a physical From cc0adbeb04dde46ba4dcda1503fe9231e993edd0 Mon Sep 17 00:00:00 2001 From: Shubham Sharma Date: Fri, 8 Dec 2023 02:49:27 +0530 Subject: [PATCH 087/179] HIVE-27240: NPE on Hive Hook Proto Log Writer. (#4925). (Shubham Sharma, reviewed by Ayush Saxena) --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index a8d4a8cbf53d..0cd931391a3c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -950,7 +950,7 @@ private String getRuleExcludedRegex() { JSONObject outputPlan(Object work, PrintStream out, boolean extended, boolean jsonOutput, int indent, String appendToHeader) throws Exception { return outputPlan(work, out, extended, jsonOutput, indent, appendToHeader, - queryState.getConf().getBoolVar(ConfVars.HIVE_IN_TEST)); + queryState != null && queryState.getConf().getBoolVar(ConfVars.HIVE_IN_TEST)); } public JSONObject outputPlan(Object work, PrintStream out, From 223241548054b95ec3ec497a855a5f602c419e74 Mon Sep 17 00:00:00 2001 From: Vikram Ahuja Date: Fri, 8 Dec 2023 10:52:10 +0530 Subject: [PATCH 088/179] HIVE-27893: Add a range validator in hive.metastore.batch.retrieve.max to only have values greater than 0 (Vikram Ahuja, Reviewed by Attila Turoczy, Zoltan Ratkai, Chinna Rao Lalam) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 3 ++- .../hive/ql/metadata/PartitionIterable.java | 3 +++ .../ql/exec/TestGetPartitionInBatches.java | 21 +++++++++++++++++++ .../hive/metastore/conf/MetastoreConf.java | 3 ++- .../hive/metastore/PartitionIterable.java | 3 +++ 5 files changed, 31 insertions(+), 2 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 991c97e250a0..2497ab367358 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1301,9 +1301,10 @@ public static enum ConfVars { */ @Deprecated METASTORE_BATCH_RETRIEVE_MAX("hive.metastore.batch.retrieve.max", 300, + new RangeValidator(1, null), "Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch. \n" + "The higher the number, the less the number of round trips is needed to the Hive metastore server, \n" + - "but it may also cause higher memory requirement at the client side."), + "but it may also cause higher memory requirement at the client side. Batch value should be greater than 0."), /** * @deprecated Use MetastoreConf.BATCH_RETRIEVE_OBJECTS_MAX */ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java index e31f9ef2d15c..158f31bf7f41 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java @@ -156,6 +156,9 @@ public PartitionIterable(Hive db, Table table, Map partialPartit */ public PartitionIterable(Hive db, Table table, Map partialPartitionSpec, int batchSize, boolean getColStats) throws HiveException { + if (batchSize < 1) { + throw new HiveException("Invalid batch size for partition iterable. Please use a batch size greater than 0"); + } this.currType = Type.LAZY_FETCH_PARTITIONS; this.db = db; this.table = table; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java index 246afeb79217..f11eee0500d3 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java @@ -23,11 +23,13 @@ import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.MetastoreException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.PartitionIterable; import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Assert; import org.junit.After; @@ -256,4 +258,23 @@ public void testBatchingWhenException() throws Exception { // In case any duplicate/incomplete list is given by hive.getAllPartitionsInBatches, the below assertion will fail assert(partNames.size() == 0); } + + @Test + public void testBatchingWhenBatchSizeIsZero() throws MetaException { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + int batchSize = 0; + try { + new PartitionIterable(hive, hive.getTable(dbName, tableName), null, batchSize); + } catch (HiveException e) { + Assert.assertTrue(e.getMessage().contains("Invalid batch size for partition iterable." + + " Please use a batch size greater than 0")); + } + try { + new org.apache.hadoop.hive.metastore.PartitionIterable(msc, table, batchSize); + } catch (MetastoreException e) { + Assert.assertTrue(e.getMessage().contains("Invalid batch size for partition iterable." + + " Please use a batch size greater than 0")); + } + } } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index 65449d4362c3..a37501a8d116 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -353,9 +353,10 @@ public enum ConfVars { + "To enable auto create also set hive.metastore.schema.verification=false. Auto creation is not " + "recommended for production use cases, run schematool command instead." ), BATCH_RETRIEVE_MAX("metastore.batch.retrieve.max", "hive.metastore.batch.retrieve.max", 300, + new RangeValidator(1, null), "Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch. \n" + "The higher the number, the less the number of round trips is needed to the Hive metastore server, \n" + - "but it may also cause higher memory requirement at the client side."), + "but it may also cause higher memory requirement at the client side. Batch value should be greater than 0"), BATCH_RETRIEVE_OBJECTS_MAX("metastore.batch.retrieve.table.partition.max", "hive.metastore.batch.retrieve.table.partition.max", 1000, "Maximum number of objects that metastore internally retrieves in one batch."), diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionIterable.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionIterable.java index 30a7ffcc3f60..26864d0bb958 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionIterable.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionIterable.java @@ -164,6 +164,9 @@ public PartitionIterable(Collection ptnsProvided) { * a Hive object and a table object, and a partial partition spec. */ public PartitionIterable(IMetaStoreClient msc, Table table, int batch_size) throws MetastoreException { + if (batch_size < 1) { + throw new MetastoreException("Invalid batch size for partition iterable. Please use a batch size greater than 0"); + } this.currType = Type.LAZY_FETCH_PARTITIONS; this.msc = msc; this.table = table; From f396676b09d9dd706b4bff4e1c1d999f9f3b1d2f Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Thu, 7 Dec 2023 17:56:08 +0100 Subject: [PATCH 089/179] HIVE-27658: Error resolving join keys during conversion to dynamic partition hashjoin (Stamatis Zampetakis reviewed by Denys Kuzmenko) Sometimes when the compiler attempts to convert a Join to a Dynamic Partition HashJoin (DPHJ) and certain assumptions about the shape of the plan do not hold a SemanticException is thrown. The DPHJ is a performance optimization so there is no reason to raise a fatal error when the conversion cannot be performed. It is preferable to simply skip the conversion and use a regular join instead of blocking completely the query. The `MapJoinProcessor.getMapJoinDesc` method already returns null in certain cases, so it is safe to add another exit condition. Overview of changes: 1. Return null when join key resolution fails and simply skip conversion to DPHJ. 2. Log a warning instead of throwing a fatal SemanticException. 3. Enrich error message with more information to improve diagnosability. Bringing the plan into a shape that will allow the DPHJ conversion is still meaningful but can be tracked independently with other tickets. Close apache/hive#4930 --- .../hive/ql/optimizer/MapJoinProcessor.java | 3 +- .../clientpositive/tez_dynpart_hashjoin_4.q | 24 ++ .../llap/tez_dynpart_hashjoin_4.q.out | 210 ++++++++++++++++++ 3 files changed, 236 insertions(+), 1 deletion(-) create mode 100644 ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_4.q create mode 100644 ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java index e922ce477964..adf4fbe1b216 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java @@ -1306,7 +1306,8 @@ public static MapJoinDesc getMapJoinDesc(HiveConf hconf, List keyExprList = ExprNodeDescUtils.resolveJoinKeysAsRSColumns(mapEntry.getValue(), rsParent); if (keyExprList == null) { - throw new SemanticException("Error resolving join keys"); + LOG.warn("Error resolving join keys {} in {} {}", mapEntry.getValue(), rsParent, rsParent.getColumnExprMap()); + return null; } newKeyExprMap.put(pos, keyExprList); } diff --git a/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_4.q b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_4.q new file mode 100644 index 000000000000..d15307a42de1 --- /dev/null +++ b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_4.q @@ -0,0 +1,24 @@ +CREATE TABLE table_a (start_date date, product_id int); + +ALTER TABLE table_a UPDATE STATISTICS SET('numRows'='200000000','rawDataSize'='0' ); +ALTER TABLE table_a UPDATE STATISTICS FOR COLUMN product_id SET('lowValue'='1000000','highValue'='100000000','numNulls'='0','numDVs'='300000' ); +ALTER TABLE table_a UPDATE STATISTICS FOR COLUMN start_date SET('lowValue'='10000','highValue'='20000','numNulls'='0','numDVs'='2500' ); + +CREATE TABLE table_b (start_date date, product_id int, product_sk string); + +ALTER TABLE table_b UPDATE STATISTICS SET('numRows'='100000000','rawDataSize'='0' ); +ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_id SET('lowValue'='1000000','highValue'='100000000','numNulls'='0','numDVs'='300000' ); +ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN start_date SET('lowValue'='10000','highValue'='20000','numNulls'='0','numDVs'='500' ); +ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_sk SET ('numDVs'='300000','numNulls'='0','avgColLen'='10','maxColLen'='10'); + +set hive.optimize.dynamic.partition.hashjoin=true; +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask.size=180000000; + +EXPLAIN +SELECT TC.CONST_DATE, TB.PRODUCT_SK +FROM TABLE_A TA +INNER JOIN (SELECT TO_DATE(FROM_UNIXTIME(1701088643)) AS CONST_DATE) TC + ON TA.START_DATE = TC.CONST_DATE +INNER JOIN TABLE_B TB + ON TB.START_DATE = TC.CONST_DATE AND TA.PRODUCT_ID = TB.PRODUCT_ID; diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out new file mode 100644 index 000000000000..5ca79e22b2b0 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out @@ -0,0 +1,210 @@ +PREHOOK: query: CREATE TABLE table_a (start_date date, product_id int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table_a +POSTHOOK: query: CREATE TABLE table_a (start_date date, product_id int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_a +PREHOOK: query: ALTER TABLE table_a UPDATE STATISTICS SET('numRows'='200000000','rawDataSize'='0' ) +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_a +PREHOOK: Output: default@table_a +POSTHOOK: query: ALTER TABLE table_a UPDATE STATISTICS SET('numRows'='200000000','rawDataSize'='0' ) +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_a +POSTHOOK: Output: default@table_a +PREHOOK: query: ALTER TABLE table_a UPDATE STATISTICS FOR COLUMN product_id SET('lowValue'='1000000','highValue'='100000000','numNulls'='0','numDVs'='300000' ) +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_a +PREHOOK: Output: default@table_a +POSTHOOK: query: ALTER TABLE table_a UPDATE STATISTICS FOR COLUMN product_id SET('lowValue'='1000000','highValue'='100000000','numNulls'='0','numDVs'='300000' ) +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_a +POSTHOOK: Output: default@table_a +PREHOOK: query: ALTER TABLE table_a UPDATE STATISTICS FOR COLUMN start_date SET('lowValue'='10000','highValue'='20000','numNulls'='0','numDVs'='2500' ) +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_a +PREHOOK: Output: default@table_a +POSTHOOK: query: ALTER TABLE table_a UPDATE STATISTICS FOR COLUMN start_date SET('lowValue'='10000','highValue'='20000','numNulls'='0','numDVs'='2500' ) +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_a +POSTHOOK: Output: default@table_a +PREHOOK: query: CREATE TABLE table_b (start_date date, product_id int, product_sk string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table_b +POSTHOOK: query: CREATE TABLE table_b (start_date date, product_id int, product_sk string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_b +PREHOOK: query: ALTER TABLE table_b UPDATE STATISTICS SET('numRows'='100000000','rawDataSize'='0' ) +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_b +PREHOOK: Output: default@table_b +POSTHOOK: query: ALTER TABLE table_b UPDATE STATISTICS SET('numRows'='100000000','rawDataSize'='0' ) +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_b +POSTHOOK: Output: default@table_b +PREHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_id SET('lowValue'='1000000','highValue'='100000000','numNulls'='0','numDVs'='300000' ) +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_b +PREHOOK: Output: default@table_b +POSTHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_id SET('lowValue'='1000000','highValue'='100000000','numNulls'='0','numDVs'='300000' ) +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_b +POSTHOOK: Output: default@table_b +PREHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN start_date SET('lowValue'='10000','highValue'='20000','numNulls'='0','numDVs'='500' ) +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_b +PREHOOK: Output: default@table_b +POSTHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN start_date SET('lowValue'='10000','highValue'='20000','numNulls'='0','numDVs'='500' ) +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_b +POSTHOOK: Output: default@table_b +PREHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_sk SET ('numDVs'='300000','numNulls'='0','avgColLen'='10','maxColLen'='10') +PREHOOK: type: ALTERTABLE_UPDATETABLESTATS +PREHOOK: Input: default@table_b +PREHOOK: Output: default@table_b +POSTHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_sk SET ('numDVs'='300000','numNulls'='0','avgColLen'='10','maxColLen'='10') +POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS +POSTHOOK: Input: default@table_b +POSTHOOK: Output: default@table_b +PREHOOK: query: EXPLAIN +SELECT TC.CONST_DATE, TB.PRODUCT_SK +FROM TABLE_A TA +INNER JOIN (SELECT TO_DATE(FROM_UNIXTIME(1701088643)) AS CONST_DATE) TC + ON TA.START_DATE = TC.CONST_DATE +INNER JOIN TABLE_B TB + ON TB.START_DATE = TC.CONST_DATE AND TA.PRODUCT_ID = TB.PRODUCT_ID +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Input: default@table_a +PREHOOK: Input: default@table_b +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN +SELECT TC.CONST_DATE, TB.PRODUCT_SK +FROM TABLE_A TA +INNER JOIN (SELECT TO_DATE(FROM_UNIXTIME(1701088643)) AS CONST_DATE) TC + ON TA.START_DATE = TC.CONST_DATE +INNER JOIN TABLE_B TB + ON TB.START_DATE = TC.CONST_DATE AND TA.PRODUCT_ID = TB.PRODUCT_ID +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Input: default@table_a +POSTHOOK: Input: default@table_b +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 1 <- Map 3 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: ta + filterExpr: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) + Statistics: Num rows: 200000000 Data size: 12000000000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) + Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: product_id (type: int) + outputColumnNames: _col1 + Statistics: Num rows: 100000000 Data size: 400000000 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 DATE'2023-11-27' (type: date) + 1 DATE'2023-11-27' (type: date) + outputColumnNames: _col1, _col2 + input vertices: + 1 Map 3 + Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int), _col2 (type: date) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col1 (type: int), _col2 (type: date) + Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 3 + Map Operator Tree: + TableScan + alias: _dummy_table + Row Limit Per Split: 1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: DATE'2023-11-27' (type: date) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: DATE'2023-11-27' (type: date) + null sort order: z + sort order: + + Map-reduce partition columns: DATE'2023-11-27' (type: date) + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: date) + Execution mode: llap + LLAP IO: no inputs + Map 4 + Map Operator Tree: + TableScan + alias: tb + filterExpr: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) + Statistics: Num rows: 100000000 Data size: 15400000000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) + Statistics: Num rows: 50000000 Data size: 7700000000 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: product_id (type: int), product_sk (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int), DATE'2023-11-27' (type: date) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col1 (type: int), DATE'2023-11-27' (type: date) + Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int), _col2 (type: date) + 1 _col1 (type: int), DATE'2023-11-27' (type: date) + outputColumnNames: _col2, _col5 + Statistics: Num rows: 16666666666 Data size: 2499999999900 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: date), _col5 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 16666666666 Data size: 1566666666604 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 16666666666 Data size: 1566666666604 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + From 7e5b5de9ebaad037b81eb1a581013db88fac6c51 Mon Sep 17 00:00:00 2001 From: dengzh Date: Sat, 9 Dec 2023 18:06:55 +0800 Subject: [PATCH 090/179] HIVE-27555: Upgrade issues with Kudu table on backend db (#4872) (Zhihua Deng, reviewed by Attila Turoczy, Denys Kuzmenko) --- ...grade-4.0.0-alpha-2-to-4.0.0-beta-1.derby.sql | 16 ++++++++++------ ...grade-4.0.0-alpha-2-to-4.0.0-beta-1.mssql.sql | 16 ++++++++++------ ...grade-4.0.0-alpha-2-to-4.0.0-beta-1.mysql.sql | 16 ++++++++++------ ...rade-4.0.0-alpha-2-to-4.0.0-beta-1.oracle.sql | 16 ++++++++++------ ...de-4.0.0-alpha-2-to-4.0.0-beta-1.postgres.sql | 16 ++++++++++------ 5 files changed, 50 insertions(+), 30 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.derby.sql index d24722518a23..b17732541070 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.derby.sql @@ -31,18 +31,22 @@ ALTER TABLE "APP"."METASTORE_DB_PROPERTIES" ADD PROPERTYCONTENT BLOB; -- HIVE-27457 UPDATE "SDS" - SET "SDS"."INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', - "SDS"."OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' - WHERE "SDS"."SD_ID" IN ( - SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%' + SET "INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', "OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' + WHERE "SD_ID" IN ( + SELECT "TBLS"."SD_ID" + FROM "TBLS" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); + UPDATE "SERDES" SET "SERDES"."SLIB" = 'org.apache.hadoop.hive.kudu.KuduSerDe' WHERE "SERDE_ID" IN ( SELECT "SDS"."SERDE_ID" FROM "TBLS" - LEFT JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" - WHERE "TBL_ID" IN (SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%') + INNER JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); -- This needs to be the last thing done. Insert any changes above this line. diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mssql.sql index 5712d24847b7..e4253ac1ff94 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mssql.sql @@ -31,18 +31,22 @@ ALTER TABLE METASTORE_DB_PROPERTIES ADD PROPERTYCONTENT varbinary(max); -- HIVE-27457 UPDATE "SDS" - SET "SDS"."INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', - "SDS"."OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' - WHERE "SDS"."SD_ID" IN ( - SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%' + SET "INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', "OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' + WHERE "SD_ID" IN ( + SELECT "TBLS"."SD_ID" + FROM "TBLS" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); + UPDATE "SERDES" SET "SERDES"."SLIB" = 'org.apache.hadoop.hive.kudu.KuduSerDe' WHERE "SERDE_ID" IN ( SELECT "SDS"."SERDE_ID" FROM "TBLS" - LEFT JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" - WHERE "TBL_ID" IN (SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%') + INNER JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); -- These lines need to be last. Insert any changes above. diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mysql.sql index fee7042115fa..0d4b22747610 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.mysql.sql @@ -32,18 +32,22 @@ ALTER TABLE METASTORE_DB_PROPERTIES ADD PROPERTYCONTENT blob; -- HIVE-27457 UPDATE SDS - SET SDS.INPUT_FORMAT = "org.apache.hadoop.hive.kudu.KuduInputFormat", - SDS.OUTPUT_FORMAT = "org.apache.hadoop.hive.kudu.KuduOutputFormat" - WHERE SDS.SD_ID IN ( - SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_VALUE LIKE '%KuduStorageHandler%' + SET INPUT_FORMAT = 'org.apache.hadoop.hive.kudu.KuduInputFormat', OUTPUT_FORMAT = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' + WHERE SD_ID IN ( + SELECT TBLS.SD_ID + FROM TBLS + INNER JOIN TABLE_PARAMS ON TBLS.TBL_ID = TABLE_PARAMS.TBL_ID + WHERE PARAM_VALUE LIKE '%KuduStorageHandler%' ); + UPDATE SERDES SET SERDES.SLIB = "org.apache.hadoop.hive.kudu.KuduSerDe" WHERE SERDE_ID IN ( SELECT SDS.SERDE_ID FROM TBLS - LEFT JOIN SDS ON TBLS.SD_ID = SDS.SD_ID - WHERE TBL_ID IN (SELECT TBL_ID FROM TABLE_PARAMS WHERE PARAM_VALUE LIKE '%KuduStorageHandler%') + INNER JOIN SDS ON TBLS.SD_ID = SDS.SD_ID + INNER JOIN TABLE_PARAMS ON TBLS.TBL_ID = TABLE_PARAMS.TBL_ID + WHERE TABLE_PARAMS.PARAM_VALUE LIKE '%KuduStorageHandler%' ); -- These lines need to be last. Insert any changes above. diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.oracle.sql index 1e25c1271c1e..781609ee1db1 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.oracle.sql @@ -31,18 +31,22 @@ ALTER TABLE METASTORE_DB_PROPERTIES ADD PROPERTYCONTENT BLOB; -- HIVE-27457 UPDATE "SDS" - SET "SDS"."INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', - "SDS"."OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' - WHERE "SDS"."SD_ID" IN ( - SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%' + SET "INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', "OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' + WHERE "SD_ID" IN ( + SELECT "TBLS"."SD_ID" + FROM "TBLS" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); + UPDATE "SERDES" SET "SERDES"."SLIB" = 'org.apache.hadoop.hive.kudu.KuduSerDe' WHERE "SERDE_ID" IN ( SELECT "SDS"."SERDE_ID" FROM "TBLS" - LEFT JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" - WHERE "TBL_ID" IN (SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%') + INNER JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); -- These lines need to be last. Insert any changes above. diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.postgres.sql index fd6459e234a9..d6a3162d842e 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0-beta-1.postgres.sql @@ -31,18 +31,22 @@ ALTER TABLE "METASTORE_DB_PROPERTIES" ADD "PROPERTYCONTENT" bytea; -- HIVE-27457 UPDATE "SDS" - SET "INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', - "OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' - WHERE "SDS"."SD_ID" IN ( - SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%' + SET "INPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduInputFormat', "OUTPUT_FORMAT" = 'org.apache.hadoop.hive.kudu.KuduOutputFormat' + WHERE "SD_ID" IN ( + SELECT "TBLS"."SD_ID" + FROM "TBLS" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%' ); + UPDATE "SERDES" SET "SLIB" = 'org.apache.hadoop.hive.kudu.KuduSerDe' WHERE "SERDE_ID" IN ( SELECT "SDS"."SERDE_ID" FROM "TBLS" - LEFT JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" - WHERE "TBL_ID" IN (SELECT "TBL_ID" FROM "TABLE_PARAMS" WHERE "PARAM_VALUE" LIKE '%KuduStorageHandler%') + INNER JOIN "SDS" ON "TBLS"."SD_ID" = "SDS"."SD_ID" + INNER JOIN "TABLE_PARAMS" ON "TBLS"."TBL_ID" = "TABLE_PARAMS"."TBL_ID" + WHERE "TABLE_PARAMS"."PARAM_VALUE" LIKE '%KuduStorageHandler%' ); -- These lines need to be last. Insert any changes above. From 0e9282b3ed7c59dee3e3a950bd3cd390cca2652b Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Mon, 11 Dec 2023 20:39:54 +0800 Subject: [PATCH 091/179] HIVE-27936: Disable flaky test testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites (#4934)(Butao Zhang, reviewed by Ayush Saxena) --- .../ql/parse/TestReplicationScenariosAcidTablesBootstrap.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java index e393c3dcd9ce..5546f41447e2 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import org.junit.BeforeClass; @@ -238,6 +239,7 @@ public void testAcidTablesBootstrapDuringIncrementalWithOpenTxnsTimeout() throws } @Test + @Ignore("HIVE-27936") public void testBootstrapAcidTablesDuringIncrementalWithConcurrentWrites() throws Throwable { // Dump and load bootstrap without ACID tables. WarehouseInstance.Tuple bootstrapDump = prepareDataAndDump(primaryDbName, From c00891a26282c8615f9b56088efe91fd5ac674e2 Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Wed, 13 Dec 2023 06:37:22 +0100 Subject: [PATCH 092/179] HIVE-27446: Exception when rebuild materialized view incrementally in presence of delete operations (Krisztian Kasa, reviewed by Laszlo Vegh) --- .../AlterMaterializedViewRebuildAnalyzer.java | 7 +- ...nInsertDeleteIncrementalRewritingRule.java | 62 +++++--- .../views/HiveRowIsDeletedPropagator.java | 25 ---- .../calcite/rules/views/InputRefShifter.java | 49 +++++++ .../materialized_view_join_rebuild.q | 33 +++++ .../materialized_view_repeated_rebuild.q | 37 +++++ .../materialized_view_create_rewrite_5.q.out | 112 +++++++-------- .../materialized_view_create_rewrite_8.q.out | 52 +++---- .../llap/materialized_view_join_rebuild.q.out | 108 ++++++++++++++ .../materialized_view_repeated_rebuild.q.out | 134 ++++++++++++++++++ 10 files changed, 486 insertions(+), 133 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/InputRefShifter.java create mode 100644 ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q create mode 100644 ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q create mode 100644 ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out create mode 100644 ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java index c9020b7cbdd8..3542ac06ab0d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java @@ -808,19 +808,20 @@ private void fixUpASTJoinInsertDeleteIncrementalRebuild(ASTNode newAST, Material // We bypass the OR clause and select the first disjunct int indexDelete; int indexInsert; - if (whereClauseInInsert.getChild(0).getChild(0).getType() == HiveParser.DOT) { + if (whereClauseInInsert.getChild(0).getChild(0).getType() == HiveParser.KW_AND) { indexDelete = 0; indexInsert = 1; - } else if (whereClauseInInsert.getChild(0).getChild(1).getType() == HiveParser.DOT) { + } else if (whereClauseInInsert.getChild(0).getChild(1).getType() == HiveParser.KW_AND) { indexDelete = 1; indexInsert = 0; } else { throw new SemanticException("Unexpected condition in incremental rewriting"); } + ASTNode deletePredicate = + (ASTNode) ParseDriver.adaptor.dupTree(whereClauseInInsert.getChild(0).getChild(indexDelete)); ASTNode newCondInInsert = (ASTNode) whereClauseInInsert.getChild(0).getChild(indexInsert); ParseDriver.adaptor.setChild(whereClauseInInsert, 0, newCondInInsert); - ASTNode deletePredicate = (ASTNode) whereClauseInInsert.getChild(0).getChild(indexDelete); addDeleteBranch(insertNode, subqueryNodeInputROJ, deletePredicate, astBuilder); // 3) Add sort node to delete branch diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java index 57274e55d87e..10a9c5c27605 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject; -import org.apache.hadoop.hive.ql.parse.ASTNode; import org.apache.hadoop.hive.ql.parse.CalcitePlanner; import java.util.ArrayList; @@ -46,12 +45,12 @@ * Since CBO plan does not contain the INSERT branches we focus on the SELECT part of the plan in this rule. * See also {@link CalcitePlanner} * - * FROM (select mv.ROW__ID, mv.a, mv.b from mv) mv + * FROM (select mv.ROW__ID, mv.a, mv.b, true as flag from mv) mv * RIGHT OUTER JOIN (SELECT _source_.ROW__IS_DELETED,_source_.a, _source_.b FROM _source_) source * ON (mv.a <=> source.a AND mv.b <=> source.b) * INSERT INTO TABLE mv_delete_delta * SELECT mv.ROW__ID - * WHERE source.ROW__IS__DELETED + * WHERE source.ROW__IS__DELETED AND flag * INSERT INTO TABLE mv * SELECT source.a, source.b * WHERE NOT source.ROW__IS__DELETED @@ -81,7 +80,7 @@ public void onMatch(RelOptRuleCall call) { // expressions for project operator List projExprs = new ArrayList<>(); List joinConjs = new ArrayList<>(); - for (int leftPos = 0; leftPos < joinLeftInput.getRowType().getFieldCount() - 1; leftPos++) { + for (int leftPos = 0; leftPos < joinLeftInput.getRowType().getFieldCount(); leftPos++) { RexNode leftRef = rexBuilder.makeInputRef( joinLeftInput.getRowType().getFieldList().get(leftPos).getType(), leftPos); RexNode rightRef = rexBuilder.makeInputRef( @@ -95,12 +94,6 @@ public void onMatch(RelOptRuleCall call) { RexNode joinCond = RexUtil.composeConjunction(rexBuilder, joinConjs); - int rowIsDeletedIdx = joinRightInput.getRowType().getFieldCount() - 1; - RexNode rowIsDeleted = rexBuilder.makeInputRef( - joinRightInput.getRowType().getFieldList().get(rowIsDeletedIdx).getType(), - joinLeftInput.getRowType().getFieldCount() + rowIsDeletedIdx); - projExprs.add(rowIsDeleted); - // 3) Build plan RelNode newNode = call.builder() .push(union.getInput(1)) @@ -152,6 +145,7 @@ public RelNode visit(HiveJoin join) { } private RelNode createFilter(HiveJoin join) { + RexBuilder rexBuilder = relBuilder.getRexBuilder(); // This should be a Scan on the MV RelNode leftInput = join.getLeft(); @@ -161,34 +155,56 @@ private RelNode createFilter(HiveJoin join) { RelNode tmpJoin = visitChild(join, 1, rightInput); RelNode newRightInput = tmpJoin.getInput(1); + List leftProjects = new ArrayList<>(leftInput.getRowType().getFieldCount() + 1); + List leftProjectNames = new ArrayList<>(leftInput.getRowType().getFieldCount() + 1); + for (int i = 0; i < leftInput.getRowType().getFieldCount(); ++i) { + RelDataTypeField relDataTypeField = leftInput.getRowType().getFieldList().get(i); + leftProjects.add(rexBuilder.makeInputRef(relDataTypeField.getType(), i)); + leftProjectNames.add(relDataTypeField.getName()); + } + List projects = new ArrayList<>(leftProjects.size() + newRightInput.getRowType().getFieldCount()); + projects.addAll(leftProjects); + List projectNames = new ArrayList<>(leftProjects.size() + newRightInput.getRowType().getFieldCount()); + projectNames.addAll(leftProjectNames); + + leftProjects.add(rexBuilder.makeLiteral(true)); + leftProjectNames.add("flag"); + + leftInput = relBuilder + .push(leftInput) + .project(leftProjects, leftProjectNames) + .build(); + + // Create input ref to flag. It is used in filter condition later. + int flagIndex = leftProjects.size() - 1; + RexNode flagNode = rexBuilder.makeInputRef( + leftInput.getRowType().getFieldList().get(flagIndex).getType(), flagIndex); + // Create input ref to rowIsDeleteColumn. It is used in filter condition later. RelDataType newRowType = newRightInput.getRowType(); int rowIsDeletedIdx = newRowType.getFieldCount() - 1; - RexBuilder rexBuilder = relBuilder.getRexBuilder(); RexNode rowIsDeleted = rexBuilder.makeInputRef( - newRowType.getFieldList().get(rowIsDeletedIdx).getType(), - leftInput.getRowType().getFieldCount() + rowIsDeletedIdx); + newRowType.getFieldList().get(rowIsDeletedIdx).getType(), + leftInput.getRowType().getFieldCount() + rowIsDeletedIdx); + + RexNode deleteBranchFilter = rexBuilder.makeCall(SqlStdOperatorTable.AND, flagNode, rowIsDeleted); + RexNode insertBranchFilter = rexBuilder.makeCall(SqlStdOperatorTable.NOT, rowIsDeleted); - List projects = new ArrayList<>(newRowType.getFieldCount()); - List projectNames = new ArrayList<>(newRowType.getFieldCount()); - for (int i = 0; i < leftInput.getRowType().getFieldCount(); ++i) { - RelDataTypeField relDataTypeField = leftInput.getRowType().getFieldList().get(i); - projects.add(rexBuilder.makeInputRef(relDataTypeField.getType(), i)); - projectNames.add(relDataTypeField.getName()); - } for (int i = 0; i < newRowType.getFieldCount() - 1; ++i) { RelDataTypeField relDataTypeField = newRowType.getFieldList().get(i); projects.add(rexBuilder.makeInputRef(relDataTypeField.getType(), leftInput.getRowType().getFieldCount() + i)); projectNames.add(relDataTypeField.getName()); } + RexNode newJoinCondition = new InputRefShifter(leftInput.getRowType().getFieldCount() - 1, relBuilder) + .apply(join.getCondition()); + // Create new Top Right Join and a Filter. The filter condition is used in CalcitePlanner.fixUpASTJoinIncrementalRebuild(). return relBuilder .push(leftInput) .push(newRightInput) - .join(join.getJoinType(), join.getCondition()) - .filter(rexBuilder.makeCall(SqlStdOperatorTable.OR, - rowIsDeleted, rexBuilder.makeCall(SqlStdOperatorTable.NOT, rowIsDeleted))) + .join(join.getJoinType(), newJoinCondition) + .filter(rexBuilder.makeCall(SqlStdOperatorTable.OR, deleteBranchFilter, insertBranchFilter)) .project(projects, projectNames) .build(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveRowIsDeletedPropagator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveRowIsDeletedPropagator.java index 9537acd16a48..8562d7425116 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveRowIsDeletedPropagator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveRowIsDeletedPropagator.java @@ -174,31 +174,6 @@ public RelNode visit(HiveJoin join) { .build(); } - private static class InputRefShifter extends RexShuttle { - private final int startIndex; - private final RelBuilder relBuilder; - - private InputRefShifter(int startIndex, RelBuilder relBuilder) { - this.startIndex = startIndex; - this.relBuilder = relBuilder; - } - - /** - * Shift input reference index by one if the referenced column index is higher or equals with the startIndex. - * @param inputRef - {@link RexInputRef} to transform - * @return new {@link RexInputRef} if the referenced column index is higher or equals with the startIndex, - * original otherwise - */ - @Override - public RexNode visitInputRef(RexInputRef inputRef) { - if (inputRef.getIndex() >= startIndex) { - RexBuilder rexBuilder = relBuilder.getRexBuilder(); - return rexBuilder.makeInputRef(inputRef.getType(), inputRef.getIndex() + 1); - } - return inputRef; - } - } - private void populateProjects(RexBuilder rexBuilder, RelDataType inputRowType, List projects, List projectNames) { populateProjects(rexBuilder, inputRowType, 0, inputRowType.getFieldCount(), projects, projectNames); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/InputRefShifter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/InputRefShifter.java new file mode 100644 index 000000000000..a9d7639c0c2d --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/InputRefShifter.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views; + +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.tools.RelBuilder; + +public class InputRefShifter extends RexShuttle { + private final int startIndex; + private final RelBuilder relBuilder; + + InputRefShifter(int startIndex, RelBuilder relBuilder) { + this.startIndex = startIndex; + this.relBuilder = relBuilder; + } + + /** + * Shift input reference index by one if the referenced column index is higher or equals with the startIndex. + * @param inputRef - {@link RexInputRef} to transform + * @return new {@link RexInputRef} if the referenced column index is higher or equals with the startIndex, + * original otherwise + */ + @Override + public RexNode visitInputRef(RexInputRef inputRef) { + if (inputRef.getIndex() >= startIndex) { + RexBuilder rexBuilder = relBuilder.getRexBuilder(); + return rexBuilder.makeInputRef(inputRef.getType(), inputRef.getIndex() + 1); + } + return inputRef; + } +} diff --git a/ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q b/ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q new file mode 100644 index 000000000000..b104c2b637c5 --- /dev/null +++ b/ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q @@ -0,0 +1,33 @@ +-- Test Incremental rebuild of materialized view without aggregate when source tables have +-- delete operations since last rebuild. +-- The view projects only one column. + +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); + +insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (2, 'bob', 3.14, 3), + (2, 'bonnie', 172342.2, 3), + (3, 'calvin', 978.76, 3), + (3, 'charlie', 9.8, 1); + +create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); + +insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), + (3, 'calvin', 978.76, 3); + +CREATE MATERIALIZED VIEW cmv_mat_view_n6 + TBLPROPERTIES ('transactional'='true') AS + SELECT cmv_basetable_n6.a + FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) + WHERE cmv_basetable_2_n3.c > 10.0; + +DELETE from cmv_basetable_2_n3 WHERE a=1; + +ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD; + +SELECT * FROM cmv_mat_view_n6; diff --git a/ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q b/ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q new file mode 100644 index 000000000000..8fec80b3d2d9 --- /dev/null +++ b/ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q @@ -0,0 +1,37 @@ +-- Test Incremental rebuild of materialized view without aggregate when source tables have +-- 1) delete operations since last rebuild. +-- 2) delete records with the same join key from the other joined table + +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); + +insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (2, 'bob', 3.14, 3), + (2, 'bonnie', 172342.2, 3), + (3, 'calvin', 978.76, 3), + (3, 'charlie', 9.8, 1); + +create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); + +insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), + (3, 'calvin', 978.76, 3); + +CREATE MATERIALIZED VIEW cmv_mat_view_n6 + TBLPROPERTIES ('transactional'='true') AS + SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c + FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) + WHERE cmv_basetable_2_n3.c > 10.0; + +DELETE from cmv_basetable_2_n3 WHERE a=1; + +ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD; + +DELETE FROM cmv_basetable_n6 WHERE a=1; + +ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD; + +SELECT * FROM cmv_mat_view_n6; diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out index 482c113239b5..6c6bf7c1b6ef 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out @@ -532,18 +532,18 @@ STAGE PLANS: Map Operator Tree: TableScan alias: default.cmv_mat_view_n6 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), ROW__ID (type: struct) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int), c (type: decimal(10,2)), true (type: boolean), ROW__ID (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: struct) + key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) + Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: boolean), _col3 (type: struct) Execution mode: llap LLAP IO: may be used (ACID table) Map 5 @@ -601,16 +601,16 @@ STAGE PLANS: condition map: Right Outer Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - nullSafes: [true] - outputColumnNames: _col1, _col2, _col3, _col4 - Statistics: Num rows: 6 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int), _col1 (type: decimal(10,2)) + 1 _col0 (type: int), _col1 (type: decimal(10,2)) + nullSafes: [true, true] + outputColumnNames: _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 5 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: _col4 (type: boolean) - Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col2 and _col6) (type: boolean) + Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col1 (type: struct) + expressions: _col3 (type: struct) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator @@ -620,10 +620,10 @@ STAGE PLANS: Map-reduce partition columns: UDFToInteger(_col0) (type: int) Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (not _col4) (type: boolean) - Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (not _col6) (type: boolean) + Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: int), _col3 (type: decimal(10,2)) + expressions: _col4 (type: int), _col5 (type: decimal(10,2)) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -701,12 +701,12 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2 Data size: 240 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) + key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) Statistics: Num rows: 2 Data size: 240 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: decimal(10,2)), _col2 (type: boolean) + value expressions: _col2 (type: boolean) Stage: Stage-3 Dependency Collection @@ -872,18 +872,18 @@ STAGE PLANS: Map Operator Tree: TableScan alias: default.cmv_mat_view_n6 - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 696 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), ROW__ID (type: struct) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 6 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int), c (type: decimal(10,2)), true (type: boolean), ROW__ID (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 6 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 6 Data size: 480 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: struct) + key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) + Statistics: Num rows: 6 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: boolean), _col3 (type: struct) Execution mode: llap LLAP IO: may be used (ACID table) Map 5 @@ -941,29 +941,29 @@ STAGE PLANS: condition map: Right Outer Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - nullSafes: [true] - outputColumnNames: _col1, _col2, _col3, _col4 - Statistics: Num rows: 10 Data size: 1960 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int), _col1 (type: decimal(10,2)) + 1 _col0 (type: int), _col1 (type: decimal(10,2)) + nullSafes: [true, true] + outputColumnNames: _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 9 Data size: 1640 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: _col4 (type: boolean) - Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col2 and _col6) (type: boolean) + Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col1 (type: struct) + expressions: _col3 (type: struct) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: struct) null sort order: z sort order: + Map-reduce partition columns: UDFToInteger(_col0) (type: int) - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (not _col4) (type: boolean) - Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (not _col6) (type: boolean) + Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: int), _col3 (type: decimal(10,2)) + expressions: _col4 (type: int), _col5 (type: decimal(10,2)) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -996,10 +996,10 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: struct) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat @@ -1041,12 +1041,12 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) + key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) Statistics: Num rows: 3 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: decimal(10,2)), _col2 (type: boolean) + value expressions: _col2 (type: boolean) Stage: Stage-3 Dependency Collection diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out index 6fa072efb478..7aee7df9de4d 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out @@ -402,18 +402,18 @@ STAGE PLANS: Map Operator Tree: TableScan alias: default.cmv_mat_view_n6 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), ROW__ID (type: struct) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int), c (type: decimal(10,2)), true (type: boolean), ROW__ID (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: struct) + key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) + Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col2 (type: boolean), _col3 (type: struct) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) Map 5 @@ -471,16 +471,16 @@ STAGE PLANS: condition map: Right Outer Join 0 to 1 keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - nullSafes: [true] - outputColumnNames: _col1, _col2, _col3, _col4 - Statistics: Num rows: 2 Data size: 392 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int), _col1 (type: decimal(10,2)) + 1 _col0 (type: int), _col1 (type: decimal(10,2)) + nullSafes: [true, true] + outputColumnNames: _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: _col4 (type: boolean) - Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col2 and _col6) (type: boolean) + Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col1 (type: struct) + expressions: _col3 (type: struct) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator @@ -490,10 +490,10 @@ STAGE PLANS: Map-reduce partition columns: UDFToInteger(_col0) (type: int) Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (not _col4) (type: boolean) - Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (not _col6) (type: boolean) + Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: int), _col3 (type: decimal(10,2)) + expressions: _col4 (type: int), _col5 (type: decimal(10,2)) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -571,12 +571,12 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) + key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + null sort order: zz + sort order: ++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: decimal(10,2)), _col2 (type: boolean) + value expressions: _col2 (type: boolean) Stage: Stage-3 Dependency Collection diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out new file mode 100644 index 000000000000..9f66625aaa2f --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out @@ -0,0 +1,108 @@ +PREHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cmv_basetable_n6 +PREHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (2, 'bob', 3.14, 3), + (2, 'bonnie', 172342.2, 3), + (3, 'calvin', 978.76, 3), + (3, 'charlie', 9.8, 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (2, 'bob', 3.14, 3), + (2, 'bonnie', 172342.2, 3), + (3, 'calvin', 978.76, 3), + (3, 'charlie', 9.8, 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: Lineage: cmv_basetable_n6.a SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_n6.b SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_n6.c SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_n6.d SCRIPT [] +PREHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cmv_basetable_2_n3 +PREHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), + (3, 'calvin', 978.76, 3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), + (3, 'calvin', 978.76, 3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: Lineage: cmv_basetable_2_n3.a SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_2_n3.b SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_2_n3.c SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_2_n3.d SCRIPT [] +PREHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 + TBLPROPERTIES ('transactional'='true') AS + SELECT cmv_basetable_n6.a + FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) + WHERE cmv_basetable_2_n3.c > 10.0 +PREHOOK: type: CREATE_MATERIALIZED_VIEW +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Input: default@cmv_basetable_n6 +PREHOOK: Output: database:default +PREHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 + TBLPROPERTIES ('transactional'='true') AS + SELECT cmv_basetable_n6.a + FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) + WHERE cmv_basetable_2_n3.c > 10.0 +POSTHOOK: type: CREATE_MATERIALIZED_VIEW +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Input: default@cmv_basetable_n6 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] +PREHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Output: default@cmv_basetable_2_n3 +PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD +PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Input: default@cmv_basetable_n6 +PREHOOK: Input: default@cmv_mat_view_n6 +PREHOOK: Output: default@cmv_mat_view_n6 +PREHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Input: default@cmv_basetable_n6 +POSTHOOK: Input: default@cmv_mat_view_n6 +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] +PREHOOK: query: SELECT * FROM cmv_mat_view_n6 +PREHOOK: type: QUERY +PREHOOK: Input: default@cmv_mat_view_n6 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM cmv_mat_view_n6 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cmv_mat_view_n6 +#### A masked pattern was here #### +3 +3 diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out new file mode 100644 index 000000000000..9739f87e96e3 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out @@ -0,0 +1,134 @@ +PREHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cmv_basetable_n6 +PREHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (2, 'bob', 3.14, 3), + (2, 'bonnie', 172342.2, 3), + (3, 'calvin', 978.76, 3), + (3, 'charlie', 9.8, 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (2, 'bob', 3.14, 3), + (2, 'bonnie', 172342.2, 3), + (3, 'calvin', 978.76, 3), + (3, 'charlie', 9.8, 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: Lineage: cmv_basetable_n6.a SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_n6.b SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_n6.c SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_n6.d SCRIPT [] +PREHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cmv_basetable_2_n3 +PREHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), + (3, 'calvin', 978.76, 3) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), + (3, 'calvin', 978.76, 3) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: Lineage: cmv_basetable_2_n3.a SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_2_n3.b SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_2_n3.c SCRIPT [] +POSTHOOK: Lineage: cmv_basetable_2_n3.d SCRIPT [] +PREHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 + TBLPROPERTIES ('transactional'='true') AS + SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c + FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) + WHERE cmv_basetable_2_n3.c > 10.0 +PREHOOK: type: CREATE_MATERIALIZED_VIEW +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Input: default@cmv_basetable_n6 +PREHOOK: Output: database:default +PREHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 + TBLPROPERTIES ('transactional'='true') AS + SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c + FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) + WHERE cmv_basetable_2_n3.c > 10.0 +POSTHOOK: type: CREATE_MATERIALIZED_VIEW +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Input: default@cmv_basetable_n6 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] +POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] +PREHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Output: default@cmv_basetable_2_n3 +POSTHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Output: default@cmv_basetable_2_n3 +PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD +PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Input: default@cmv_basetable_n6 +PREHOOK: Input: default@cmv_mat_view_n6 +PREHOOK: Output: default@cmv_mat_view_n6 +PREHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Input: default@cmv_basetable_n6 +POSTHOOK: Input: default@cmv_mat_view_n6 +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] +POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] +PREHOOK: query: DELETE FROM cmv_basetable_n6 WHERE a=1 +PREHOOK: type: QUERY +PREHOOK: Input: default@cmv_basetable_n6 +PREHOOK: Output: default@cmv_basetable_n6 +POSTHOOK: query: DELETE FROM cmv_basetable_n6 WHERE a=1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cmv_basetable_n6 +POSTHOOK: Output: default@cmv_basetable_n6 +PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD +PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +PREHOOK: Input: default@cmv_basetable_2_n3 +PREHOOK: Input: default@cmv_basetable_n6 +PREHOOK: Input: default@cmv_mat_view_n6 +PREHOOK: Output: default@cmv_mat_view_n6 +PREHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +POSTHOOK: Input: default@cmv_basetable_2_n3 +POSTHOOK: Input: default@cmv_basetable_n6 +POSTHOOK: Input: default@cmv_mat_view_n6 +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Output: default@cmv_mat_view_n6 +POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] +POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] +PREHOOK: query: SELECT * FROM cmv_mat_view_n6 +PREHOOK: type: QUERY +PREHOOK: Input: default@cmv_mat_view_n6 +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM cmv_mat_view_n6 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@cmv_mat_view_n6 +#### A masked pattern was here #### +3 978.76 +3 978.76 From 841be7f43e73387c1024c26c26566f336f1972d3 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Wed, 13 Dec 2023 11:15:18 +0200 Subject: [PATCH 093/179] HIVE-27801: Exists subquery rewrite results in a wrong plan (Denys Kuzmenko, reviewed by Attila Turoczy, Ayush Saxena) Closes #4922 --- .../calcite/rules/HiveRelDecorrelator.java | 16 ++--- .../subquery_complex_correlation_predicates.q | 15 +++++ ...query_complex_correlation_predicates.q.out | 66 +++++++++++++++++++ 3 files changed, 85 insertions(+), 12 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java index fd54c0fd689e..f10a19e41ada 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java @@ -3014,18 +3014,10 @@ public boolean traverse(RelNode root) { } /** Builds a {@link org.apache.calcite.sql2rel.RelDecorrelator.CorelMap}. */ private static class CorelMapBuilder extends HiveRelShuttleImpl { - private final SortedMap mapCorToCorRel = - new TreeMap<>(); - - private final SortedSetMultimap mapRefRelToCorRef = - Multimaps.newSortedSetMultimap( - new HashMap>(), - new Supplier>() { - @Override - public TreeSet get() { - return Sets.newTreeSet(); - } - }); + private final SortedMap mapCorToCorRel = new TreeMap<>(); + + private final Multimap mapRefRelToCorRef = + Multimaps.newListMultimap(new HashMap<>(), Lists::newArrayList); private final Map mapFieldAccessToCorVar = new HashMap<>(); diff --git a/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q b/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q index ccfed8dcb23a..1d6dbbee8c04 100644 --- a/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q +++ b/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q @@ -78,3 +78,18 @@ where not exists (select a_authorkey from author a where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400)); + +-- HIVE-27801: Exists subquery rewrite results in a wrong plan +drop table if exists store_sales; +create table store_sales (promo_sk int, sales_price int, list_price int); + +insert into store_sales values (1, 20, 15), (1, 15, 20), (1, 10, 15); + +explain cbo +select * from store_sales A where exists( +select 1 from store_sales B + where A.promo_sk = B.promo_sk and A.sales_price > B.list_price and A.sales_price < B.sales_price); + +select * from store_sales A where exists( +select 1 from store_sales B + where A.promo_sk = B.promo_sk and A.sales_price > B.list_price and A.sales_price < B.sales_price); diff --git a/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out b/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out index fa9768cc5de4..95055833efd6 100644 --- a/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out +++ b/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out @@ -290,3 +290,69 @@ POSTHOOK: Input: default@author POSTHOOK: Input: default@book #### A masked pattern was here #### Men Without Women +PREHOOK: query: drop table if exists store_sales +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists store_sales +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create table store_sales (promo_sk int, sales_price int, list_price int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@store_sales +POSTHOOK: query: create table store_sales (promo_sk int, sales_price int, list_price int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@store_sales +PREHOOK: query: insert into store_sales values (1, 20, 15), (1, 15, 20), (1, 10, 15) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@store_sales +POSTHOOK: query: insert into store_sales values (1, 20, 15), (1, 15, 20), (1, 10, 15) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@store_sales +POSTHOOK: Lineage: store_sales.list_price SCRIPT [] +POSTHOOK: Lineage: store_sales.promo_sk SCRIPT [] +POSTHOOK: Lineage: store_sales.sales_price SCRIPT [] +PREHOOK: query: explain cbo +select * from store_sales A where exists( +select 1 from store_sales B + where A.promo_sk = B.promo_sk and A.sales_price > B.list_price and A.sales_price < B.sales_price) +PREHOOK: type: QUERY +PREHOOK: Input: default@store_sales +#### A masked pattern was here #### +POSTHOOK: query: explain cbo +select * from store_sales A where exists( +select 1 from store_sales B + where A.promo_sk = B.promo_sk and A.sales_price > B.list_price and A.sales_price < B.sales_price) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@store_sales +#### A masked pattern was here #### +CBO PLAN: +HiveSemiJoin(condition=[AND(=($3, $0), =($4, $1))], joinType=[semi]) + HiveProject(promo_sk=[$0], sales_price=[$1], list_price=[$2]) + HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))]) + HiveTableScan(table=[[default, store_sales]], table:alias=[a]) + HiveProject(promo_sk0=[$3], sales_price0=[$4]) + HiveJoin(condition=[AND(=($3, $0), >($4, $2), <($4, $1))], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(promo_sk=[$0], sales_price=[$1], list_price=[$2]) + HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($2), IS NOT NULL($1))]) + HiveTableScan(table=[[default, store_sales]], table:alias=[b]) + HiveProject(promo_sk=[$0], sales_price=[$1]) + HiveAggregate(group=[{0, 1}]) + HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))]) + HiveTableScan(table=[[default, store_sales]], table:alias=[a]) + +PREHOOK: query: select * from store_sales A where exists( +select 1 from store_sales B + where A.promo_sk = B.promo_sk and A.sales_price > B.list_price and A.sales_price < B.sales_price) +PREHOOK: type: QUERY +PREHOOK: Input: default@store_sales +#### A masked pattern was here #### +POSTHOOK: query: select * from store_sales A where exists( +select 1 from store_sales B + where A.promo_sk = B.promo_sk and A.sales_price > B.list_price and A.sales_price < B.sales_price) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@store_sales +#### A masked pattern was here #### From 1455f6201b0f7b061361bc9acc23cb810ff02483 Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Thu, 7 Dec 2023 16:20:17 +0100 Subject: [PATCH 094/179] HIVE-27943: NPE in VectorMapJoinCommonOperator.setUpHashTable when running query with join on date (Stamatis Zampetakis reviewed by Attila Turoczy, Krisztian Kasa) Close apache/hive#4929 --- ...VectorMapJoinOptimizedCreateHashTable.java | 1 + .../queries/clientpositive/mapjoin_date.q | 8 + .../clientpositive/llap/mapjoin_date.q.out | 186 ++++++++++++++++++ 3 files changed, 195 insertions(+) create mode 100644 ql/src/test/queries/clientpositive/mapjoin_date.q create mode 100644 ql/src/test/results/clientpositive/llap/mapjoin_date.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java index 4171f2038c29..dff1f51da603 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedCreateHashTable.java @@ -55,6 +55,7 @@ public static VectorMapJoinOptimizedHashTable createHashTable(MapJoinDesc desc, case SHORT: case INT: case LONG: + case DATE: switch (hashTableKind) { case HASH_MAP: hashTable = new VectorMapJoinOptimizedLongHashMap( diff --git a/ql/src/test/queries/clientpositive/mapjoin_date.q b/ql/src/test/queries/clientpositive/mapjoin_date.q new file mode 100644 index 000000000000..8126c017a90c --- /dev/null +++ b/ql/src/test/queries/clientpositive/mapjoin_date.q @@ -0,0 +1,8 @@ +set hive.auto.convert.join=true; + +CREATE TABLE person (fname string, birthDate date); +INSERT INTO person VALUES ('Victor', '2023-11-27'), ('Alexandre', '2023-11-28'); + +EXPLAIN VECTORIZATION DETAIL SELECT * FROM person p1 INNER JOIN person p2 ON p1.birthDate=p2.birthDate; + +SELECT * FROM person p1 INNER JOIN person p2 ON p1.birthDate=p2.birthDate; diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_date.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_date.q.out new file mode 100644 index 000000000000..c5dfc75a5f30 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/mapjoin_date.q.out @@ -0,0 +1,186 @@ +PREHOOK: query: CREATE TABLE person (fname string, birthDate date) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@person +POSTHOOK: query: CREATE TABLE person (fname string, birthDate date) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@person +PREHOOK: query: INSERT INTO person VALUES ('Victor', '2023-11-27'), ('Alexandre', '2023-11-28') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@person +POSTHOOK: query: INSERT INTO person VALUES ('Victor', '2023-11-27'), ('Alexandre', '2023-11-28') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@person +POSTHOOK: Lineage: person.birthdate SCRIPT [] +POSTHOOK: Lineage: person.fname SCRIPT [] +PREHOOK: query: EXPLAIN VECTORIZATION DETAIL SELECT * FROM person p1 INNER JOIN person p2 ON p1.birthDate=p2.birthDate +PREHOOK: type: QUERY +PREHOOK: Input: default@person +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN VECTORIZATION DETAIL SELECT * FROM person p1 INNER JOIN person p2 ON p1.birthDate=p2.birthDate +POSTHOOK: type: QUERY +POSTHOOK: Input: default@person +#### A masked pattern was here #### +PLAN VECTORIZATION: + enabled: true + enabledConditionsMet: [hive.vectorized.execution.enabled IS true] + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 1 <- Map 2 (BROADCAST_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: p1 + filterExpr: birthdate is not null (type: boolean) + probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:birthdate, smallTablePos:1, keyRatio:0.0 + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + TableScan Vectorization: + native: true + vectorizationSchemaColumns: [0:fname:string, 1:birthdate:date, 2:ROW__ID:struct, 3:ROW__IS__DELETED:boolean] + Filter Operator + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: SelectColumnIsNotNull(col 1:date) + predicate: birthdate is not null (type: boolean) + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: fname (type: string), birthdate (type: date) + outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: date) + 1 _col1 (type: date) + Map Join Vectorization: + bigTableKeyColumns: 1:date + bigTableRetainColumnNums: [0, 1] + bigTableValueColumns: 0:string, 1:date + className: VectorMapJoinInnerLongOperator + native: true + nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Optimized Table and Supports Key Types IS true + nonOuterSmallTableKeyMapping: [] + projectedOutput: 0:string, 1:date, 4:string, 1:date + smallTableValueMapping: 4:string + hashTableImplementationType: OPTIMIZED + outputColumnNames: _col0, _col1, _col2, _col3 + input vertices: + 1 Map 2 + Statistics: Num rows: 4 Data size: 1184 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: false + Statistics: Num rows: 4 Data size: 1184 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Execution mode: vectorized, llap + LLAP IO: all inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + inputFormatFeatureSupport: [DECIMAL_64] + featureSupportInUse: [DECIMAL_64] + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: false + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: fname:string, birthdate:date + partitionColumnCount: 0 + scratchColumnTypeNames: [string] + Map 2 + Map Operator Tree: + TableScan + alias: p2 + filterExpr: birthdate is not null (type: boolean) + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + TableScan Vectorization: + native: true + vectorizationSchemaColumns: [0:fname:string, 1:birthdate:date, 2:ROW__ID:struct, 3:ROW__IS__DELETED:boolean] + Filter Operator + Filter Vectorization: + className: VectorFilterOperator + native: true + predicateExpression: SelectColumnIsNotNull(col 1:date) + predicate: birthdate is not null (type: boolean) + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: fname (type: string), birthdate (type: date) + outputColumnNames: _col0, _col1 + Select Vectorization: + className: VectorSelectOperator + native: true + projectedOutputColumnNums: [0, 1] + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: date) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: date) + Reduce Sink Vectorization: + className: VectorReduceSinkLongOperator + keyColumns: 1:date + native: true + nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true + valueColumns: 0:string + Statistics: Num rows: 2 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map Vectorization: + enabled: true + enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true + inputFormatFeatureSupport: [DECIMAL_64] + featureSupportInUse: [DECIMAL_64] + inputFileFormats: org.apache.hadoop.mapred.TextInputFormat + allNative: true + usesVectorUDFAdaptor: false + vectorized: true + rowBatchContext: + dataColumnCount: 2 + includeColumns: [0, 1] + dataColumns: fname:string, birthdate:date + partitionColumnCount: 0 + scratchColumnTypeNames: [] + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT * FROM person p1 INNER JOIN person p2 ON p1.birthDate=p2.birthDate +PREHOOK: type: QUERY +PREHOOK: Input: default@person +#### A masked pattern was here #### +POSTHOOK: query: SELECT * FROM person p1 INNER JOIN person p2 ON p1.birthDate=p2.birthDate +POSTHOOK: type: QUERY +POSTHOOK: Input: default@person +#### A masked pattern was here #### +Victor 2023-11-27 Victor 2023-11-27 +Alexandre 2023-11-28 Alexandre 2023-11-28 From a455933d3d57fce20b053fa0f4fd023b28bd22de Mon Sep 17 00:00:00 2001 From: Kiran Velumuri Date: Wed, 13 Dec 2023 19:44:35 +0530 Subject: [PATCH 095/179] HIVE-27930: Insert/Load overwrite table partition does not clean up directory before overwriting (#4915)(Kiran Velumuri, reviewed by Indhumathi Muthumurugesh, Butao Zhang) --- .../apache/hadoop/hive/ql/metadata/Hive.java | 7 + ...insert_and_load_overwrite_drop_partition.q | 61 +++++ ...rt_and_load_overwrite_drop_partition.q.out | 256 ++++++++++++++++++ 3 files changed, 324 insertions(+) create mode 100644 ql/src/test/queries/clientpositive/insert_and_load_overwrite_drop_partition.q create mode 100644 ql/src/test/results/clientpositive/llap/insert_and_load_overwrite_drop_partition.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 5022b77fc368..cca06126369b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -5542,6 +5542,13 @@ private void replaceFiles(Path tablePath, Path srcf, Path destf, Path oldPath, H throw new HiveException("Getting globStatus " + srcf.toString(), e); } + // For insert/load overwrite cases, where external.table.purge is disabled for the table, there may be stale + // partitions present in the table location after Alter table drop partition operation. In such cases, oldPath will be + // null, since those partitions will not be present in metastore. Added below check to clean up those stale partitions. + if (oldPath == null && isInsertOverwrite) { + deleteOldPathForReplace(destf, destf, conf, purge, deletePathFilter, isNeedRecycle); + } + // the extra check is required to make ALTER TABLE ... CONCATENATE work if (oldPath != null && (srcs != null || isInsertOverwrite)) { deleteOldPathForReplace(destf, oldPath, conf, purge, deletePathFilter, isNeedRecycle); diff --git a/ql/src/test/queries/clientpositive/insert_and_load_overwrite_drop_partition.q b/ql/src/test/queries/clientpositive/insert_and_load_overwrite_drop_partition.q new file mode 100644 index 000000000000..fabf49022804 --- /dev/null +++ b/ql/src/test/queries/clientpositive/insert_and_load_overwrite_drop_partition.q @@ -0,0 +1,61 @@ +CREATE EXTERNAL TABLE `table1`( + `name` string, + `number` string) +PARTITIONED BY ( + `part_col` string); + +CREATE EXTERNAL TABLE `table2`( + `name` string, + `number` string) +PARTITIONED BY ( + `part_col` string); + +insert into table table1 values ('a', '10', 'part1'); +insert into table table1 values ('b', '11', 'part1'); +insert into table table1 values ('a2', '2', 'part2'); + +insert into table table2 values ('x', '100', 'part1'); +insert into table table2 values ('y', '101', 'part1'); +insert into table table2 values ('z', '102', 'part1'); +insert into table table2 values ('x2', '200', 'part2'); +insert into table table2 values ('y2', '201', 'part2'); +insert into table table2 values ('x3', '300', 'part3'); + +--non empty input case +alter table table2 drop partition(part_col='part1'); + +select count(*) from table2 where part_col='part1'; + +dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/table2/part_col=part1; + +insert overwrite table table2 partition(part_col='part1') select name, number from table1 where part_col='part1'; + +select count(*) from table2 where part_col='part1'; + +dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/table2/part_col=part1; + +--empty input case +alter table table2 drop partition(part_col='part2'); + +select count(*) from table2 where part_col='part2'; + +dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/table2/part_col=part2; + +insert overwrite table table2 partition(part_col='part2') select name, number from table1 where part_col='dummy_part'; + +select count(*) from table2 where part_col='part2'; + +dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/table2/part_col=part2; + +--load overwrite partition +alter table table2 drop partition(part_col='part3'); + +select count(*) from table2 where part_col='part3'; + +dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/table2/part_col=part3; + +LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' OVERWRITE INTO TABLE table2 PARTITION(part_col='part3'); + +select count(*) from table2 where part_col='part3'; + +dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/table2/part_col=part3; \ No newline at end of file diff --git a/ql/src/test/results/clientpositive/llap/insert_and_load_overwrite_drop_partition.q.out b/ql/src/test/results/clientpositive/llap/insert_and_load_overwrite_drop_partition.q.out new file mode 100644 index 000000000000..c16cb464568d --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/insert_and_load_overwrite_drop_partition.q.out @@ -0,0 +1,256 @@ +PREHOOK: query: CREATE EXTERNAL TABLE `table1`( + `name` string, + `number` string) +PARTITIONED BY ( + `part_col` string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table1 +POSTHOOK: query: CREATE EXTERNAL TABLE `table1`( + `name` string, + `number` string) +PARTITIONED BY ( + `part_col` string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table1 +PREHOOK: query: CREATE EXTERNAL TABLE `table2`( + `name` string, + `number` string) +PARTITIONED BY ( + `part_col` string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table2 +POSTHOOK: query: CREATE EXTERNAL TABLE `table2`( + `name` string, + `number` string) +PARTITIONED BY ( + `part_col` string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table2 +PREHOOK: query: insert into table table1 values ('a', '10', 'part1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table1 +POSTHOOK: query: insert into table table1 values ('a', '10', 'part1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table1 +POSTHOOK: Output: default@table1@part_col=part1 +POSTHOOK: Lineage: table1 PARTITION(part_col=part1).name SCRIPT [] +POSTHOOK: Lineage: table1 PARTITION(part_col=part1).number SCRIPT [] +PREHOOK: query: insert into table table1 values ('b', '11', 'part1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table1 +POSTHOOK: query: insert into table table1 values ('b', '11', 'part1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table1 +POSTHOOK: Output: default@table1@part_col=part1 +POSTHOOK: Lineage: table1 PARTITION(part_col=part1).name SCRIPT [] +POSTHOOK: Lineage: table1 PARTITION(part_col=part1).number SCRIPT [] +PREHOOK: query: insert into table table1 values ('a2', '2', 'part2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table1 +POSTHOOK: query: insert into table table1 values ('a2', '2', 'part2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table1 +POSTHOOK: Output: default@table1@part_col=part2 +POSTHOOK: Lineage: table1 PARTITION(part_col=part2).name SCRIPT [] +POSTHOOK: Lineage: table1 PARTITION(part_col=part2).number SCRIPT [] +PREHOOK: query: insert into table table2 values ('x', '100', 'part1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table2 +POSTHOOK: query: insert into table table2 values ('x', '100', 'part1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part1 +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).name SCRIPT [] +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).number SCRIPT [] +PREHOOK: query: insert into table table2 values ('y', '101', 'part1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table2 +POSTHOOK: query: insert into table table2 values ('y', '101', 'part1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part1 +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).name SCRIPT [] +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).number SCRIPT [] +PREHOOK: query: insert into table table2 values ('z', '102', 'part1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table2 +POSTHOOK: query: insert into table table2 values ('z', '102', 'part1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part1 +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).name SCRIPT [] +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).number SCRIPT [] +PREHOOK: query: insert into table table2 values ('x2', '200', 'part2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table2 +POSTHOOK: query: insert into table table2 values ('x2', '200', 'part2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part2 +POSTHOOK: Lineage: table2 PARTITION(part_col=part2).name SCRIPT [] +POSTHOOK: Lineage: table2 PARTITION(part_col=part2).number SCRIPT [] +PREHOOK: query: insert into table table2 values ('y2', '201', 'part2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table2 +POSTHOOK: query: insert into table table2 values ('y2', '201', 'part2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part2 +POSTHOOK: Lineage: table2 PARTITION(part_col=part2).name SCRIPT [] +POSTHOOK: Lineage: table2 PARTITION(part_col=part2).number SCRIPT [] +PREHOOK: query: insert into table table2 values ('x3', '300', 'part3') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@table2 +POSTHOOK: query: insert into table table2 values ('x3', '300', 'part3') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part3 +POSTHOOK: Lineage: table2 PARTITION(part_col=part3).name SCRIPT [] +POSTHOOK: Lineage: table2 PARTITION(part_col=part3).number SCRIPT [] +PREHOOK: query: alter table table2 drop partition(part_col='part1') +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@table2 +PREHOOK: Output: default@table2@part_col=part1 +POSTHOOK: query: alter table table2 drop partition(part_col='part1') +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@table2 +POSTHOOK: Output: default@table2@part_col=part1 +PREHOOK: query: select count(*) from table2 where part_col='part1' +PREHOOK: type: QUERY +PREHOOK: Input: default@table2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from table2 where part_col='part1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table2 +#### A masked pattern was here #### +0 +Found 3 items +#### A masked pattern was here #### +PREHOOK: query: insert overwrite table table2 partition(part_col='part1') select name, number from table1 where part_col='part1' +PREHOOK: type: QUERY +PREHOOK: Input: default@table1 +PREHOOK: Input: default@table1@part_col=part1 +PREHOOK: Output: default@table2@part_col=part1 +POSTHOOK: query: insert overwrite table table2 partition(part_col='part1') select name, number from table1 where part_col='part1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table1 +POSTHOOK: Input: default@table1@part_col=part1 +POSTHOOK: Output: default@table2@part_col=part1 +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).name SIMPLE [(table1)table1.FieldSchema(name:name, type:string, comment:null), ] +POSTHOOK: Lineage: table2 PARTITION(part_col=part1).number SIMPLE [(table1)table1.FieldSchema(name:number, type:string, comment:null), ] +PREHOOK: query: select count(*) from table2 where part_col='part1' +PREHOOK: type: QUERY +PREHOOK: Input: default@table2 +PREHOOK: Input: default@table2@part_col=part1 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from table2 where part_col='part1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table2 +POSTHOOK: Input: default@table2@part_col=part1 +#### A masked pattern was here #### +2 +Found 1 items +#### A masked pattern was here #### +PREHOOK: query: alter table table2 drop partition(part_col='part2') +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@table2 +PREHOOK: Output: default@table2@part_col=part2 +POSTHOOK: query: alter table table2 drop partition(part_col='part2') +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@table2 +POSTHOOK: Output: default@table2@part_col=part2 +PREHOOK: query: select count(*) from table2 where part_col='part2' +PREHOOK: type: QUERY +PREHOOK: Input: default@table2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from table2 where part_col='part2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table2 +#### A masked pattern was here #### +0 +Found 2 items +#### A masked pattern was here #### +PREHOOK: query: insert overwrite table table2 partition(part_col='part2') select name, number from table1 where part_col='dummy_part' +PREHOOK: type: QUERY +PREHOOK: Input: default@table1 +PREHOOK: Output: default@table2@part_col=part2 +POSTHOOK: query: insert overwrite table table2 partition(part_col='part2') select name, number from table1 where part_col='dummy_part' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table1 +POSTHOOK: Output: default@table2@part_col=part2 +POSTHOOK: Lineage: table2 PARTITION(part_col=part2).name SIMPLE [(table1)table1.FieldSchema(name:name, type:string, comment:null), ] +POSTHOOK: Lineage: table2 PARTITION(part_col=part2).number SIMPLE [(table1)table1.FieldSchema(name:number, type:string, comment:null), ] +PREHOOK: query: select count(*) from table2 where part_col='part2' +PREHOOK: type: QUERY +PREHOOK: Input: default@table2 +PREHOOK: Input: default@table2@part_col=part2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from table2 where part_col='part2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table2 +POSTHOOK: Input: default@table2@part_col=part2 +#### A masked pattern was here #### +0 +PREHOOK: query: alter table table2 drop partition(part_col='part3') +PREHOOK: type: ALTERTABLE_DROPPARTS +PREHOOK: Input: default@table2 +PREHOOK: Output: default@table2@part_col=part3 +POSTHOOK: query: alter table table2 drop partition(part_col='part3') +POSTHOOK: type: ALTERTABLE_DROPPARTS +POSTHOOK: Input: default@table2 +POSTHOOK: Output: default@table2@part_col=part3 +PREHOOK: query: select count(*) from table2 where part_col='part3' +PREHOOK: type: QUERY +PREHOOK: Input: default@table2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from table2 where part_col='part3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table2 +#### A masked pattern was here #### +0 +Found 1 items +#### A masked pattern was here #### +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' OVERWRITE INTO TABLE table2 PARTITION(part_col='part3') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@table2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' OVERWRITE INTO TABLE table2 PARTITION(part_col='part3') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@table2 +POSTHOOK: Output: default@table2@part_col=part3 +PREHOOK: query: select count(*) from table2 where part_col='part3' +PREHOOK: type: QUERY +PREHOOK: Input: default@table2 +PREHOOK: Input: default@table2@part_col=part3 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from table2 where part_col='part3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@table2 +POSTHOOK: Input: default@table2@part_col=part3 +#### A masked pattern was here #### +24 +Found 1 items +#### A masked pattern was here #### From 5e9695cbdded7d7a40ecd9fc93ee6803625bb197 Mon Sep 17 00:00:00 2001 From: Mayank Kunwar <55212670+mayankkunwar@users.noreply.github.com> Date: Wed, 13 Dec 2023 23:08:54 +0530 Subject: [PATCH 096/179] HIVE-27892: Hive 'insert overwrite table' for multiple partition table issue (#4893) (Mayank Kunwar, Reviewed by Sai Hemanth Gantasala) --- .../hive/ql/parse/SemanticAnalyzer.java | 8 ++++- .../command/CommandAuthorizerV2.java | 8 +++-- ...stHivePrivilegeObjectOwnerNameAndType.java | 33 +++++++++++++++++++ 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 0823b6d9ba4d..2f3869319573 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -8640,7 +8640,13 @@ private WriteEntity generateTableWriteEntity(String dest, Table dest_tab, new DummyPartition(dest_tab, dest_tab.getDbName() + "@" + dest_tab.getTableName() + "@" + ppath, partSpec); - output = new WriteEntity(p, getWriteType(dest), false); + WriteEntity.WriteType writeType; + if (ltd.isInsertOverwrite()) { + writeType = WriteEntity.WriteType.INSERT_OVERWRITE; + } else { + writeType = getWriteType(dest); + } + output = new WriteEntity(p, writeType, false); output.setDynamicPartitionWrite(true); outputs.add(output); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/command/CommandAuthorizerV2.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/command/CommandAuthorizerV2.java index 08e016223e4a..50fc7975da14 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/command/CommandAuthorizerV2.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/command/CommandAuthorizerV2.java @@ -233,8 +233,12 @@ private static void addHivePrivObject(Entity privObject, Map col = tableName2Cols == null ? null : + tableName2Cols.get(Table.getCompleteName(tbl.getDbName(), tbl.getTableName())); + hivePrivObject = new HivePrivilegeObject(privObjType, tbl.getDbName(), tbl.getTableName(), + null, col, actionType, null, null, tbl.getOwner(), tbl.getOwnerType()); + break; case SERVICE_NAME: hivePrivObject = new HivePrivilegeObject(privObjType, null, privObject.getServiceName(), null, null, actionType, null, null, null, null); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java index a04e5f0227be..410d9b7d9184 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.security.UserGroupInformation; @@ -77,6 +78,7 @@ public static void beforeTest() throws Exception { conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, true); conf.setVar(ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName()); conf.setVar(ConfVars.HIVEMAPREDMODE, "nonstrict"); + conf.setVar(ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); TestTxnDbUtil.prepDb(conf); SessionState.start(conf); @@ -145,6 +147,37 @@ public void testOwnerType() throws Exception { Assert.assertTrue(containsOwnerType); } + @Test + public void testActionTypeForPartitionedTable() throws Exception { + runCmd("CREATE EXTERNAL TABLE Part (eid int, name int) PARTITIONED BY (position int, dept int, sal int)"); + reset(mockedAuthorizer); + runCmd("insert overwrite table part partition(position=2,DEPT,SAL) select 2,2,2,2"); + Pair, List> io = getHivePrivilegeObjectInputs(); + List hpoList = io.getValue(); + Assert.assertFalse(hpoList.isEmpty()); + for (HivePrivilegeObject hpo : hpoList) { + Assert.assertEquals(hpo.getActionType(), HivePrivilegeObject.HivePrivObjectActionType.INSERT_OVERWRITE); + } + } + + /** + * Test to check, if only single instance of Hive Privilege object is created, + * during bulk insert into a partitioned table. + */ + @Test + public void testSingleInstanceOfHPOForPartitionedTable() throws Exception { + reset(mockedAuthorizer); + runCmd("insert overwrite table part partition(position=2,DEPT,SAL)" + + " select 2,2,2,2" + + " union all" + + " select 1,2,3,4" + + " union all" + + " select 3,4,5,6"); + Pair, List> io = getHivePrivilegeObjectInputs(); + List hpoList = io.getValue(); + Assert.assertEquals(1, hpoList.size()); + } + /** * @return pair with left value as inputs and right value as outputs, * passed in current call to authorizer.checkPrivileges From 553ee3b908f89841713044a8e5440b6e19c65103 Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Wed, 13 Dec 2023 13:30:32 +0100 Subject: [PATCH 097/179] HIVE-24219: Disable flaky TestStreaming (Stamatis Zampetakis reviewed by Sourabh Badhya) Recent failure: http://ci.hive.apache.org/job/hive-flaky-check/772 Close apache/hive#4944 --- streaming/src/test/org/apache/hive/streaming/TestStreaming.java | 1 + 1 file changed, 1 insertion(+) diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java index 37c64429b78d..f68c396de7e7 100644 --- a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java +++ b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java @@ -118,6 +118,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@Ignore("HIVE-24219") public class TestStreaming { private static final Logger LOG = LoggerFactory.getLogger(TestStreaming.class); From 3054fb3f138b19986586f0840c01444647bceb84 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Thu, 14 Dec 2023 20:45:28 +0800 Subject: [PATCH 098/179] HIVE-27934: Fix incorrect description about the execution framework in README.md (#4917)(Butao Zhang, reviewed by Stamatis Zampetakis, Attila Turoczy) --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index c31d6108292b..c5f1d18663fa 100644 --- a/README.md +++ b/README.md @@ -24,10 +24,10 @@ subqueries, common table expressions, and more. Hive's SQL can also be extended with user code via user defined functions (UDFs), user defined aggregates (UDAFs), and user defined table functions (UDTFs). -Hive users have a choice of 3 runtimes when executing SQL queries. -Users can choose between Apache Hadoop MapReduce or Apache Tez -frameworks as their execution backend. MapReduce is a -mature framework that is proven at large scales. However, MapReduce +Hive users can choose between Apache Hadoop MapReduce or Apache Tez +frameworks as their execution backend. Note that MapReduce framework +has been deprecated since Hive 2, and Apache Tez is recommended. MapReduce +is a mature framework that is proven at large scales. However, MapReduce is a purely batch framework, and queries using it may experience higher latencies (tens of seconds), even over small datasets. Apache Tez is designed for interactive query, and has substantially reduced From 56497db75c4695b458bbedf460a9f434f8160844 Mon Sep 17 00:00:00 2001 From: Dmitriy Fingerman Date: Sat, 16 Dec 2023 05:22:02 -0500 Subject: [PATCH 099/179] HIVE-27850: Iceberg: Major QB Compaction (Dmitriy Fingerman, reviewed by Attila Turoczy, Ayush Saxena, Butao Zhang, Denys Kuzmenko) Closes #4855 --- .../org/apache/hadoop/hive/conf/HiveConf.java | 8 +- .../iceberg/mr/hive/HiveIcebergMetaHook.java | 3 +- .../mr/hive/HiveIcebergOutputCommitter.java | 24 +- .../mr/hive/HiveIcebergStorageHandler.java | 7 + .../compaction/IcebergCompactionService.java | 77 ++ .../IcebergMajorQueryCompactor.java | 64 ++ ...erg_major_compaction_partition_evolution.q | 65 ++ .../iceberg_major_compaction_partitioned.q | 91 +++ ...ceberg_major_compaction_schema_evolution.q | 66 ++ .../iceberg_major_compaction_unpartitioned.q | 58 ++ ...major_compaction_partition_evolution.q.out | 377 ++++++++++ ...iceberg_major_compaction_partitioned.q.out | 655 ++++++++++++++++++ ...rg_major_compaction_schema_evolution.q.out | 362 ++++++++++ ...eberg_major_compaction_unpartitioned.q.out | 310 +++++++++ .../apache/hive/jdbc/TestRestrictedList.java | 1 + ...estIcebergLlapLocalCompactorCliDriver.java | 83 +++ .../resources/testconfiguration.properties | 6 + .../hadoop/hive/cli/control/CliConfigs.java | 24 + .../org/apache/hadoop/hive/ql/Context.java | 17 + .../compact/AlterTableCompactOperation.java | 2 +- .../hadoop/hive/ql/txn/compactor/Cleaner.java | 4 +- .../compactor/CompactionHeartbeatService.java | 8 +- .../ql/txn/compactor/CompactorContext.java | 6 +- .../ql/txn/compactor/CompactorFactory.java | 21 + .../ql/txn/compactor/CompactorThread.java | 101 +-- .../hive/ql/txn/compactor/CompactorUtil.java | 107 +++ .../hive/ql/txn/compactor/Initiator.java | 6 +- .../hive/ql/txn/compactor/InitiatorBase.java | 12 +- .../compactor/MetaStoreCompactorThread.java | 5 + .../hive/ql/txn/compactor/QueryCompactor.java | 37 +- .../txn/compactor/RemoteCompactorThread.java | 24 +- .../ql/txn/compactor/RemoteCompactorUtil.java | 62 ++ .../hadoop/hive/ql/txn/compactor/Worker.java | 402 +---------- .../ql/txn/compactor/handler/TaskHandler.java | 24 +- .../service/AcidCompactionService.java | 420 +++++++++++ .../service/CompactionExecutorFactory.java | 58 ++ .../compactor/service/CompactionService.java | 86 +++ 37 files changed, 3121 insertions(+), 562 deletions(-) create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution.q create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partitioned.q create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_schema_evolution.q create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out create mode 100644 iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out create mode 100644 iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out create mode 100644 iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out create mode 100644 itests/qtest-iceberg/src/test/java/org/apache/hadoop/hive/cli/TestIcebergLlapLocalCompactorCliDriver.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionExecutorFactory.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 2497ab367358..714df4c22a96 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -5579,7 +5579,8 @@ public static enum ConfVars { "hive.zookeeper.ssl.truststore.location," + "hive.zookeeper.ssl.truststore.password," + "hive.zookeeper.ssl.truststore.type," + - "hive.iceberg.allow.datafiles.in.table.location.only", + "hive.iceberg.allow.datafiles.in.table.location.only," + + "hive.rewrite.data.policy", "Comma separated list of configuration options which are immutable at runtime"), HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list", METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname @@ -5739,7 +5740,10 @@ public static enum ConfVars { HIVE_ADDITIONAL_CONFIG_FILES("hive.additional.config.files", "", "The names of additional config files, such as ldap-site.xml," + - "tez-site.xml, etc in comma separated list."); + "tez-site.xml, etc in comma separated list."), + + REWRITE_POLICY("hive.rewrite.data.policy", "DEFAULT", + "Defines the rewrite policy, the valid values are those defined in RewritePolicy enum"); public final String varname; public final String altName; diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 540a3b8638c0..9a108e519726 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -148,7 +148,8 @@ public class HiveIcebergMetaHook implements HiveMetaHook { AlterTableType.ADDCOLS, AlterTableType.REPLACE_COLUMNS, AlterTableType.RENAME_COLUMN, AlterTableType.ADDPROPS, AlterTableType.DROPPROPS, AlterTableType.SETPARTITIONSPEC, AlterTableType.UPDATE_COLUMNS, AlterTableType.RENAME, AlterTableType.EXECUTE, AlterTableType.CREATE_BRANCH, - AlterTableType.CREATE_TAG, AlterTableType.DROP_BRANCH, AlterTableType.DROPPARTITION, AlterTableType.DROP_TAG); + AlterTableType.CREATE_TAG, AlterTableType.DROP_BRANCH, AlterTableType.DROPPARTITION, AlterTableType.DROP_TAG, + AlterTableType.COMPACT); private static final List MIGRATION_ALLOWED_SOURCE_FORMATS = ImmutableList.of( FileFormat.PARQUET.name().toLowerCase(), FileFormat.ORC.name().toLowerCase(), diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index 37941c62761a..c4a6005818ae 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -42,7 +42,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.Context.Operation; +import org.apache.hadoop.hive.ql.Context.RewritePolicy; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.session.SessionStateUtil; import org.apache.hadoop.mapred.JobConf; @@ -62,6 +64,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.hadoop.Util; @@ -453,7 +456,13 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output commitWrite(table, branchName, snapshotId, startTime, filesForCommit, operation); } } else { - commitOverwrite(table, branchName, startTime, filesForCommit); + + RewritePolicy rewritePolicy = RewritePolicy.fromString(outputTable.jobContexts.stream() + .findAny() + .map(x -> x.getJobConf().get(ConfVars.REWRITE_POLICY.varname)) + .orElse(RewritePolicy.DEFAULT.name())); + + commitOverwrite(table, branchName, startTime, filesForCommit, rewritePolicy); } } @@ -532,16 +541,25 @@ private void commitWrite(Table table, String branchName, Long snapshotId, long s * @param table The table we are changing * @param startTime The start time of the commit - used only for logging * @param results The object containing the new files + * @param rewritePolicy The rewrite policy to use for the insert overwrite commit */ - private void commitOverwrite(Table table, String branchName, long startTime, FilesForCommit results) { + private void commitOverwrite(Table table, String branchName, long startTime, FilesForCommit results, + RewritePolicy rewritePolicy) { Preconditions.checkArgument(results.deleteFiles().isEmpty(), "Can not handle deletes with overwrite"); if (!results.dataFiles().isEmpty()) { - ReplacePartitions overwrite = table.newReplacePartitions(); + Transaction transaction = table.newTransaction(); + if (rewritePolicy == RewritePolicy.ALL_PARTITIONS) { + DeleteFiles delete = transaction.newDelete(); + delete.deleteFromRowFilter(Expressions.alwaysTrue()); + delete.commit(); + } + ReplacePartitions overwrite = transaction.newReplacePartitions(); results.dataFiles().forEach(overwrite::addFile); if (StringUtils.isNotEmpty(branchName)) { overwrite.toBranch(HiveUtils.getTableSnapshotRef(branchName)); } overwrite.commit(); + transaction.commitTransaction(); LOG.info("Overwrite commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, table, results.dataFiles().size()); } else if (table.spec().isUnpartitioned()) { diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 9b430af805c6..6e9e9870cfe8 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hive.common.type.Timestamp; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.HiveMetaHook; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; @@ -66,6 +67,7 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Context.Operation; +import org.apache.hadoop.hive.ql.Context.RewritePolicy; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; @@ -1106,6 +1108,11 @@ public void validateSinkDesc(FileSinkDesc sinkDesc) throws SemanticException { // If the table is empty we don't have any danger that some data can get lost. return; } + if (RewritePolicy.fromString(conf.get(ConfVars.REWRITE_POLICY.varname, RewritePolicy.DEFAULT.name())) == + RewritePolicy.ALL_PARTITIONS) { + // Table rewriting has special logic as part of IOW that handles the case when table had a partition evolution + return; + } if (IcebergTableUtil.isBucketed(table)) { throw new SemanticException("Cannot perform insert overwrite query on bucket partitioned Iceberg table."); } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java new file mode 100644 index 000000000000..e0ff9782c690 --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.mr.hive.compaction; + +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorPipeline; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; +import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergCompactionService extends CompactionService { + private static final String CLASS_NAME = IcebergCompactionService.class.getName(); + private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); + + public IcebergCompactionService() { + } + + public Boolean compact(Table table, CompactionInfo ci) throws Exception { + + if (!ci.isMajorCompaction()) { + ci.errorMessage = "Presently Iceberg tables support only Major compaction"; + LOG.error(ci.errorMessage + " Compaction info: {}", ci); + try { + msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); + } catch (Throwable tr) { + LOG.error("Caught an exception while trying to mark compaction {} as failed: {}", ci, tr); + } + return false; + } + CompactorUtil.checkInterrupt(CLASS_NAME); + + try { + CompactorPipeline compactorPipeline = compactorFactory.getCompactorPipeline(table, conf, ci, msc); + computeStats = collectGenericStats; + + LOG.info("Starting " + ci.type.toString() + " compaction for " + ci.getFullPartitionName() + ", id:" + + ci.id + " with compute stats set to " + computeStats); + + CompactorContext compactorContext = new CompactorContext(conf, table, ci); + compactorPipeline.execute(compactorContext); + + LOG.info("Completed " + ci.type.toString() + " compaction for " + ci.getFullPartitionName() + + ", marking as compacted."); + msc.markCleaned(CompactionInfo.compactionInfoToStruct(ci)); + + } catch (Throwable e) { + computeStats = false; + throw e; + } + + return true; + } + + @Override + public void cleanupResultDirs(CompactionInfo ci) { + + } +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java new file mode 100644 index 000000000000..96141e50494d --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive.compaction; + +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.Context.RewritePolicy; +import org.apache.hadoop.hive.ql.DriverUtils; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext; +import org.apache.hadoop.hive.ql.txn.compactor.QueryCompactor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergMajorQueryCompactor extends QueryCompactor { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergMajorQueryCompactor.class.getName()); + + @Override + public boolean run(CompactorContext context) throws IOException, HiveException, InterruptedException { + + String compactTableName = context.getTable().getTableName(); + Map tblProperties = context.getTable().getParameters(); + LOG.debug("Initiating compaction for the {} table", compactTableName); + + String compactionQuery = String.format("insert overwrite table %s select * from %s", + compactTableName, compactTableName); + + SessionState sessionState = setupQueryCompactionSession(context.getConf(), + context.getCompactionInfo(), tblProperties); + HiveConf.setVar(context.getConf(), ConfVars.REWRITE_POLICY, RewritePolicy.ALL_PARTITIONS.name()); + try { + DriverUtils.runOnDriver(context.getConf(), sessionState, compactionQuery); + LOG.info("Completed compaction for table {}", compactTableName); + } catch (HiveException e) { + LOG.error("Error doing query based {} compaction", RewritePolicy.ALL_PARTITIONS.name(), e); + throw new RuntimeException(e); + } finally { + sessionState.setCompaction(false); + } + + return true; + } +} diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution.q new file mode 100644 index 000000000000..67b44d625a55 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution.q @@ -0,0 +1,65 @@ +-- SORT_QUERY_RESULTS +-- Mask neededVirtualColumns due to non-strict order +--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ +-- Mask the totalSize value as it can have slight variability, causing test flakiness +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/ +-- Mask random uuid +--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask a random snapshot id +--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/ +-- Mask added file size +--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask total file size +--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask current-snapshot-timestamp-ms +--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/ +--! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ +-- Mask compaction id as they will be allocated in parallel threads +--! qt:replace:/^[0-9]/#Masked#/ + +set hive.llap.io.enabled=true; +set hive.vectorized.execution.enabled=true; +set hive.optimize.shared.work.merge.ts.schema=true; + +create table ice_orc ( + first_name string, + last_name string, + dept_id bigint, + team_id bigint + ) +partitioned by (company_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2'); + +insert into ice_orc VALUES ('fn1','ln1', 1, 10, 100); +insert into ice_orc VALUES ('fn2','ln2', 1, 10, 100); +insert into ice_orc VALUES ('fn3','ln3', 1, 11, 100); +alter table ice_orc set partition spec(company_id, dept_id); +insert into ice_orc VALUES ('fn4','ln4', 1, 11, 100); +insert into ice_orc VALUES ('fn5','ln5', 2, 20, 100); +insert into ice_orc VALUES ('fn6','ln6', 2, 20, 100); +alter table ice_orc set partition spec(company_id, dept_id, team_id); +insert into ice_orc VALUES ('fn7','ln7', 2, 21, 100); +insert into ice_orc VALUES ('fn8','ln8', 2, 21, 100); + +update ice_orc set last_name = 'ln1a' where first_name='fn1'; +update ice_orc set last_name = 'ln2a' where first_name='fn2'; +update ice_orc set last_name = 'ln3a' where first_name='fn3'; +update ice_orc set last_name = 'ln4a' where first_name='fn4'; +alter table ice_orc set partition spec(company_id, dept_id); +update ice_orc set last_name = 'ln5a' where first_name='fn5'; +update ice_orc set last_name = 'ln6a' where first_name='fn6'; +update ice_orc set last_name = 'ln7a' where first_name='fn7'; +update ice_orc set last_name = 'ln8a' where first_name='fn8'; + +delete from ice_orc where last_name in ('ln1a', 'ln8a'); + +select * from ice_orc; +describe formatted ice_orc; + +explain alter table ice_orc COMPACT 'major' and wait; +alter table ice_orc COMPACT 'major' and wait; + +select * from ice_orc; +describe formatted ice_orc; +show compactions; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partitioned.q new file mode 100644 index 000000000000..4b8dc30bdaa3 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partitioned.q @@ -0,0 +1,91 @@ +-- SORT_QUERY_RESULTS +-- Mask neededVirtualColumns due to non-strict order +--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ +-- Mask the totalSize value as it can have slight variability, causing test flakiness +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/ +-- Mask random uuid +--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask a random snapshot id +--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/ +-- Mask added file size +--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask total file size +--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask current-snapshot-timestamp-ms +--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/ +--! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ +-- Mask compaction id as they will be allocated in parallel threads +--! qt:replace:/^[0-9]/#Masked#/ + +set hive.llap.io.enabled=true; +set hive.vectorized.execution.enabled=true; +set hive.optimize.shared.work.merge.ts.schema=true; + +create table ice_orc ( + first_name string, + last_name string + ) +partitioned by (dept_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2'); + +insert into ice_orc VALUES ('fn1','ln1', 1); +insert into ice_orc VALUES ('fn2','ln2', 1); +insert into ice_orc VALUES ('fn3','ln3', 1); +insert into ice_orc VALUES ('fn4','ln4', 1); +insert into ice_orc VALUES ('fn5','ln5', 2); +insert into ice_orc VALUES ('fn6','ln6', 2); +insert into ice_orc VALUES ('fn7','ln7', 2); + +update ice_orc set last_name = 'ln1a' where first_name='fn1'; +update ice_orc set last_name = 'ln2a' where first_name='fn2'; +update ice_orc set last_name = 'ln3a' where first_name='fn3'; +update ice_orc set last_name = 'ln4a' where first_name='fn4'; +update ice_orc set last_name = 'ln5a' where first_name='fn5'; +update ice_orc set last_name = 'ln6a' where first_name='fn6'; +update ice_orc set last_name = 'ln7a' where first_name='fn7'; + +delete from ice_orc where last_name in ('ln1a', 'ln2a', 'ln7a'); + +select * from ice_orc; +describe formatted ice_orc; + +explain alter table ice_orc COMPACT 'major' and wait; +alter table ice_orc COMPACT 'major' and wait; + +select * from ice_orc; +describe formatted ice_orc; +show compactions; + +-- Starting second set of inserts/updates/deletes and calling compaction at the end +-- to check that subsequent compaction works + +insert into ice_orc VALUES ('fn11','ln11', 1); +insert into ice_orc VALUES ('fn12','ln12', 1); +insert into ice_orc VALUES ('fn13','ln13', 1); +insert into ice_orc VALUES ('fn14','ln14', 1); +insert into ice_orc VALUES ('fn15','ln15', 2); +insert into ice_orc VALUES ('fn16','ln16', 2); +insert into ice_orc VALUES ('fn17','ln17', 2); +insert into ice_orc VALUES ('fn18','ln18', 2); + +update ice_orc set last_name = 'ln11a' where first_name='fn11'; +update ice_orc set last_name = 'ln12a' where first_name='fn12'; +update ice_orc set last_name = 'ln13a' where first_name='fn13'; +update ice_orc set last_name = 'ln14a' where first_name='fn14'; +update ice_orc set last_name = 'ln15a' where first_name='fn15'; +update ice_orc set last_name = 'ln16a' where first_name='fn16'; +update ice_orc set last_name = 'ln17a' where first_name='fn17'; +update ice_orc set last_name = 'ln18a' where first_name='fn18'; + +delete from ice_orc where last_name in ('ln11a', 'ln12a', 'ln17a', 'ln18a'); + +select * from ice_orc; +describe formatted ice_orc; + +explain alter table ice_orc COMPACT 'major' and wait; +alter table ice_orc COMPACT 'major' and wait; + +select * from ice_orc; +describe formatted ice_orc; +show compactions; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_schema_evolution.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_schema_evolution.q new file mode 100644 index 000000000000..939904a8cba1 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_schema_evolution.q @@ -0,0 +1,66 @@ +-- SORT_QUERY_RESULTS +-- Mask neededVirtualColumns due to non-strict order +--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ +-- Mask the totalSize value as it can have slight variability, causing test flakiness +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/ +-- Mask random uuid +--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask a random snapshot id +--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/ +-- Mask added file size +--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask total file size +--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask current-snapshot-timestamp-ms +--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/ +--! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ +-- Mask compaction id as they will be allocated in parallel threads +--! qt:replace:/^[0-9]/#Masked#/ + +set hive.llap.io.enabled=true; +set hive.vectorized.execution.enabled=true; +set hive.optimize.shared.work.merge.ts.schema=true; + +create table ice_orc ( + first_name string, + last_name string + ) +partitioned by (dept_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2'); + +insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn1','ln1', 1); +insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn2','ln2', 1); +insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn3','ln3', 1); +insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn4','ln4', 1); + +alter table ice_orc add columns (address string); + +insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn5','ln5', 'addr_5', 2); +insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn6','ln6', 'addr_6', 2); +insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn7','ln7', 'addr_7', 2); +insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn8','ln8', 'addr_8', 2); + +update ice_orc set last_name = 'ln1a' where first_name='fn1'; +update ice_orc set last_name = 'ln2a' where first_name='fn2'; +update ice_orc set last_name = 'ln3a' where first_name='fn3'; +update ice_orc set last_name = 'ln4a' where first_name='fn4'; + +alter table ice_orc change first_name fname string; + +update ice_orc set last_name = 'ln5a' where fname='fn5'; +update ice_orc set last_name = 'ln6a' where fname='fn6'; +update ice_orc set last_name = 'ln7a' where fname='fn7'; +update ice_orc set last_name = 'ln8a' where fname='fn8'; + +delete from ice_orc where fname in ('fn1', 'fn3', 'fn7'); + +select * from ice_orc; +describe formatted ice_orc; + +explain alter table ice_orc COMPACT 'major' and wait; +alter table ice_orc COMPACT 'major' and wait; + +select * from ice_orc; +describe formatted ice_orc; +show compactions; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q new file mode 100644 index 000000000000..92efb00767ec --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q @@ -0,0 +1,58 @@ +-- SORT_QUERY_RESULTS +-- Mask neededVirtualColumns due to non-strict order +--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ +-- Mask the totalSize value as it can have slight variability, causing test flakiness +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/ +-- Mask random uuid +--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask a random snapshot id +--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/ +-- Mask added file size +--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask total file size +--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask current-snapshot-timestamp-ms +--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask the enqueue time which is based on current time +--! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ +-- Mask compaction id as they will be allocated in parallel threads +--! qt:replace:/^[0-9]/#Masked#/ + +set hive.llap.io.enabled=true; +set hive.vectorized.execution.enabled=true; +set hive.optimize.shared.work.merge.ts.schema=true; + +create table ice_orc ( + first_name string, + last_name string + ) +stored by iceberg stored as orc +tblproperties ('format-version'='2'); + +insert into ice_orc VALUES ('fn1','ln1'); +insert into ice_orc VALUES ('fn2','ln2'); +insert into ice_orc VALUES ('fn3','ln3'); +insert into ice_orc VALUES ('fn4','ln4'); +insert into ice_orc VALUES ('fn5','ln5'); +insert into ice_orc VALUES ('fn6','ln6'); +insert into ice_orc VALUES ('fn7','ln7'); + +update ice_orc set last_name = 'ln1a' where first_name='fn1'; +update ice_orc set last_name = 'ln2a' where first_name='fn2'; +update ice_orc set last_name = 'ln3a' where first_name='fn3'; +update ice_orc set last_name = 'ln4a' where first_name='fn4'; +update ice_orc set last_name = 'ln5a' where first_name='fn5'; +update ice_orc set last_name = 'ln6a' where first_name='fn6'; +update ice_orc set last_name = 'ln7a' where first_name='fn7'; + +delete from ice_orc where last_name in ('ln5a', 'ln6a', 'ln7a'); + +select * from ice_orc; +describe formatted ice_orc; + +explain alter table ice_orc COMPACT 'major' and wait; +alter table ice_orc COMPACT 'major' and wait; + +select * from ice_orc; +describe formatted ice_orc; +show compactions; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out new file mode 100644 index 000000000000..a5ead609d451 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out @@ -0,0 +1,377 @@ +PREHOOK: query: create table ice_orc ( + first_name string, + last_name string, + dept_id bigint, + team_id bigint + ) +partitioned by (company_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_orc +POSTHOOK: query: create table ice_orc ( + first_name string, + last_name string, + dept_id bigint, + team_id bigint + ) +partitioned by (company_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn1','ln1', 1, 10, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn1','ln1', 1, 10, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn2','ln2', 1, 10, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn2','ln2', 1, 10, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn3','ln3', 1, 11, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn3','ln3', 1, 11, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: alter table ice_orc set partition spec(company_id, dept_id) +PREHOOK: type: ALTERTABLE_SETPARTSPEC +PREHOOK: Input: default@ice_orc +POSTHOOK: query: alter table ice_orc set partition spec(company_id, dept_id) +POSTHOOK: type: ALTERTABLE_SETPARTSPEC +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn4','ln4', 1, 11, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn4','ln4', 1, 11, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn5','ln5', 2, 20, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn5','ln5', 2, 20, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn6','ln6', 2, 20, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn6','ln6', 2, 20, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: alter table ice_orc set partition spec(company_id, dept_id, team_id) +PREHOOK: type: ALTERTABLE_SETPARTSPEC +PREHOOK: Input: default@ice_orc +POSTHOOK: query: alter table ice_orc set partition spec(company_id, dept_id, team_id) +POSTHOOK: type: ALTERTABLE_SETPARTSPEC +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn7','ln7', 2, 21, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn7','ln7', 2, 21, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn8','ln8', 2, 21, 100) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn8','ln8', 2, 21, 100) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: alter table ice_orc set partition spec(company_id, dept_id) +PREHOOK: type: ALTERTABLE_SETPARTSPEC +PREHOOK: Input: default@ice_orc +POSTHOOK: query: alter table ice_orc set partition spec(company_id, dept_id) +POSTHOOK: type: ALTERTABLE_SETPARTSPEC +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln8a' where first_name='fn8' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln8a' where first_name='fn8' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: delete from ice_orc where last_name in ('ln1a', 'ln8a') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: delete from ice_orc where last_name in ('ln1a', 'ln8a') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn2 ln2a 1 10 100 +fn3 ln3a 1 11 100 +fn4 ln4a 1 11 100 +fn5 ln5a 2 20 100 +fn6 ln6a 2 20 100 +fn7 ln7a 2 21 100 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string +dept_id bigint +team_id bigint +company_id bigint + +# Partition Transform Information +# col_name transform_type +company_id IDENTITY +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"},{\"id\":4,\"name\":\"team_id\",\"required\":false,\"type\":\"long\"},{\"id\":5,\"name\":\"company_id\",\"required\":false,\"type\":\"long\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"deleted-data-files\":\"2\",\"deleted-records\":\"2\",\"removed-files-size\":\"1256\",\"changed-partition-count\":\"2\",\"total-records\":\"14\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"14\",\"total-delete-files\":\"8\",\"total-position-deletes\":\"8\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":1,\"fields\":[{\"name\":\"company_id\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 14 + numRows 14 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 17 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.ice_orc + numberOfBuckets: 0 + table name: default.ice_orc + blocking: true + +PREHOOK: query: alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn2 ln2a 1 10 100 +fn3 ln3a 1 11 100 +fn4 ln4a 1 11 100 +fn5 ln5a 2 20 100 +fn6 ln6a 2 20 100 +fn7 ln7a 2 21 100 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string +dept_id bigint +team_id bigint +company_id bigint + +# Partition Transform Information +# col_name transform_type +company_id IDENTITY +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"company_id\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"team_id\":\"true\"}} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"},{\"id\":4,\"name\":\"team_id\",\"required\":false,\"type\":\"long\"},{\"id\":5,\"name\":\"company_id\",\"required\":false,\"type\":\"long\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"2\",\"added-records\":\"6\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"6\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":1,\"fields\":[{\"name\":\"company_id\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 2 + numRows 6 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 19 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: show compactions +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: show compactions +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out new file mode 100644 index 000000000000..095957023144 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out @@ -0,0 +1,655 @@ +PREHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +partitioned by (dept_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_orc +POSTHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +partitioned by (dept_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn1','ln1', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn1','ln1', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn2','ln2', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn2','ln2', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn3','ln3', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn3','ln3', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn4','ln4', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn4','ln4', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn5','ln5', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn5','ln5', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn6','ln6', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn6','ln6', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn7','ln7', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn7','ln7', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: delete from ice_orc where last_name in ('ln1a', 'ln2a', 'ln7a') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: delete from ice_orc where last_name in ('ln1a', 'ln2a', 'ln7a') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn3 ln3a 1 +fn4 ln4a 1 +fn5 ln5a 2 +fn6 ln6a 2 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string +dept_id bigint + +# Partition Transform Information +# col_name transform_type +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"1440\",\"changed-partition-count\":\"2\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 11 + numRows 11 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 15 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.ice_orc + numberOfBuckets: 0 + table name: default.ice_orc + blocking: true + +PREHOOK: query: alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn3 ln3a 1 +fn4 ln4a 1 +fn5 ln5a 2 +fn6 ln6a 2 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string +dept_id bigint + +# Partition Transform Information +# col_name transform_type +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\"}} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"2\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 2 + numRows 4 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 17 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: show compactions +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: show compactions +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- +PREHOOK: query: insert into ice_orc VALUES ('fn11','ln11', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn11','ln11', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn12','ln12', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn12','ln12', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn13','ln13', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn13','ln13', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn14','ln14', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn14','ln14', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn15','ln15', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn15','ln15', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn16','ln16', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn16','ln16', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn17','ln17', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn17','ln17', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn18','ln18', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn18','ln18', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln11a' where first_name='fn11' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln11a' where first_name='fn11' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln12a' where first_name='fn12' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln12a' where first_name='fn12' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln13a' where first_name='fn13' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln13a' where first_name='fn13' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln14a' where first_name='fn14' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln14a' where first_name='fn14' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln15a' where first_name='fn15' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln15a' where first_name='fn15' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln16a' where first_name='fn16' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln16a' where first_name='fn16' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln17a' where first_name='fn17' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln17a' where first_name='fn17' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln18a' where first_name='fn18' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln18a' where first_name='fn18' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: delete from ice_orc where last_name in ('ln11a', 'ln12a', 'ln17a', 'ln18a') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: delete from ice_orc where last_name in ('ln11a', 'ln12a', 'ln17a', 'ln18a') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn13 ln13a 1 +fn14 ln14a 1 +fn15 ln15a 2 +fn16 ln16a 2 +fn3 ln3a 1 +fn4 ln4a 1 +fn5 ln5a 2 +fn6 ln6a 2 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string +dept_id bigint + +# Partition Transform Information +# col_name transform_type +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"deleted-data-files\":\"4\",\"deleted-records\":\"4\",\"removed-files-size\":\"1948\",\"changed-partition-count\":\"2\",\"total-records\":\"16\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"14\",\"total-delete-files\":\"8\",\"total-position-deletes\":\"8\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 14 + numRows 16 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 34 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.ice_orc + numberOfBuckets: 0 + table name: default.ice_orc + blocking: true + +PREHOOK: query: alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn13 ln13a 1 +fn14 ln14a 1 +fn15 ln15a 2 +fn16 ln16a 2 +fn3 ln3a 1 +fn4 ln4a 1 +fn5 ln5a 2 +fn6 ln6a 2 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string +dept_id bigint + +# Partition Transform Information +# col_name transform_type +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\"}} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"2\",\"added-records\":\"8\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"8\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 2 + numRows 8 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 36 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: show compactions +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: show compactions +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out new file mode 100644 index 000000000000..f2f453b34d09 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out @@ -0,0 +1,362 @@ +PREHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +partitioned by (dept_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_orc +POSTHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +partitioned by (dept_id bigint) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn1','ln1', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn1','ln1', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn2','ln2', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn2','ln2', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn3','ln3', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn3','ln3', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn4','ln4', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, dept_id) VALUES ('fn4','ln4', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: alter table ice_orc add columns (address string) +PREHOOK: type: ALTERTABLE_ADDCOLS +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc add columns (address string) +POSTHOOK: type: ALTERTABLE_ADDCOLS +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn5','ln5', 'addr_5', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn5','ln5', 'addr_5', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn6','ln6', 'addr_6', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn6','ln6', 'addr_6', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn7','ln7', 'addr_7', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn7','ln7', 'addr_7', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn8','ln8', 'addr_8', 2) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc (first_name, last_name, address, dept_id) VALUES ('fn8','ln8', 'addr_8', 2) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: alter table ice_orc change first_name fname string +PREHOOK: type: ALTERTABLE_RENAMECOL +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc change first_name fname string +POSTHOOK: type: ALTERTABLE_RENAMECOL +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln5a' where fname='fn5' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln5a' where fname='fn5' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln6a' where fname='fn6' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln6a' where fname='fn6' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln7a' where fname='fn7' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln7a' where fname='fn7' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln8a' where fname='fn8' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln8a' where fname='fn8' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: delete from ice_orc where fname in ('fn1', 'fn3', 'fn7') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: delete from ice_orc where fname in ('fn1', 'fn3', 'fn7') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn2 ln2a 1 NULL +fn4 ln4a 1 NULL +fn5 ln5a 2 addr_5 +fn6 ln6a 2 addr_6 +fn8 ln8a 2 addr_8 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +fname string +last_name string +dept_id bigint +address string + +# Partition Transform Information +# col_name transform_type +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":2,\"fields\":[{\"id\":1,\"name\":\"fname\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"},{\"id\":4,\"name\":\"address\",\"required\":false,\"type\":\"string\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"deleted-data-files\":\"6\",\"deleted-records\":\"6\",\"removed-files-size\":\"3167\",\"changed-partition-count\":\"2\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"10\",\"total-delete-files\":\"8\",\"total-position-deletes\":\"8\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 10 + numRows 10 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 17 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.ice_orc + numberOfBuckets: 0 + table name: default.ice_orc + blocking: true + +PREHOOK: query: alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn2 ln2a 1 NULL +fn4 ln4a 1 NULL +fn5 ln5a 2 addr_5 +fn6 ln6a 2 addr_6 +fn8 ln8a 2 addr_8 +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +fname string +last_name string +dept_id bigint +address string + +# Partition Transform Information +# col_name transform_type +dept_id IDENTITY + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"address\":\"true\",\"dept_id\":\"true\",\"fname\":\"true\",\"last_name\":\"true\"}} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":2,\"fields\":[{\"id\":1,\"name\":\"fname\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"},{\"id\":4,\"name\":\"address\",\"required\":false,\"type\":\"string\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"2\",\"added-records\":\"5\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"2\",\"total-records\":\"5\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"2\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + default-partition-spec {\"spec-id\":0,\"fields\":[{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1000}]} + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 2 + numRows 5 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 19 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: show compactions +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: show compactions +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out new file mode 100644 index 000000000000..a061d210dbe5 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out @@ -0,0 +1,310 @@ +PREHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_orc +POSTHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn1','ln1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn1','ln1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn2','ln2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn2','ln2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn3','ln3') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn3','ln3') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn4','ln4') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn4','ln4') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn5','ln5') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn5','ln5') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn6','ln6') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn6','ln6') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn7','ln7') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn7','ln7') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: delete from ice_orc where last_name in ('ln5a', 'ln6a', 'ln7a') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: delete from ice_orc where last_name in ('ln5a', 'ln6a', 'ln7a') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn1 ln1a +fn2 ln2a +fn3 ln3a +fn4 ln4a +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"1131\",\"changed-partition-count\":\"1\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 11 + numRows 11 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 15 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: explain alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.ice_orc + numberOfBuckets: 0 + table name: default.ice_orc + blocking: true + +PREHOOK: query: alter table ice_orc COMPACT 'major' and wait +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: alter table ice_orc COMPACT 'major' and wait +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn1 ln1a +fn2 ln2a +fn3 ln3a +fn4 ln4a +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"first_name\":\"true\",\"last_name\":\"true\"}} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 1 + numRows 4 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 17 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: show compactions +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: show compactions +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java index 52be546dca8d..a9c49efc54d3 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java @@ -110,6 +110,7 @@ public static void startServices() throws Exception { addToExpectedRestrictedMap("hive.zookeeper.ssl.truststore.password"); addToExpectedRestrictedMap("hive.zookeeper.ssl.truststore.type"); addToExpectedRestrictedMap("hive.iceberg.allow.datafiles.in.table.location.only"); + addToExpectedRestrictedMap("hive.rewrite.data.policy"); checkRestrictedListMatch(); } diff --git a/itests/qtest-iceberg/src/test/java/org/apache/hadoop/hive/cli/TestIcebergLlapLocalCompactorCliDriver.java b/itests/qtest-iceberg/src/test/java/org/apache/hadoop/hive/cli/TestIcebergLlapLocalCompactorCliDriver.java new file mode 100644 index 000000000000..795cc3cc09fb --- /dev/null +++ b/itests/qtest-iceberg/src/test/java/org/apache/hadoop/hive/cli/TestIcebergLlapLocalCompactorCliDriver.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.cli; + +import java.io.File; +import java.util.List; + +import org.apache.hadoop.hive.cli.control.CliAdapter; +import org.apache.hadoop.hive.cli.control.CliConfigs; + +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.txn.compactor.Worker; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import java.util.concurrent.atomic.AtomicBoolean; + +@RunWith(Parameterized.class) +public class TestIcebergLlapLocalCompactorCliDriver { + + static CliAdapter adapter = new CliConfigs.IcebergLlapLocalCompactorCliConfig().getCliAdapter(); + private static final AtomicBoolean stop = new AtomicBoolean(); + private static Worker worker; + + @Parameters(name ="{0}") + public static List getParameters() throws Exception { + return adapter.getParameters(); + } + + @BeforeClass + public static void setup() throws Exception { + worker = new Worker(); + worker.setConf(SessionState.get().getConf()); + stop.set(false); + worker.init(stop); + worker.start(); + } + + @AfterClass + public static void tearDown(){ + stop.set(true); + } + + @ClassRule + public static TestRule cliClassRule = adapter.buildClassRule(); + + @Rule + public TestRule cliTestRule = adapter.buildTestRule(); + + private String name; + private File qfile; + + public TestIcebergLlapLocalCompactorCliDriver(String name, File qfile) { + this.name = name; + this.qfile = qfile; + } + + @Test + public void testCliDriver() throws Exception { + adapter.runTest(name, qfile); + } +} diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 7a76db86d7c5..65b705d65181 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -417,6 +417,12 @@ iceberg.llap.query.files=\ vectorized_iceberg_read_orc.q,\ vectorized_iceberg_read_parquet.q +iceberg.llap.query.compactor.files=\ + iceberg_major_compaction_partition_evolution.q,\ + iceberg_major_compaction_partitioned.q,\ + iceberg_major_compaction_schema_evolution.q,\ + iceberg_major_compaction_unpartitioned.q + iceberg.llap.only.query.files=\ llap_iceberg_read_orc.q,\ llap_iceberg_read_parquet.q diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java index 7288eaeb9691..5705e5f38b77 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java @@ -639,6 +639,7 @@ public IcebergCliConfig() { try { setQueryDir("iceberg/iceberg-handler/src/test/queries/positive"); excludesFrom(testConfigProps, "iceberg.llap.only.query.files"); + excludesFrom(testConfigProps, "iceberg.llap.query.compactor.files"); setResultsDir("iceberg/iceberg-handler/src/test/results/positive"); setLogDir("itests/qtest/target/qfile-results/iceberg-handler/positive"); @@ -691,4 +692,27 @@ public IcebergLlapLocalCliConfig() { } } } + + public static class IcebergLlapLocalCompactorCliConfig extends AbstractCliConfig { + + public IcebergLlapLocalCompactorCliConfig() { + super(CoreCliDriver.class); + try { + setQueryDir("iceberg/iceberg-handler/src/test/queries/positive"); + + includesFrom(testConfigProps, "iceberg.llap.query.compactor.files"); + + setResultsDir("iceberg/iceberg-handler/src/test/results/positive/llap"); + setLogDir("itests/qtest/target/qfile-results/iceberg-handler/positive"); + + setInitScript("q_test_init_tez.sql"); + setCleanupScript("q_test_cleanup_tez.sql"); + + setHiveConfDir("data/conf/iceberg/llap"); + setClusterType(MiniClusterType.LLAP_LOCAL); + } catch (Exception e) { + throw new RuntimeException("can't contruct cliconfig", e); + } + } + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java index 17e4d4ae724b..ff7ed2d272ac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -28,12 +28,14 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.base.Preconditions; import org.antlr.runtime.TokenRewriteStream; import org.apache.commons.compress.utils.Lists; import org.apache.commons.lang3.tuple.Pair; @@ -250,6 +252,21 @@ public String toString() { return prefix; } } + public enum RewritePolicy { + + DEFAULT, + ALL_PARTITIONS; + + public static RewritePolicy fromString(String rewritePolicy) { + Preconditions.checkArgument(null != rewritePolicy, "Invalid rewrite policy: null"); + + try { + return valueOf(rewritePolicy.toUpperCase(Locale.ENGLISH)); + } catch (IllegalArgumentException var2) { + throw new IllegalArgumentException(String.format("Invalid rewrite policy: %s", rewritePolicy), var2); + } + } + } private String getMatchedText(ASTNode n) { return getTokenRewriteStream().toString(n.getTokenStartIndex(), n.getTokenStopIndex() + 1).trim(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java index cc896331aff7..8e86056053c8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java @@ -52,7 +52,7 @@ public AlterTableCompactOperation(DDLOperationContext context, AlterTableCompact @Override public int execute() throws Exception { Table table = context.getDb().getTable(desc.getTableName()); - if (!AcidUtils.isTransactionalTable(table)) { + if (!AcidUtils.isTransactionalTable(table) && !AcidUtils.isNonNativeAcidTable(table)) { throw new HiveException(ErrorMsg.NONACID_COMPACTION_NOT_SUPPORTED, table.getDbName(), table.getTableName()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java index ce822effe7b0..3493ea4d8da6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java @@ -89,7 +89,7 @@ public void run() { for (TaskHandler cleanupHandler : cleanupHandlers) { try { - checkInterrupt(); + CompactorUtil.checkInterrupt(CLASS_NAME); List tasks = cleanupHandler.getTasks(); List> asyncTasks = new ArrayList<>(); for (Runnable task : tasks) { @@ -112,7 +112,7 @@ public void run() { throw t; } } - checkInterrupt(); + CompactorUtil.checkInterrupt(CLASS_NAME); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java index 788955e35c20..6ca74e4bf2fb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactionHeartbeatService.java @@ -40,7 +40,7 @@ /** * Singleton service responsible for heartbeating the compaction transactions. */ -class CompactionHeartbeatService { +public class CompactionHeartbeatService { private static final Logger LOG = LoggerFactory.getLogger(CompactionHeartbeatService.class); @@ -53,7 +53,7 @@ class CompactionHeartbeatService { * @return Returns the singleton {@link CompactionHeartbeatService} * @throws IllegalStateException Thrown when the service has already been destroyed. */ - static CompactionHeartbeatService getInstance(HiveConf conf) { + public static CompactionHeartbeatService getInstance(HiveConf conf) { if (instance == null) { synchronized (CompactionHeartbeatService.class) { if (instance == null) { @@ -82,7 +82,7 @@ static CompactionHeartbeatService getInstance(HiveConf conf) { * @param tableName Required for logging only * @throws IllegalStateException Thrown when the heartbeat for the given txn has already been started. */ - void startHeartbeat(long txnId, long lockId, String tableName) { + public void startHeartbeat(long txnId, long lockId, String tableName) { if (shuttingDown) { throw new IllegalStateException("Service is shutting down, starting new heartbeats is not possible!"); } @@ -101,7 +101,7 @@ void startHeartbeat(long txnId, long lockId, String tableName) { * @throws IllegalStateException Thrown when there is no {@link CompactionHeartbeater} task associated with the * given txnId. */ - void stopHeartbeat(long txnId) throws InterruptedException { + public void stopHeartbeat(long txnId) throws InterruptedException { LOG.info("Stopping heartbeat task for TXN {}", txnId); CompactionHeartbeater heartbeater = tasks.get(txnId); if (heartbeater == null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java index 0d36402e56e7..dece9ed939c1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java @@ -38,7 +38,7 @@ public class CompactorContext { private final CompactionInfo compactionInfo; private final AcidDirectory dir; - CompactorContext(HiveConf conf, Table table, Partition p, StorageDescriptor sd, ValidWriteIdList tblValidWriteIds, CompactionInfo ci, AcidDirectory dir) { + public CompactorContext(HiveConf conf, Table table, Partition p, StorageDescriptor sd, ValidWriteIdList tblValidWriteIds, CompactionInfo ci, AcidDirectory dir) { this.conf = conf; this.table = table; this.partition = p; @@ -48,6 +48,10 @@ public class CompactorContext { this.dir = dir; } + public CompactorContext(HiveConf conf, Table table, CompactionInfo ci) { + this(conf, table, null, null, null, ci, null); + } + public HiveConf getConf() { return conf; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java index 786391c48e7b..2184bcd5072c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java @@ -22,6 +22,8 @@ import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.slf4j.Logger; @@ -32,6 +34,7 @@ */ public final class CompactorFactory { private static final Logger LOG = LoggerFactory.getLogger(CompactorFactory.class.getName()); + private static final String ICEBERG_MAJOR_QUERY_COMPACTOR_CLASS = "org.apache.iceberg.mr.hive.compaction.IcebergMajorQueryCompactor"; private static final CompactorFactory INSTANCE = new CompactorFactory(); @@ -108,6 +111,24 @@ public CompactorPipeline getCompactorPipeline(Table table, HiveConf configuratio throw new HiveException( compactionInfo.type.name() + " compaction is not supported on insert only tables."); } + } else if (MetaStoreUtils.isIcebergTable(table.getParameters())) { + switch (compactionInfo.type) { + case MAJOR: + + try { + Class icebergMajorQueryCompactor = (Class) + Class.forName(ICEBERG_MAJOR_QUERY_COMPACTOR_CLASS, true, + Utilities.getSessionSpecifiedClassLoader()); + + return new CompactorPipeline(icebergMajorQueryCompactor.newInstance()); + } + catch (Exception e) { + throw new HiveException("Failed instantiating and calling Iceberg compactor"); + } + default: + throw new HiveException( + compactionInfo.type.name() + " compaction is not supported on Iceberg tables."); + } } throw new HiveException("Only transactional tables can be compacted, " + table.getTableName() + "is not suitable " + "for compaction!"); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java index f26e832e0886..dda8b219f908 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java @@ -23,28 +23,18 @@ import org.apache.hadoop.hive.common.ServerUtils; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.LockComponentBuilder; -import org.apache.hadoop.hive.metastore.LockRequestBuilder; import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.DataOperationType; -import org.apache.hadoop.hive.metastore.api.LockRequest; -import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; -import org.apache.hadoop.hive.ql.io.AcidDirectory; -import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; -import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -90,12 +80,6 @@ public void init(AtomicBoolean stop) throws Exception { this.runtimeVersion = getRuntimeVersion(); } - protected void checkInterrupt() throws InterruptedException { - if (Thread.interrupted()) { - throw new InterruptedException(getClass().getName() + " execution is interrupted."); - } - } - /** * Find the table being compacted * @param ci compaction info returned from the compaction queue @@ -121,67 +105,7 @@ protected void checkInterrupt() throws InterruptedException { * @throws MetaException if underlying calls throw, or if the partition name resolves to more than * one partition. */ - protected Partition resolvePartition(CompactionInfo ci) throws MetaException { - if (ci.partName != null) { - List parts; - try { - parts = getPartitionsByNames(ci); - if (parts == null || parts.size() == 0) { - // The partition got dropped before we went looking for it. - return null; - } - } catch (Exception e) { - LOG.error("Unable to find partition " + ci.getFullPartitionName(), e); - throw e; - } - if (parts.size() != 1) { - LOG.error(ci.getFullPartitionName() + " does not refer to a single partition. " + - Arrays.toString(parts.toArray())); - throw new MetaException("Too many partitions for : " + ci.getFullPartitionName()); - } - return parts.get(0); - } else { - return null; - } - } - - /** - * Check for that special case when minor compaction is supported or not. - *
    - *
  • The table is Insert-only OR
  • - *
  • Query based compaction is not enabled OR
  • - *
  • The table has only acid data in it.
  • - *
- * @param tblproperties The properties of the table to check - * @param dir The {@link AcidDirectory} instance pointing to the table's folder on the filesystem. - * @return Returns true if minor compaction is supported based on the given parameters, false otherwise. - */ - protected boolean isMinorCompactionSupported(Map tblproperties, AcidDirectory dir) { - //Query based Minor compaction is not possible for full acid tables having raw format (non-acid) data in them. - return AcidUtils.isInsertOnlyTable(tblproperties) || !conf.getBoolVar(HiveConf.ConfVars.COMPACTOR_CRUD_QUERY_BASED) - || !(dir.getOriginalFiles().size() > 0 || dir.getCurrentDirectories().stream().anyMatch(AcidUtils.ParsedDelta::isRawFormat)); - } - - /** - * Get the storage descriptor for a compaction. - * @param t table from {@link #resolveTable(org.apache.hadoop.hive.metastore.txn.CompactionInfo)} - * @param p table from {@link #resolvePartition(org.apache.hadoop.hive.metastore.txn.CompactionInfo)} - * @return metastore storage descriptor. - */ - protected StorageDescriptor resolveStorageDescriptor(Table t, Partition p) { - return (p == null) ? t.getSd() : p.getSd(); - } - - /** - * Determine whether to run this job as the current user or whether we need a doAs to switch - * users. - * @param owner of the directory we will be working in, as determined by - * {@link org.apache.hadoop.hive.metastore.txn.TxnUtils#findUserToRunAs(String, Table, Configuration)} - * @return true if the job should run as the current user, false if a doAs is needed. - */ - protected boolean runJobAsSelf(String owner) { - return (owner.equals(System.getProperty("user.name"))); - } + abstract protected Partition resolvePartition(CompactionInfo ci) throws MetaException; protected String tableName(Table t) { return Warehouse.getQualifiedName(t); @@ -209,29 +133,6 @@ protected String getRuntimeVersion() { return this.getClass().getPackage().getImplementationVersion(); } - protected LockRequest createLockRequest(CompactionInfo ci, long txnId, LockType lockType, DataOperationType opType) { - String agentInfo = Thread.currentThread().getName(); - LockRequestBuilder requestBuilder = new LockRequestBuilder(agentInfo); - requestBuilder.setUser(ci.runAs); - requestBuilder.setTransactionId(txnId); - - LockComponentBuilder lockCompBuilder = new LockComponentBuilder() - .setLock(lockType) - .setOperationType(opType) - .setDbName(ci.dbname) - .setTableName(ci.tableName) - .setIsTransactional(true); - - if (ci.partName != null) { - lockCompBuilder.setPartitionName(ci.partName); - } - requestBuilder.addLockComponent(lockCompBuilder.build()); - - requestBuilder.setZeroWaitReadEnabled(!conf.getBoolVar(HiveConf.ConfVars.TXN_OVERWRITE_X_LOCK) || - !conf.getBoolVar(HiveConf.ConfVars.TXN_WRITE_X_LOCK)); - return requestBuilder.build(); - } - protected void doPostLoopActions(long elapsedTime) throws InterruptedException { String threadTypeName = getClass().getName(); if (elapsedTime < checkInterval && !stop.get()) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java index a8b45b0baec2..6daf665dd369 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java @@ -20,7 +20,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockComponentBuilder; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.DataOperationType; import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; @@ -30,6 +36,7 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.metastore.utils.StringableMap; import org.apache.hadoop.hive.ql.io.AcidDirectory; +import org.apache.hadoop.hive.ql.io.AcidUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +44,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.ForkJoinPool; @@ -121,6 +129,10 @@ public static StorageDescriptor resolveStorageDescriptor(Table t, Partition p) { return (p == null) ? t.getSd() : p.getSd(); } + public static StorageDescriptor resolveStorageDescriptor(Table t) { + return resolveStorageDescriptor(t, null); + } + public static boolean isDynPartAbort(Table t, String partName) { return Optional.ofNullable(t).map(Table::getPartitionKeys).filter(pk -> !pk.isEmpty()).isPresent() && partName == null; @@ -188,4 +200,99 @@ public static List getObsoleteDirs(AcidDirectory dir, boolean isDynPartAbo } return obsoleteDirs; } + + public static Partition resolvePartition(HiveConf conf, IMetaStoreClient msc, String dbName, String tableName, + String partName, METADATA_FETCH_MODE fetchMode) throws MetaException { + if (partName != null) { + List parts = null; + try { + + switch (fetchMode) { + case LOCAL: parts = CompactorUtil.getPartitionsByNames(conf, dbName, tableName, partName); + break; + case REMOTE: parts = RemoteCompactorUtil.getPartitionsByNames(msc, dbName, tableName, partName); + break; + } + + if (parts == null || parts.size() == 0) { + // The partition got dropped before we went looking for it. + return null; + } + } catch (Exception e) { + LOG.error("Unable to find partition " + getFullPartitionName(dbName, tableName, partName), e); + throw e; + } + if (parts.size() != 1) { + LOG.error(getFullPartitionName(dbName, tableName, partName) + " does not refer to a single partition. " + + Arrays.toString(parts.toArray())); + throw new MetaException("Too many partitions for : " + getFullPartitionName(dbName, tableName, partName)); + } + return parts.get(0); + } else { + return null; + } + } + + public static String getFullPartitionName(String dbName, String tableName, String partName) { + StringBuilder buf = new StringBuilder(); + buf.append(dbName); + buf.append('.'); + buf.append(tableName); + if (partName != null) { + buf.append('.'); + buf.append(partName); + } + return buf.toString(); + } + + public enum METADATA_FETCH_MODE { + LOCAL, + REMOTE + } + + public static void checkInterrupt(String callerClassName) throws InterruptedException { + if (Thread.interrupted()) { + throw new InterruptedException(callerClassName + " execution is interrupted."); + } + } + + /** + * Check for that special case when minor compaction is supported or not. + *
    + *
  • The table is Insert-only OR
  • + *
  • Query based compaction is not enabled OR
  • + *
  • The table has only acid data in it.
  • + *
+ * @param tblproperties The properties of the table to check + * @param dir The {@link AcidDirectory} instance pointing to the table's folder on the filesystem. + * @return Returns true if minor compaction is supported based on the given parameters, false otherwise. + */ + public static boolean isMinorCompactionSupported(HiveConf conf, Map tblproperties, AcidDirectory dir) { + //Query based Minor compaction is not possible for full acid tables having raw format (non-acid) data in them. + return AcidUtils.isInsertOnlyTable(tblproperties) || !conf.getBoolVar(HiveConf.ConfVars.COMPACTOR_CRUD_QUERY_BASED) + || !(dir.getOriginalFiles().size() > 0 || dir.getCurrentDirectories().stream().anyMatch(AcidUtils.ParsedDelta::isRawFormat)); + } + + public static LockRequest createLockRequest(HiveConf conf, CompactionInfo ci, long txnId, LockType lockType, DataOperationType opType) { + String agentInfo = Thread.currentThread().getName(); + LockRequestBuilder requestBuilder = new LockRequestBuilder(agentInfo); + requestBuilder.setUser(ci.runAs); + requestBuilder.setTransactionId(txnId); + + LockComponentBuilder lockCompBuilder = new LockComponentBuilder() + .setLock(lockType) + .setOperationType(opType) + .setDbName(ci.dbname) + .setTableName(ci.tableName) + .setIsTransactional(true); + + if (ci.partName != null) { + lockCompBuilder.setPartitionName(ci.partName); + } + requestBuilder.addLockComponent(lockCompBuilder.build()); + + requestBuilder.setZeroWaitReadEnabled(!conf.getBoolVar(HiveConf.ConfVars.TXN_OVERWRITE_X_LOCK) || + !conf.getBoolVar(HiveConf.ConfVars.TXN_WRITE_X_LOCK)); + return requestBuilder.build(); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java index bb48c8f219bf..6396aa8c30d8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java @@ -107,7 +107,7 @@ public void run() { final ShowCompactResponse currentCompactions = txnHandler.showCompact(new ShowCompactRequest()); - checkInterrupt(); + CompactorUtil.checkInterrupt(CLASS_NAME); // Currently we invalidate all entries after each cycle, because the bootstrap replication is marked via // table property hive.repl.first.inc.pending which would be cached. @@ -122,7 +122,7 @@ public void run() { .collect(Collectors.toSet())).get(); LOG.debug("Found {} potential compactions, checking to see if we should compact any of them", potentials.size()); - checkInterrupt(); + CompactorUtil.checkInterrupt(CLASS_NAME); Map tblNameOwners = new HashMap<>(); List> compactionList = new ArrayList<>(); @@ -234,7 +234,7 @@ protected String resolveUserToRunAs(Map cache, Table t, Partitio throws IOException, InterruptedException { //Figure out who we should run the file operations as String fullTableName = TxnUtils.getFullTableName(t.getDbName(), t.getTableName()); - StorageDescriptor sd = resolveStorageDescriptor(t, p); + StorageDescriptor sd = CompactorUtil.resolveStorageDescriptor(t, p); String user = cache.get(fullTableName); if (user == null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java index 14dd2ebffe02..369dc69fc309 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.api.CompactionRequest; import org.apache.hadoop.hive.metastore.api.CompactionResponse; import org.apache.hadoop.hive.metastore.api.CompactionType; @@ -43,7 +42,6 @@ import org.apache.hadoop.hive.metastore.txn.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnUtils; -import org.apache.hadoop.hive.metastore.utils.JavaUtils; import org.apache.hadoop.hive.ql.io.AcidDirectory; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.shims.HadoopShims; @@ -68,7 +66,7 @@ private List initiateCompactionForMultiplePartitions(Table t List compactionResponses = new ArrayList<>(); partitions.entrySet().parallelStream().forEach(entry -> { try { - StorageDescriptor sd = resolveStorageDescriptor(table, entry.getValue()); + StorageDescriptor sd = CompactorUtil.resolveStorageDescriptor(table, entry.getValue()); String runAs = TxnUtils.findUserToRunAs(sd.getLocation(), table, conf); CompactionInfo ci = new CompactionInfo(table.getDbName(), table.getTableName(), entry.getKey(), request.getType()); @@ -195,7 +193,7 @@ private CompactionType determineCompactionType(CompactionInfo ci, AcidDirectory LOG.debug("Found " + deltas.size() + " delta files, and " + (noBase ? "no" : "has") + " base," + "requesting " + (noBase ? "major" : "minor") + " compaction"); - return noBase || !isMinorCompactionSupported(tblProperties, dir) ? CompactionType.MAJOR : CompactionType.MINOR; + return noBase || !CompactorUtil.isMinorCompactionSupported(conf, tblProperties, dir) ? CompactionType.MAJOR : CompactionType.MINOR; } private long getBaseSize(AcidDirectory dir) throws IOException { @@ -243,7 +241,7 @@ private CompactionType checkForCompaction(final CompactionInfo ci, final ValidWr AcidMetricService.updateMetricsFromInitiator(ci.dbname, ci.tableName, ci.partName, conf, txnHandler, baseSize, deltaSizes, acidDirectory.getObsolete()); - if (runJobAsSelf(runAs)) { + if (CompactorUtil.runJobAsSelf(runAs)) { return determineCompactionType(ci, acidDirectory, tblProperties, baseSize, deltaSize); } else { LOG.info("Going to initiate as user " + runAs + " for " + ci.getFullPartitionName()); @@ -279,11 +277,11 @@ private ValidWriteIdList resolveValidWriteIds(Table t) protected CompactionResponse scheduleCompactionIfRequired(CompactionInfo ci, Table t, Partition p, String runAs, boolean metricsEnabled) throws MetaException { - StorageDescriptor sd = resolveStorageDescriptor(t, p); + StorageDescriptor sd = CompactorUtil.resolveStorageDescriptor(t, p); try { ValidWriteIdList validWriteIds = resolveValidWriteIds(t); - checkInterrupt(); + CompactorUtil.checkInterrupt(InitiatorBase.class.getName()); CompactionType type = checkForCompaction(ci, validWriteIds, sd, t.getParameters(), runAs); if (type != null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java index 39d0e10f589a..bfd59c6fdc71 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java @@ -88,6 +88,11 @@ public void init(AtomicBoolean stop) throws Exception { return CompactorUtil.getPartitionsByNames(conf, ci.dbname, ci.tableName, ci.partName); } + protected Partition resolvePartition(CompactionInfo ci) throws MetaException { + return CompactorUtil.resolvePartition(conf, null, ci.dbname, ci.tableName, ci.partName, + CompactorUtil.METADATA_FETCH_MODE.LOCAL); + } + protected abstract boolean isCacheEnabled(); protected void startCycleUpdater(long updateInterval, Runnable taskToRun) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java index 01b24404beb7..ee0d9c9a7d74 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java @@ -49,7 +49,7 @@ /** * Common interface for query based compactions. */ -abstract class QueryCompactor implements Compactor { +public abstract class QueryCompactor implements Compactor { private static final Logger LOG = LoggerFactory.getLogger(QueryCompactor.class.getName()); private static final String COMPACTOR_PREFIX = "compactor."; @@ -68,6 +68,22 @@ abstract class QueryCompactor implements Compactor { protected void commitCompaction(String dest, String tmpTableName, HiveConf conf, ValidWriteIdList actualWriteIds, long compactorTxnId) throws IOException, HiveException {} + protected SessionState setupQueryCompactionSession(HiveConf conf, CompactionInfo compactionInfo, Map tblProperties) { + String queueName = HiveConf.getVar(conf, HiveConf.ConfVars.COMPACTOR_JOB_QUEUE); + if (queueName != null && queueName.length() > 0) { + conf.set(TezConfiguration.TEZ_QUEUE_NAME, queueName); + } + Util.disableLlapCaching(conf); + conf.set(HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT.varname, "column"); + conf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS, true); + conf.setBoolVar(HiveConf.ConfVars.HIVE_HDFS_ENCRYPTION_SHIM_CACHE_ON, false); + Util.overrideConfProps(conf, compactionInfo, tblProperties); + String user = compactionInfo.runAs; + SessionState sessionState = DriverUtils.setUpSessionState(conf, user, true); + sessionState.setCompaction(true); + return sessionState; + } + /** * Run all the queries which performs the compaction. * @param conf hive configuration, must be not null. @@ -86,18 +102,7 @@ void runCompactionQueries(HiveConf conf, String tmpTableName, StorageDescriptor ValidWriteIdList writeIds, CompactionInfo compactionInfo, List resultDirs, List createQueries, List compactionQueries, List dropQueries, Map tblProperties) throws IOException { - String queueName = HiveConf.getVar(conf, HiveConf.ConfVars.COMPACTOR_JOB_QUEUE); - if (queueName != null && queueName.length() > 0) { - conf.set(TezConfiguration.TEZ_QUEUE_NAME, queueName); - } - Util.disableLlapCaching(conf); - conf.set(HiveConf.ConfVars.HIVE_QUOTEDID_SUPPORT.varname, "column"); - conf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_HDFS_ENCRYPTION_SHIM_CACHE_ON, false); - Util.overrideConfProps(conf, compactionInfo, tblProperties); - String user = compactionInfo.runAs; - SessionState sessionState = DriverUtils.setUpSessionState(conf, user, true); - sessionState.setCompaction(true); + SessionState sessionState = setupQueryCompactionSession(conf, compactionInfo, tblProperties); long compactorTxnId = Compactor.getCompactorTxnId(conf); try { for (String query : createQueries) { @@ -170,7 +175,7 @@ private void removeResultDirs(List resultDirPaths, HiveConf conf) throws I /** * Collection of some helper functions. */ - static class Util { + public static class Util { /** * Get the path of the base, delta, or delete delta directory that will be the final @@ -186,8 +191,8 @@ static class Util { * * @return Path of new base/delta/delete delta directory */ - static Path getCompactionResultDir(StorageDescriptor sd, ValidWriteIdList writeIds, HiveConf conf, - boolean writingBase, boolean createDeleteDelta, boolean bucket0, AcidDirectory directory) { + public static Path getCompactionResultDir(StorageDescriptor sd, ValidWriteIdList writeIds, HiveConf conf, + boolean writingBase, boolean createDeleteDelta, boolean bucket0, AcidDirectory directory) { long minWriteID = writingBase ? 1 : getMinWriteID(directory); long highWatermark = writeIds.getHighWatermark(); long compactorTxnId = Compactor.getCompactorTxnId(conf); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java index 9055a1a67163..819e39579c51 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java @@ -20,7 +20,6 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; @@ -29,11 +28,9 @@ import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; import org.apache.thrift.TException; -import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.convertToGetPartitionsByNamesRequest; import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; /** @@ -61,12 +58,7 @@ public void init(AtomicBoolean stop) throws Exception { } @Override Table resolveTable(CompactionInfo ci) throws MetaException { - try { - return msc.getTable(getDefaultCatalog(conf), ci.dbname, ci.tableName); - } catch (TException e) { - LOG.error("Unable to find table " + ci.getFullTableName(), e); - throw new MetaException(e.toString()); - } + return RemoteCompactorUtil.resolveTable(conf, msc, ci); } @Override boolean replIsCompactionDisabledForDatabase(String dbName) throws TException { @@ -81,13 +73,11 @@ public void init(AtomicBoolean stop) throws Exception { } @Override List getPartitionsByNames(CompactionInfo ci) throws MetaException { - try { - GetPartitionsByNamesRequest req = convertToGetPartitionsByNamesRequest(ci.dbname, ci.tableName, - Collections.singletonList(ci.partName)); - return msc.getPartitionsByNames(req).getPartitions(); - } catch (TException e) { - LOG.error("Unable to get partitions by name for CompactionInfo=" + ci); - throw new MetaException(e.toString()); - } + return RemoteCompactorUtil.getPartitionsByNames(msc, ci.dbname, ci.tableName, ci.tableName); + } + + protected Partition resolvePartition(CompactionInfo ci) throws MetaException { + return CompactorUtil.resolvePartition(conf, msc, ci.dbname, ci.tableName, ci.partName, + CompactorUtil.METADATA_FETCH_MODE.REMOTE); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java new file mode 100644 index 000000000000..676a11891a0d --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.txn.compactor; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.List; + +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.convertToGetPartitionsByNamesRequest; +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; + +public class RemoteCompactorUtil { + + static final private String CLASS_NAME = RemoteCompactorUtil.class.getName(); + protected static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); + + public static Table resolveTable(HiveConf conf, IMetaStoreClient msc, CompactionInfo ci) throws MetaException { + try { + return msc.getTable(getDefaultCatalog(conf), ci.dbname, ci.tableName); + } catch (TException e) { + LOG.error("Unable to find table " + ci.getFullTableName(), e); + throw new MetaException(e.toString()); + } + } + + public static List getPartitionsByNames(IMetaStoreClient msc, String dbName, String tableName, + String partName) throws MetaException { + try { + GetPartitionsByNamesRequest req = convertToGetPartitionsByNamesRequest(dbName, tableName, + Collections.singletonList(partName)); + return msc.getPartitionsByNames(req).getPartitions(); + } catch (TException e) { + LOG.error("Unable to get partitions by name = {}.{}.{}", dbName, tableName, partName); + throw new MetaException(e.toString()); + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java index 459b7d2bbd57..8f75c06d54b7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java @@ -21,48 +21,27 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; -import org.apache.hadoop.hive.common.ValidTxnList; -import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils; import org.apache.hadoop.hive.metastore.MetaStoreThread; -import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; -import org.apache.hadoop.hive.metastore.api.CompactionType; -import org.apache.hadoop.hive.metastore.api.DataOperationType; import org.apache.hadoop.hive.metastore.api.FindNextCompactRequest; -import org.apache.hadoop.hive.metastore.api.LockRequest; -import org.apache.hadoop.hive.metastore.api.LockType; -import org.apache.hadoop.hive.metastore.api.LockResponse; -import org.apache.hadoop.hive.metastore.api.LockState; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; -import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg; -import org.apache.hadoop.hive.metastore.txn.TxnStatus; -import org.apache.hadoop.hive.ql.io.AcidDirectory; -import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hive.common.util.Ref; +import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionService; +import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionExecutorFactory; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.txn.CompactionInfo; -import org.apache.hadoop.hive.metastore.txn.TxnUtils; -import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; import java.security.PrivilegedExceptionAction; -import java.util.Collections; -import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -70,7 +49,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; /** * A class to do compactions. This will run in a separate thread. It will spin on the @@ -170,66 +148,6 @@ public void init(AtomicBoolean stop) throws Exception { setName(workerName); } - /** - * Determine if compaction can run in a specified directory. - * @param ci {@link CompactionInfo} - * @param dir the delta directory - * @param sd resolved storage descriptor - * @return true, if compaction can run. - */ - static boolean isEnoughToCompact(CompactionInfo ci, AcidDirectory dir, StorageDescriptor sd) { - int deltaCount = dir.getCurrentDirectories().size(); - int origCount = dir.getOriginalFiles().size(); - - StringBuilder deltaInfo = new StringBuilder().append(deltaCount); - boolean isEnoughToCompact; - - if (ci.isRebalanceCompaction()) { - //TODO: For now, we are allowing rebalance compaction regardless of the table state. Thresholds will be added later. - return true; - } else if (ci.isMajorCompaction()) { - isEnoughToCompact = - (origCount > 0 || deltaCount + (dir.getBaseDirectory() == null ? 0 : 1) > 1); - - } else { - isEnoughToCompact = (deltaCount > 1); - - if (deltaCount == 2) { - Map deltaByType = dir.getCurrentDirectories().stream().collect(Collectors - .groupingBy(delta -> (delta - .isDeleteDelta() ? AcidUtils.DELETE_DELTA_PREFIX : AcidUtils.DELTA_PREFIX), - Collectors.counting())); - - isEnoughToCompact = (deltaByType.size() != deltaCount); - deltaInfo.append(" ").append(deltaByType); - } - } - - if (!isEnoughToCompact) { - LOG.info("Not enough files in {} to compact; current base: {}, delta files: {}, originals: {}", - sd.getLocation(), dir.getBaseDirectory(), deltaInfo, origCount); - } - return isEnoughToCompact; - } - - /** - * Check for obsolete directories, and return true if any exist and Cleaner should be - * run. For example if we insert overwrite into a table with only deltas, a new base file with - * the highest writeId is created so there will be no live delta directories, only obsolete - * ones. Compaction is not needed, but the cleaner should still be run. - * - * @return true if cleaning is needed - */ - public static boolean needsCleaning(AcidDirectory dir, StorageDescriptor sd) { - int numObsoleteDirs = dir.getObsolete().size() + dir.getAbortedDirectories().size(); - boolean needsJustCleaning = numObsoleteDirs > 0; - if (needsJustCleaning) { - LOG.info("{} obsolete directories in {} found; marked for cleaning.", numObsoleteDirs, - sd.getLocation()); - } - return needsJustCleaning; - } - /** * Creates a single threaded executor used for handling timeouts. * The thread settings are inherited from the current thread. @@ -258,10 +176,10 @@ protected Boolean findNextCompactionAndExecute(boolean collectGenericStats, bool // so wrap it in a big catch Throwable statement. PerfLogger perfLogger = SessionState.getPerfLogger(false); String workerMetric = null; - CompactionInfo ci = null; - boolean computeStats = false; Table table = null; + CompactionService compactionService = null; + boolean compactionResult = false; // If an exception is thrown in the try-with-resources block below, msc is closed and nulled, so a new instance // is need to be obtained here. @@ -274,7 +192,7 @@ protected Boolean findNextCompactionAndExecute(boolean collectGenericStats, bool } } - try (CompactionTxn compactionTxn = new CompactionTxn()) { + try { FindNextCompactRequest findNextCompactRequest = new FindNextCompactRequest(); findNextCompactRequest.setWorkerId(workerName); @@ -297,7 +215,7 @@ protected Boolean findNextCompactionAndExecute(boolean collectGenericStats, bool LOG.warn("The returned compaction request ({}) belong to a different pool. Although the worker is assigned to the {} pool," + " it will process the request.", ci, getPoolName()); } - checkInterrupt(); + CompactorUtil.checkInterrupt(CLASS_NAME); if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) { workerMetric = MetricsConstants.COMPACTION_WORKER_CYCLE + "_" + @@ -321,158 +239,27 @@ protected Boolean findNextCompactionAndExecute(boolean collectGenericStats, bool return false; } - if (ci.isRebalanceCompaction() && table.getSd().getNumBuckets() > 0) { - LOG.error("Cannot execute rebalancing compaction on bucketed tables."); - ci.errorMessage = "Cannot execute rebalancing compaction on bucketed tables."; - msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); - return false; - } - - if (!ci.type.equals(CompactionType.REBALANCE) && ci.numberOfBuckets > 0) { - if (LOG.isWarnEnabled()) { - LOG.warn("Only the REBALANCE compaction accepts the number of buckets clause (CLUSTERED INTO {N} BUCKETS). " + - "Since the compaction request is {}, it will be ignored.", ci.type); - } - } - - checkInterrupt(); - - String fullTableName = TxnUtils.getFullTableName(table.getDbName(), table.getTableName()); - - - // Find the partition we will be working with, if there is one. - Partition p; - try { - p = resolvePartition(ci); - if (p == null && ci.partName != null) { - ci.errorMessage = "Unable to find partition " + ci.getFullPartitionName() + ", assuming it was dropped and moving on."; - LOG.warn(ci.errorMessage + " Compaction info: {}", ci); - msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); - return false; - } - } catch (Exception e) { - LOG.error("Unexpected error during resolving partition.", e); - ci.errorMessage = e.getMessage(); - msc.markFailed(CompactionInfo.compactionInfoToStruct(ci)); - return false; - } - - checkInterrupt(); - - // Find the appropriate storage descriptor - final StorageDescriptor sd = resolveStorageDescriptor(table, p); - - // Check that the table or partition isn't sorted, as we don't yet support that. - if (sd.getSortCols() != null && !sd.getSortCols().isEmpty()) { - ci.errorMessage = "Attempt to compact sorted table " + ci.getFullTableName() + ", which is not yet supported!"; - LOG.warn(ci.errorMessage + " Compaction info: {}", ci); - msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); - return false; - } - - if (ci.runAs == null) { - ci.runAs = TxnUtils.findUserToRunAs(sd.getLocation(), table, conf); - } - - checkInterrupt(); - - /** - * we cannot have Worker use HiveTxnManager (which is on ThreadLocal) since - * then the Driver would already have the an open txn but then this txn would have - * multiple statements in it (for query based compactor) which is not supported (and since - * this case some of the statements are DDL, even in the future will not be allowed in a - * multi-stmt txn. {@link Driver#setCompactionWriteIds(ValidWriteIdList, long)} */ - compactionTxn.open(ci); - - ValidTxnList validTxnList = msc.getValidTxns(compactionTxn.getTxnId()); - //with this ValidWriteIdList is capped at whatever HWM validTxnList has - final ValidCompactorWriteIdList tblValidWriteIds = - TxnUtils.createValidCompactWriteIdList(msc.getValidWriteIds( - Collections.singletonList(fullTableName), validTxnList.writeToString()).get(0)); - LOG.debug("ValidCompactWriteIdList: " + tblValidWriteIds.writeToString()); - conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); - - ci.highestWriteId = tblValidWriteIds.getHighWatermark(); - //this writes TXN_COMPONENTS to ensure that if compactorTxnId fails, we keep metadata about - //it until after any data written by it are physically removed - msc.updateCompactorState(CompactionInfo.compactionInfoToStruct(ci), compactionTxn.getTxnId()); - - checkInterrupt(); - - // Don't start compaction or cleaning if not necessary - if (isDynPartAbort(table, ci)) { - msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci)); - compactionTxn.wasSuccessful(); - return false; - } - AcidDirectory dir = getAcidStateForWorker(ci, sd, tblValidWriteIds); - if (!isEnoughToCompact(ci, dir, sd)) { - if (needsCleaning(dir, sd)) { - msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci)); - } else { - // do nothing - ci.errorMessage = "None of the compaction thresholds met, compaction request is refused!"; - LOG.debug(ci.errorMessage + " Compaction info: {}", ci); - msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); - } - compactionTxn.wasSuccessful(); - return false; - } - if (!ci.isMajorCompaction() && !isMinorCompactionSupported(table.getParameters(), dir)) { - ci.errorMessage = "Query based Minor compaction is not possible for full acid tables having raw format " + - "(non-acid) data in them."; - LOG.error(ci.errorMessage + " Compaction info: {}", ci); - try { - msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); - } catch (Throwable tr) { - LOG.error("Caught an exception while trying to mark compaction {} as failed: {}", ci, tr); - } - return false; - } - checkInterrupt(); + CompactorUtil.checkInterrupt(CLASS_NAME); + compactionService = CompactionExecutorFactory.createExecutor(conf, msc, compactorFactory, table, collectGenericStats, collectMrStats); try { - failCompactionIfSetForTest(); - - /* - First try to run compaction via HiveQL queries. - Compaction for MM tables happens here, or run compaction for Crud tables if query-based compaction is enabled. - todo Find a more generic approach to collecting files in the same logical bucket to compact within the same - task (currently we're using Tez split grouping). - */ - CompactorPipeline compactorPipeline = compactorFactory.getCompactorPipeline(table, conf, ci, msc); - computeStats = (compactorPipeline.isMRCompaction() && collectMrStats) || collectGenericStats; - - LOG.info("Starting " + ci.type.toString() + " compaction for " + ci.getFullPartitionName() + ", id:" + - ci.id + " in " + compactionTxn + " with compute stats set to " + computeStats); - - CompactorContext compactorContext = new CompactorContext(conf, table, p, sd, tblValidWriteIds, ci, dir); - compactorPipeline.execute(compactorContext); - - LOG.info("Completed " + ci.type.toString() + " compaction for " + ci.getFullPartitionName() + " in " - + compactionTxn + ", marking as compacted."); - msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci)); - compactionTxn.wasSuccessful(); - - AcidMetricService.updateMetricsFromWorker(ci.dbname, ci.tableName, ci.partName, ci.type, - dir.getCurrentDirectories().size(), dir.getDeleteDeltas().size(), conf, msc); + compactionResult = compactionService.compact(table, ci); } catch (Throwable e) { LOG.error("Caught exception while trying to compact " + ci + ". Marking failed to avoid repeated failures", e); - final CompactionType ctype = ci.type; markFailed(ci, e.getMessage()); - computeStats = false; - - if (runJobAsSelf(ci.runAs)) { - cleanupResultDirs(sd, tblValidWriteIds, ctype, dir); + if (CompactorUtil.runJobAsSelf(ci.runAs)) { + compactionService.cleanupResultDirs(ci); } else { LOG.info("Cleaning as user " + ci.runAs); UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs, UserGroupInformation.getLoginUser()); + CompactionService finalCompactionService = compactionService; + CompactionInfo finalCi = ci; ugi.doAs((PrivilegedExceptionAction) () -> { - cleanupResultDirs(sd, tblValidWriteIds, ctype, dir); + finalCompactionService.cleanupResultDirs(finalCi); return null; }); try { @@ -499,64 +286,12 @@ protected Boolean findNextCompactionAndExecute(boolean collectGenericStats, bool } } - if (computeStats) { - statsUpdater.gatherStats(ci, conf, runJobAsSelf(ci.runAs) ? ci.runAs : table.getOwner(), - CompactorUtil.getCompactorJobQueueName(conf, ci, table), msc); - } - return true; - } - - /** - * Just AcidUtils.getAcidState, but with impersonation if needed. - */ - private AcidDirectory getAcidStateForWorker(CompactionInfo ci, StorageDescriptor sd, - ValidCompactorWriteIdList tblValidWriteIds) throws IOException, InterruptedException { - if (runJobAsSelf(ci.runAs)) { - return AcidUtils.getAcidState(null, new Path(sd.getLocation()), conf, - tblValidWriteIds, Ref.from(false), true); + if (Optional.ofNullable(compactionService).map(CompactionService::isComputeStats).orElse(false)) { + statsUpdater.gatherStats(ci, conf, CompactorUtil.runJobAsSelf(ci.runAs) ? ci.runAs : table.getOwner(), + CompactorUtil.getCompactorJobQueueName(conf, ci, table), msc); } - UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs, UserGroupInformation.getLoginUser()); - try { - return ugi.doAs((PrivilegedExceptionAction) () -> - AcidUtils.getAcidState(null, new Path(sd.getLocation()), conf, tblValidWriteIds, - Ref.from(false), true)); - } finally { - try { - FileSystem.closeAllForUGI(ugi); - } catch (IOException exception) { - LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " + ci.getFullPartitionName(), - exception); - } - } - } - - private void cleanupResultDirs(StorageDescriptor sd, ValidWriteIdList writeIds, CompactionType ctype, AcidDirectory dir) { - // result directory for compactor to write new files - Path resultDir = QueryCompactor.Util.getCompactionResultDir(sd, writeIds, conf, - ctype == CompactionType.MAJOR, false, false, dir); - LOG.info("Deleting result directories created by the compactor:\n"); - try { - FileSystem fs = resultDir.getFileSystem(conf); - LOG.info(resultDir.toString()); - fs.delete(resultDir, true); - - if (ctype == CompactionType.MINOR) { - Path deleteDeltaDir = QueryCompactor.Util.getCompactionResultDir(sd, writeIds, conf, - false, true, false, dir); - - LOG.info(deleteDeltaDir.toString()); - fs.delete(deleteDeltaDir, true); - } - } catch (IOException ex) { - LOG.error("Caught exception while cleaning result directories:", ex); - } - } - - private void failCompactionIfSetForTest() { - if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION)) { - throw new RuntimeException(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION.name() + "=true"); - } + return compactionResult; } private void markFailed(CompactionInfo ci, String errorMessage) { @@ -578,107 +313,10 @@ private void markFailed(CompactionInfo ci, String errorMessage) { } } - private static boolean isDynPartAbort(Table t, CompactionInfo ci) { - return t.getPartitionKeys() != null && t.getPartitionKeys().size() > 0 - && ci.partName == null; - } - private String getWorkerId() { StringBuilder name = new StringBuilder(this.hostName); name.append("-"); name.append(getId()); return name.toString(); } - - /** - * Keep track of the compaction's transaction and its operations. - */ - class CompactionTxn implements AutoCloseable { - private long txnId = 0; - private long lockId = 0; - - private TxnStatus status = TxnStatus.UNKNOWN; - private boolean successfulCompaction = false; - - /** - * Try to open a new txn. - * @throws TException - */ - void open(CompactionInfo ci) throws TException { - this.txnId = msc.openTxn(ci.runAs, ci.type == CompactionType.REBALANCE ? TxnType.REBALANCE_COMPACTION : TxnType.COMPACTION); - status = TxnStatus.OPEN; - - LockRequest lockRequest; - if (CompactionType.REBALANCE.equals(ci.type)) { - lockRequest = createLockRequest(ci, txnId, LockType.EXCL_WRITE, DataOperationType.UPDATE); - } else { - lockRequest = createLockRequest(ci, txnId, LockType.SHARED_READ, DataOperationType.SELECT); - } - LockResponse res = msc.lock(lockRequest); - if (res.getState() != LockState.ACQUIRED) { - throw new TException("Unable to acquire lock(s) on {" + ci.getFullPartitionName() - + "}, status {" + res.getState() + "}, reason {" + res.getErrorMessage() + "}"); - } - lockId = res.getLockid(); - CompactionHeartbeatService.getInstance(conf).startHeartbeat(txnId, lockId, TxnUtils.getFullTableName(ci.dbname, ci.tableName)); - } - - /** - * Mark compaction as successful. This means the txn will be committed; otherwise it will be aborted. - */ - void wasSuccessful() { - this.successfulCompaction = true; - } - - /** - * Commit or abort txn. - * @throws Exception - */ - @Override public void close() throws Exception { - if (status == TxnStatus.UNKNOWN) { - return; - } - try { - //the transaction is about to close, we can stop heartbeating regardless of it's state - CompactionHeartbeatService.getInstance(conf).stopHeartbeat(txnId); - } finally { - if (successfulCompaction) { - commit(); - } else { - abort(); - } - } - } - - long getTxnId() { - return txnId; - } - - @Override public String toString() { - return "txnId=" + txnId + ", lockId=" + lockId + " (TxnStatus: " + status + ")"; - } - - /** - * Commit the txn if open. - */ - private void commit() throws TException { - if (status == TxnStatus.OPEN) { - msc.commitTxn(txnId); - status = TxnStatus.COMMITTED; - } - } - - /** - * Abort the txn if open. - */ - private void abort() throws TException { - if (status == TxnStatus.OPEN) { - AbortTxnRequest abortTxnRequest = new AbortTxnRequest(txnId); - abortTxnRequest.setErrorCode(TxnErrorMsg.ABORT_COMPACTION_TXN.getErrorCode()); - msc.rollbackTxn(abortTxnRequest); - status = TxnStatus.ABORTED; - } - } - } - } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java index 8a105c0bc206..d530b7efa34c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java @@ -45,8 +45,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Arrays; -import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; @@ -88,27 +86,7 @@ protected Table resolveTable(String dbName, String tableName) throws MetaExcepti } protected Partition resolvePartition(String dbName, String tableName, String partName) throws MetaException { - if (partName != null) { - List parts; - try { - parts = CompactorUtil.getPartitionsByNames(conf, dbName, tableName, partName); - if (parts == null || parts.isEmpty()) { - // The partition got dropped before we went looking for it. - return null; - } - } catch (Exception e) { - LOG.error("Unable to find partition: {}.{}.{}", dbName, tableName, partName, e); - throw e; - } - if (parts.size() != 1) { - LOG.error("{}.{}.{} does not refer to a single partition. {}", dbName, tableName, partName, - Arrays.toString(parts.toArray())); - throw new MetaException(String.join("Too many partitions for : ", dbName, tableName, partName)); - } - return parts.get(0); - } else { - return null; - } + return CompactorUtil.resolvePartition(conf, null, dbName, tableName, partName, CompactorUtil.METADATA_FETCH_MODE.LOCAL); } protected ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo info, ValidTxnList validTxnList) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java new file mode 100644 index 000000000000..4fb6775eda53 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java @@ -0,0 +1,420 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.txn.compactor.service; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.common.ValidWriteIdList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.CompactionType; +import org.apache.hadoop.hive.metastore.api.DataOperationType; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.TxnType; +import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg; +import org.apache.hadoop.hive.metastore.txn.TxnStatus; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.io.AcidDirectory; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.txn.compactor.CompactionHeartbeatService; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorFactory; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorPipeline; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; +import org.apache.hadoop.hive.ql.txn.compactor.QueryCompactor; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.common.util.Ref; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.Collections; +import java.util.Map; +import java.util.stream.Collectors; + +public class AcidCompactionService extends CompactionService { + static final private String CLASS_NAME = AcidCompactionService.class.getName(); + static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME); + + private final boolean collectMrStats; + private StorageDescriptor sd; + private ValidCompactorWriteIdList tblValidWriteIds; + private AcidDirectory dir; + + public AcidCompactionService(HiveConf conf, IMetaStoreClient msc, CompactorFactory compactorFactory, + boolean collectGenericStats, boolean collectMrStats) { + super(conf, msc, compactorFactory, collectGenericStats); + this.collectMrStats = collectMrStats; + } + + /** + * Just AcidUtils.getAcidState, but with impersonation if needed. + */ + private AcidDirectory getAcidStateForWorker(CompactionInfo ci, StorageDescriptor sd, + ValidCompactorWriteIdList tblValidWriteIds) throws IOException, InterruptedException { + if (CompactorUtil.runJobAsSelf(ci.runAs)) { + return AcidUtils.getAcidState(null, new Path(sd.getLocation()), conf, + tblValidWriteIds, Ref.from(false), true); + } + + UserGroupInformation ugi = UserGroupInformation.createProxyUser(ci.runAs, UserGroupInformation.getLoginUser()); + try { + return ugi.doAs((PrivilegedExceptionAction) () -> + AcidUtils.getAcidState(null, new Path(sd.getLocation()), conf, tblValidWriteIds, + Ref.from(false), true)); + } finally { + try { + FileSystem.closeAllForUGI(ugi); + } catch (IOException exception) { + LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " + ci.getFullPartitionName(), + exception); + } + } + } + + public void cleanupResultDirs(CompactionInfo ci) { + // result directory for compactor to write new files + Path resultDir = QueryCompactor.Util.getCompactionResultDir(sd, tblValidWriteIds, conf, + ci.type == CompactionType.MAJOR, false, false, dir); + LOG.info("Deleting result directories created by the compactor:\n"); + try { + FileSystem fs = resultDir.getFileSystem(conf); + LOG.info(resultDir.toString()); + fs.delete(resultDir, true); + + if (ci.type == CompactionType.MINOR) { + Path deleteDeltaDir = QueryCompactor.Util.getCompactionResultDir(sd, tblValidWriteIds, conf, + false, true, false, dir); + + LOG.info(deleteDeltaDir.toString()); + fs.delete(deleteDeltaDir, true); + } + } catch (IOException ex) { + LOG.error("Caught exception while cleaning result directories:", ex); + } + } + + public Boolean compact(Table table, CompactionInfo ci) throws Exception { + + try (CompactionTxn compactionTxn = new CompactionTxn()) { + + if (ci.isRebalanceCompaction() && table.getSd().getNumBuckets() > 0) { + LOG.error("Cannot execute rebalancing compaction on bucketed tables."); + ci.errorMessage = "Cannot execute rebalancing compaction on bucketed tables."; + msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); + return false; + } + + if (!ci.type.equals(CompactionType.REBALANCE) && ci.numberOfBuckets > 0) { + if (LOG.isWarnEnabled()) { + LOG.warn("Only the REBALANCE compaction accepts the number of buckets clause (CLUSTERED INTO {N} BUCKETS). " + + "Since the compaction request is {}, it will be ignored.", ci.type); + } + } + + String fullTableName = TxnUtils.getFullTableName(table.getDbName(), table.getTableName()); + + // Find the partition we will be working with, if there is one. + Partition p; + try { + p = CompactorUtil.resolvePartition(conf, msc, ci.dbname, ci.tableName, ci.partName, + CompactorUtil.METADATA_FETCH_MODE.REMOTE); + if (p == null && ci.partName != null) { + ci.errorMessage = "Unable to find partition " + ci.getFullPartitionName() + ", assuming it was dropped and moving on."; + LOG.warn(ci.errorMessage + " Compaction info: {}", ci); + msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); + return false; + } + } catch (Exception e) { + LOG.error("Unexpected error during resolving partition.", e); + ci.errorMessage = e.getMessage(); + msc.markFailed(CompactionInfo.compactionInfoToStruct(ci)); + return false; + } + + CompactorUtil.checkInterrupt(CLASS_NAME); + + // Find the appropriate storage descriptor + sd = CompactorUtil.resolveStorageDescriptor(table, p); + + if (isTableSorted(sd, ci)) { + return false; + } + + if (ci.runAs == null) { + ci.runAs = TxnUtils.findUserToRunAs(sd.getLocation(), table, conf); + } + + CompactorUtil.checkInterrupt(CLASS_NAME); + + /** + * we cannot have Worker use HiveTxnManager (which is on ThreadLocal) since + * then the Driver would already have the an open txn but then this txn would have + * multiple statements in it (for query based compactor) which is not supported (and since + * this case some of the statements are DDL, even in the future will not be allowed in a + * multi-stmt txn. {@link Driver#setCompactionWriteIds(ValidWriteIdList, long)} */ + compactionTxn.open(ci); + + ValidTxnList validTxnList = msc.getValidTxns(compactionTxn.getTxnId()); + //with this ValidWriteIdList is capped at whatever HWM validTxnList has + tblValidWriteIds = TxnUtils.createValidCompactWriteIdList(msc.getValidWriteIds( + Collections.singletonList(fullTableName), validTxnList.writeToString()).get(0)); + LOG.debug("ValidCompactWriteIdList: " + tblValidWriteIds.writeToString()); + conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); + + ci.highestWriteId = tblValidWriteIds.getHighWatermark(); + //this writes TXN_COMPONENTS to ensure that if compactorTxnId fails, we keep metadata about + //it until after any data written by it are physically removed + msc.updateCompactorState(CompactionInfo.compactionInfoToStruct(ci), compactionTxn.getTxnId()); + + CompactorUtil.checkInterrupt(CLASS_NAME); + + // Don't start compaction or cleaning if not necessary + if (isDynPartAbort(table, ci)) { + msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci)); + compactionTxn.wasSuccessful(); + return false; + } + dir = getAcidStateForWorker(ci, sd, tblValidWriteIds); + if (!isEnoughToCompact(ci, dir, sd)) { + if (needsCleaning(dir, sd)) { + msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci)); + } else { + // do nothing + ci.errorMessage = "None of the compaction thresholds met, compaction request is refused!"; + LOG.debug(ci.errorMessage + " Compaction info: {}", ci); + msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); + } + compactionTxn.wasSuccessful(); + return false; + } + if (!ci.isMajorCompaction() && !CompactorUtil.isMinorCompactionSupported(conf, table.getParameters(), dir)) { + ci.errorMessage = "Query based Minor compaction is not possible for full acid tables having raw format " + + "(non-acid) data in them."; + LOG.error(ci.errorMessage + " Compaction info: {}", ci); + try { + msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); + } catch (Throwable tr) { + LOG.error("Caught an exception while trying to mark compaction {} as failed: {}", ci, tr); + } + return false; + } + CompactorUtil.checkInterrupt(CLASS_NAME); + + try { + failCompactionIfSetForTest(); + + /* + First try to run compaction via HiveQL queries. + Compaction for MM tables happens here, or run compaction for Crud tables if query-based compaction is enabled. + todo Find a more generic approach to collecting files in the same logical bucket to compact within the same + task (currently we're using Tez split grouping). + */ + CompactorPipeline compactorPipeline = compactorFactory.getCompactorPipeline(table, conf, ci, msc); + computeStats = (compactorPipeline.isMRCompaction() && collectMrStats) || collectGenericStats; + + LOG.info("Starting " + ci.type.toString() + " compaction for " + ci.getFullPartitionName() + ", id:" + + ci.id + " in " + compactionTxn + " with compute stats set to " + computeStats); + + CompactorContext compactorContext = new CompactorContext(conf, table, p, sd, tblValidWriteIds, ci, dir); + compactorPipeline.execute(compactorContext); + + LOG.info("Completed " + ci.type.toString() + " compaction for " + ci.getFullPartitionName() + " in " + + compactionTxn + ", marking as compacted."); + msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci)); + compactionTxn.wasSuccessful(); + + AcidMetricService.updateMetricsFromWorker(ci.dbname, ci.tableName, ci.partName, ci.type, + dir.getCurrentDirectories().size(), dir.getDeleteDeltas().size(), conf, msc); + } catch (Throwable e) { + computeStats = false; + throw e; + } + + return true; + } catch (Exception e) { + LOG.error("Caught exception in " + CLASS_NAME + " while trying to compact " + ci, e); + throw e; + } + } + + /** + * Determine if compaction can run in a specified directory. + * @param ci {@link CompactionInfo} + * @param dir the delta directory + * @param sd resolved storage descriptor + * @return true, if compaction can run. + */ + static boolean isEnoughToCompact(CompactionInfo ci, AcidDirectory dir, StorageDescriptor sd) { + int deltaCount = dir.getCurrentDirectories().size(); + int origCount = dir.getOriginalFiles().size(); + + StringBuilder deltaInfo = new StringBuilder().append(deltaCount); + boolean isEnoughToCompact; + + if (ci.isRebalanceCompaction()) { + //TODO: For now, we are allowing rebalance compaction regardless of the table state. Thresholds will be added later. + return true; + } else if (ci.isMajorCompaction()) { + isEnoughToCompact = + (origCount > 0 || deltaCount + (dir.getBaseDirectory() == null ? 0 : 1) > 1); + + } else { + isEnoughToCompact = (deltaCount > 1); + + if (deltaCount == 2) { + Map deltaByType = dir.getCurrentDirectories().stream().collect(Collectors + .groupingBy(delta -> (delta + .isDeleteDelta() ? AcidUtils.DELETE_DELTA_PREFIX : AcidUtils.DELTA_PREFIX), + Collectors.counting())); + + isEnoughToCompact = (deltaByType.size() != deltaCount); + deltaInfo.append(" ").append(deltaByType); + } + } + + if (!isEnoughToCompact) { + LOG.info("Not enough files in {} to compact; current base: {}, delta files: {}, originals: {}", + sd.getLocation(), dir.getBaseDirectory(), deltaInfo, origCount); + } + return isEnoughToCompact; + } + + /** + * Check for obsolete directories, and return true if any exist and Cleaner should be + * run. For example if we insert overwrite into a table with only deltas, a new base file with + * the highest writeId is created so there will be no live delta directories, only obsolete + * ones. Compaction is not needed, but the cleaner should still be run. + * + * @return true if cleaning is needed + */ + public static boolean needsCleaning(AcidDirectory dir, StorageDescriptor sd) { + int numObsoleteDirs = dir.getObsolete().size() + dir.getAbortedDirectories().size(); + boolean needsJustCleaning = numObsoleteDirs > 0; + if (needsJustCleaning) { + LOG.info("{} obsolete directories in {} found; marked for cleaning.", numObsoleteDirs, + sd.getLocation()); + } + return needsJustCleaning; + } + + /** + * Keep track of the compaction's transaction and its operations. + */ + class CompactionTxn implements AutoCloseable { + private long txnId = 0; + private long lockId = 0; + + private TxnStatus status = TxnStatus.UNKNOWN; + private boolean successfulCompaction = false; + + /** + * Try to open a new txn. + * @throws TException + */ + void open(CompactionInfo ci) throws TException { + this.txnId = msc.openTxn(ci.runAs, ci.type == CompactionType.REBALANCE ? TxnType.REBALANCE_COMPACTION : TxnType.COMPACTION); + status = TxnStatus.OPEN; + + LockRequest lockRequest; + if (CompactionType.REBALANCE.equals(ci.type)) { + lockRequest = CompactorUtil.createLockRequest(conf, ci, txnId, LockType.EXCL_WRITE, DataOperationType.UPDATE); + } else { + lockRequest = CompactorUtil.createLockRequest(conf, ci, txnId, LockType.SHARED_READ, DataOperationType.SELECT); + } + LockResponse res = msc.lock(lockRequest); + if (res.getState() != LockState.ACQUIRED) { + throw new TException("Unable to acquire lock(s) on {" + ci.getFullPartitionName() + + "}, status {" + res.getState() + "}, reason {" + res.getErrorMessage() + "}"); + } + lockId = res.getLockid(); + CompactionHeartbeatService.getInstance(conf).startHeartbeat(txnId, lockId, TxnUtils.getFullTableName(ci.dbname, ci.tableName)); + } + + /** + * Mark compaction as successful. This means the txn will be committed; otherwise it will be aborted. + */ + void wasSuccessful() { + this.successfulCompaction = true; + } + + /** + * Commit or abort txn. + * @throws Exception + */ + @Override public void close() throws Exception { + if (status == TxnStatus.UNKNOWN) { + return; + } + try { + //the transaction is about to close, we can stop heartbeating regardless of it's state + CompactionHeartbeatService.getInstance(conf).stopHeartbeat(txnId); + } finally { + if (successfulCompaction) { + commit(); + } else { + abort(); + } + } + } + + long getTxnId() { + return txnId; + } + + @Override public String toString() { + return "txnId=" + txnId + ", lockId=" + lockId + " (TxnStatus: " + status + ")"; + } + + /** + * Commit the txn if open. + */ + private void commit() throws TException { + if (status == TxnStatus.OPEN) { + msc.commitTxn(txnId); + status = TxnStatus.COMMITTED; + } + } + + /** + * Abort the txn if open. + */ + private void abort() throws TException { + if (status == TxnStatus.OPEN) { + AbortTxnRequest abortTxnRequest = new AbortTxnRequest(txnId); + abortTxnRequest.setErrorCode(TxnErrorMsg.ABORT_COMPACTION_TXN.getErrorCode()); + msc.rollbackTxn(abortTxnRequest); + status = TxnStatus.ABORTED; + } + } + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionExecutorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionExecutorFactory.java new file mode 100644 index 000000000000..56536d9692b9 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionExecutorFactory.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.txn.compactor.service; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorFactory; + +public class CompactionExecutorFactory { + + private static final String ICEBERG_COMPACTION_SERVICE_CLASS = "org.apache.iceberg.mr.hive.compaction.IcebergCompactionService"; + + public static CompactionService createExecutor(HiveConf conf, IMetaStoreClient msc, CompactorFactory compactorFactory, + Table table, boolean collectGenericStats, boolean collectMrStats) throws HiveException { + + CompactionService compactionService; + + if (MetaStoreUtils.isIcebergTable(table.getParameters())) { + + try { + Class icebergCompactionService = (Class) + Class.forName(ICEBERG_COMPACTION_SERVICE_CLASS, true, + Utilities.getSessionSpecifiedClassLoader()); + + compactionService = icebergCompactionService.newInstance(); + compactionService.init(conf, msc, compactorFactory, collectGenericStats); + } + catch (Exception e) { + throw new HiveException("Failed instantiating and calling Iceberg compaction executor", e); + } + } + else { + compactionService = new AcidCompactionService(conf, msc, compactorFactory, collectGenericStats, + collectMrStats); + } + + return compactionService; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java new file mode 100644 index 000000000000..c896ffc77cf1 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.txn.compactor.service; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.ql.txn.compactor.CompactorFactory; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public abstract class CompactionService { + + static final private String CLASS_NAME = CompactionService.class.getName(); + static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME); + + protected IMetaStoreClient msc; + protected HiveConf conf; + protected CompactorFactory compactorFactory; + protected boolean collectGenericStats; + protected boolean computeStats = false; + + public CompactionService(HiveConf conf, IMetaStoreClient msc, CompactorFactory compactorFactory, + boolean collectGenericStats) { + init(conf, msc, compactorFactory, collectGenericStats); + } + + public CompactionService() { + } + + public void init(HiveConf conf, IMetaStoreClient msc, CompactorFactory compactorFactory, + boolean collectGenericStats) { + this.conf = conf; + this.msc = msc; + this.compactorFactory = compactorFactory; + this.collectGenericStats = collectGenericStats; + } + + public abstract Boolean compact(Table table, CompactionInfo ci) throws Exception; + abstract public void cleanupResultDirs(CompactionInfo ci); + + public boolean isComputeStats() { + return computeStats; + } + + protected boolean isDynPartAbort(Table t, CompactionInfo ci) { + return t.getPartitionKeys() != null && t.getPartitionKeys().size() > 0 + && ci.partName == null; + } + + protected void failCompactionIfSetForTest() { + if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION)) { + throw new RuntimeException(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION.name() + "=true"); + } + } + + protected boolean isTableSorted(StorageDescriptor sd, CompactionInfo ci) throws TException { + // Check that the table or partition isn't sorted, as we don't yet support that. + if (sd.getSortCols() != null && !sd.getSortCols().isEmpty()) { + ci.errorMessage = "Attempt to compact sorted table " + ci.getFullTableName() + ", which is not yet supported!"; + LOG.warn(ci.errorMessage + " Compaction info: {}", ci); + msc.markRefused(CompactionInfo.compactionInfoToStruct(ci)); + return true; + } + return false; + } +} From a823babaf60e642119d007676176b35e912ae99c Mon Sep 17 00:00:00 2001 From: Devaspati Date: Mon, 18 Dec 2023 10:10:39 +0530 Subject: [PATCH 100/179] HIVE-27824 : Upgrade ivy to 2.5.2 and htmlunit to 2.70.0 (#4939) (Devaspati Krishnatri reviewed by Attila Turoczy, Sourabh Badhya) --- itests/hive-unit/pom.xml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index 29ece2167223..7c3b2333b9eb 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -26,7 +26,7 @@ ../.. 1.15.2 - 2.67.0 + 2.70.0 diff --git a/pom.xml b/pom.xml index 376c698d181a..62ba8c78d613 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 4.5.13 4.4.13 - 2.5.1 + 2.5.2 2.13.5 2.3.4 2.4.1 From fd92b3926393f0366b87cd55d5a0ad27968f18db Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Mon, 18 Dec 2023 06:17:10 +0100 Subject: [PATCH 101/179] HIVE-27690: Handle casting NULL literal to complex type (Krisztian Kasa, reviewed by Laszlo Vegh) --- .../hadoop/hive/ql/parse/IdentifiersParser.g | 2 +- .../hadoop/hive/ql/exec/FunctionRegistry.java | 3 + .../calcite/translator/ASTConverter.java | 82 ++++---- .../calcite/translator/RexNodeConverter.java | 8 +- .../hadoop/hive/ql/parse/ParseUtils.java | 51 ++++- .../parse/type/ExprNodeDescExprFactory.java | 108 +++++------ .../ql/parse/type/TypeCheckProcFactory.java | 11 +- .../hive/ql/plan/ExprNodeConstantDesc.java | 3 + .../ql/udf/generic/GenericUDFToArray.java | 59 ++++++ .../hive/ql/udf/generic/GenericUDFToMap.java | 63 +++++++ .../ql/udf/generic/GenericUDFToStruct.java | 58 ++++++ .../calcite/translator/TestASTConverter.java | 178 ++++++++++-------- .../type/TestExprNodeDescExprFactory.java | 175 +++++++++++++++++ .../clientpositive/cast_null_to_complex.q | 13 ++ .../clientpositive/empty_result_ctas.q | 12 ++ .../clientpositive/llap/analyze_npe.q.out | 7 +- .../llap/cast_null_to_complex.q.out | 107 +++++++++++ .../llap/empty_result_ctas.q.out | 124 +++++++++++- .../clientpositive/llap/show_functions.q.out | 6 + 19 files changed, 875 insertions(+), 195 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToArray.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToMap.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToStruct.java create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprNodeDescExprFactory.java create mode 100644 ql/src/test/queries/clientpositive/cast_null_to_complex.q create mode 100644 ql/src/test/results/clientpositive/llap/cast_null_to_complex.q.out diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g index 31b63e66345b..fc34da643891 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g @@ -327,7 +327,7 @@ castExpression LPAREN expression KW_AS - toType=primitiveType + toType=type (fmt=KW_FORMAT StringLiteral)? RPAREN // simple cast diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java index 1f411971a806..c0b9519944dd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java @@ -486,6 +486,9 @@ public final class FunctionRegistry { system.registerGenericUDF("!", GenericUDFOPNot.class); system.registerGenericUDF("between", GenericUDFBetween.class); system.registerGenericUDF("in_bloom_filter", GenericUDFInBloomFilter.class); + system.registerGenericUDF("toMap", GenericUDFToMap.class); + system.registerGenericUDF("toArray", GenericUDFToArray.class); + system.registerGenericUDF("toStruct", GenericUDFToStruct.class); // Utility UDFs system.registerUDF("version", UDFVersion.class, false); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java index dec4deddd7d7..2c0fe365dac8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java @@ -180,9 +180,7 @@ public static ASTNode emptyPlan(RelDataType dataType) { ASTBuilder select = ASTBuilder.construct(HiveParser.TOK_SELECT, "TOK_SELECT"); for (int i = 0; i < dataType.getFieldCount(); ++i) { RelDataTypeField fieldType = dataType.getFieldList().get(i); - select.add(ASTBuilder.selectExpr( - createNullField(fieldType.getType()), - fieldType.getName())); + select.add(ASTBuilder.selectExpr(createNullField(fieldType.getType()), fieldType.getName())); } ASTNode insert = ASTBuilder. @@ -203,53 +201,52 @@ private static ASTNode createNullField(RelDataType fieldType) { return ASTBuilder.construct(HiveParser.TOK_NULL, "TOK_NULL").node(); } + ASTNode astNode = convertType(fieldType); + return ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION") + .add(astNode) + .add(HiveParser.TOK_NULL, "TOK_NULL") + .node(); + } + + static ASTNode convertType(RelDataType fieldType) { + if (fieldType.getSqlTypeName() == SqlTypeName.NULL) { + return ASTBuilder.construct(HiveParser.TOK_NULL, "TOK_NULL").node(); + } + if (fieldType.getSqlTypeName() == SqlTypeName.ROW) { - ASTBuilder namedStructCallNode = ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION"); - namedStructCallNode.add(HiveParser.Identifier, "named_struct"); + ASTBuilder columnListNode = ASTBuilder.construct(HiveParser.TOK_TABCOLLIST, "TOK_TABCOLLIST"); for (RelDataTypeField structFieldType : fieldType.getFieldList()) { - namedStructCallNode.add(HiveParser.Identifier, structFieldType.getName()); - namedStructCallNode.add(createNullField(structFieldType.getType())); + ASTNode colNode = ASTBuilder.construct(HiveParser.TOK_TABCOL, "TOK_TABCOL") + .add(HiveParser.Identifier, structFieldType.getName()) + .add(convertType(structFieldType.getType())) + .node(); + columnListNode.add(colNode); } - return namedStructCallNode.node(); + return ASTBuilder.construct(HiveParser.TOK_STRUCT, "TOK_STRUCT").add(columnListNode).node(); } if (fieldType.getSqlTypeName() == SqlTypeName.MAP) { - ASTBuilder mapCallNode = ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION"); - mapCallNode.add(HiveParser.Identifier, "map"); - mapCallNode.add(createNullField(fieldType.getKeyType())); - mapCallNode.add(createNullField(fieldType.getValueType())); + ASTBuilder mapCallNode = ASTBuilder.construct(HiveParser.TOK_MAP, "TOK_MAP"); + mapCallNode.add(convertType(fieldType.getKeyType())); + mapCallNode.add(convertType(fieldType.getValueType())); return mapCallNode.node(); } if (fieldType.getSqlTypeName() == SqlTypeName.ARRAY) { - ASTBuilder arrayCallNode = ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION"); - arrayCallNode.add(HiveParser.Identifier, "array"); - arrayCallNode.add(createNullField(fieldType.getComponentType())); + ASTBuilder arrayCallNode = ASTBuilder.construct(HiveParser.TOK_LIST, "TOK_LIST"); + arrayCallNode.add(convertType(fieldType.getComponentType())); return arrayCallNode.node(); } - return createCastNull(fieldType); - } - - private static ASTNode createCastNull(RelDataType fieldType) { HiveToken ht = TypeConverter.hiveToken(fieldType); - ASTNode typeNode; - if (ht == null) { - typeNode = ASTBuilder.construct( - HiveParser.Identifier, fieldType.getSqlTypeName().getName().toLowerCase()).node(); - } else { - ASTBuilder typeNodeBuilder = ASTBuilder.construct(ht.type, ht.text); - if (ht.args != null) { - for (String castArg : ht.args) { - typeNodeBuilder.add(HiveParser.Identifier, castArg); - } + ASTBuilder astBldr = ASTBuilder.construct(ht.type, ht.text); + if (ht.args != null) { + for (String castArg : ht.args) { + astBldr.add(HiveParser.Identifier, castArg); } - typeNode = typeNodeBuilder.node(); } - return ASTBuilder.construct(HiveParser.TOK_FUNCTION, "TOK_FUNCTION") - .add(typeNode) - .add(HiveParser.TOK_NULL, "TOK_NULL") - .node(); + + return astBldr.node(); } private ASTNode convert() throws CalciteSemanticException { @@ -1042,22 +1039,7 @@ public ASTNode visitCall(RexCall call) { Collections.singletonList(SqlFunctionConverter.buildAST(SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, astNodeLst, call.getType())), call.getType()); case CAST: assert(call.getOperands().size() == 1); - if (call.getType().isStruct() || - SqlTypeName.MAP.equals(call.getType().getSqlTypeName()) || - SqlTypeName.ARRAY.equals(call.getType().getSqlTypeName())) { - // cast for complex types can be ignored safely because explicit casting on such - // types are not possible, implicit casting e.g. CAST(ROW__ID as <...>) can be ignored - return call.getOperands().get(0).accept(this); - } - - HiveToken ht = TypeConverter.hiveToken(call.getType()); - ASTBuilder astBldr = ASTBuilder.construct(ht.type, ht.text); - if (ht.args != null) { - for (String castArg : ht.args) { - astBldr.add(HiveParser.Identifier, castArg); - } - } - astNodeLst.add(astBldr.node()); + astNodeLst.add(convertType(call.getType())); astNodeLst.add(call.getOperands().get(0).accept(this)); break; case EXTRACT: diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java index 89d6024cc06e..e6c41c3b6206 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java @@ -76,11 +76,14 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCase; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFTimestamp; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToArray; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToBinary; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToChar; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDate; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDecimal; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToMap; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToString; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToStruct; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToTimestampLocalTZ; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToVarchar; @@ -334,7 +337,10 @@ public static RexNode handleExplicitCast(GenericUDF udf, RelDataType returnType, || (udf instanceof GenericUDFToString) || (udf instanceof GenericUDFToDecimal) || (udf instanceof GenericUDFToDate) || (udf instanceof GenericUDFTimestamp) || (udf instanceof GenericUDFToTimestampLocalTZ) - || (udf instanceof GenericUDFToBinary) || castExprUsingUDFBridge(udf)) { + || (udf instanceof GenericUDFToBinary) || castExprUsingUDFBridge(udf) + || (udf instanceof GenericUDFToMap) + || (udf instanceof GenericUDFToArray) + || (udf instanceof GenericUDFToStruct)) { castExpr = rexBuilder.makeAbstractCast(returnType, childRexNodeLst.get(0)); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java index a3a60f3d5c41..db959192db71 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Queue; @@ -60,13 +61,20 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.getTypeStringFromAST; +import static org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.unescapeIdentifier; + /** * Library of utility functions used in the parse code. @@ -204,7 +212,7 @@ static int checkJoinFilterRefersOneAlias(String[] tabAliases, ASTNode filterCond switch(filterCondn.getType()) { case HiveParser.TOK_TABLE_OR_COL: - String tableOrCol = SemanticAnalyzer.unescapeIdentifier(filterCondn.getChild(0).getText() + String tableOrCol = unescapeIdentifier(filterCondn.getChild(0).getText() .toLowerCase()); return getIndex(tabAliases, tableOrCol); case HiveParser.Identifier: @@ -725,4 +733,45 @@ public static final class ReparseResult { } } + public static TypeInfo getComplexTypeTypeInfo(ASTNode typeNode) throws SemanticException { + switch (typeNode.getType()) { + case HiveParser.TOK_LIST: + ListTypeInfo listTypeInfo = new ListTypeInfo(); + listTypeInfo.setListElementTypeInfo(getComplexTypeTypeInfo((ASTNode) typeNode.getChild(0))); + return listTypeInfo; + case HiveParser.TOK_MAP: + MapTypeInfo mapTypeInfo = new MapTypeInfo(); + String keyTypeString = getTypeStringFromAST((ASTNode) typeNode.getChild(0)); + mapTypeInfo.setMapKeyTypeInfo(TypeInfoFactory.getPrimitiveTypeInfo(keyTypeString)); + mapTypeInfo.setMapValueTypeInfo(getComplexTypeTypeInfo((ASTNode) typeNode.getChild(1))); + return mapTypeInfo; + case HiveParser.TOK_STRUCT: + StructTypeInfo structTypeInfo = new StructTypeInfo(); + Map fields = collectStructFieldNames(typeNode); + structTypeInfo.setAllStructFieldNames(new ArrayList<>(fields.keySet())); + structTypeInfo.setAllStructFieldTypeInfos(new ArrayList<>(fields.values())); + return structTypeInfo; + default: + String typeString = getTypeStringFromAST(typeNode); + return TypeInfoFactory.getPrimitiveTypeInfo(typeString); + } + } + + private static Map collectStructFieldNames(ASTNode structTypeNode) throws SemanticException { + ASTNode fieldListNode = (ASTNode) structTypeNode.getChild(0); + assert fieldListNode.getType() == HiveParser.TOK_TABCOLLIST; + + Map result = new LinkedHashMap<>(fieldListNode.getChildCount()); + for (int i = 0; i < fieldListNode.getChildCount(); i++) { + ASTNode child = (ASTNode) fieldListNode.getChild(i); + + String attributeIdentifier = unescapeIdentifier(child.getChild(0).getText()); + if (result.containsKey(attributeIdentifier)) { + throw new SemanticException(ErrorMsg.AMBIGUOUS_STRUCT_ATTRIBUTE, attributeIdentifier); + } else { + result.put(attributeIdentifier, getComplexTypeTypeInfo((ASTNode) child.getChild(1))); + } + } + return result; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/ExprNodeDescExprFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/ExprNodeDescExprFactory.java index 40c5f6da1922..b27b98f5f7e5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/ExprNodeDescExprFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/ExprNodeDescExprFactory.java @@ -113,30 +113,40 @@ protected boolean isExprInstance(Object o) { protected ExprNodeDesc toExpr(ColumnInfo colInfo, RowResolver rowResolver, int offset) throws SemanticException { ObjectInspector inspector = colInfo.getObjectInspector(); - if (inspector instanceof ConstantObjectInspector && inspector instanceof PrimitiveObjectInspector) { - return toPrimitiveConstDesc(colInfo, inspector); - } - if (inspector instanceof ConstantObjectInspector && inspector instanceof ListObjectInspector) { - ObjectInspector listElementOI = ((ListObjectInspector)inspector).getListElementObjectInspector(); - if (listElementOI instanceof PrimitiveObjectInspector) { - return toListConstDesc(colInfo, inspector, listElementOI); + if (inspector instanceof ConstantObjectInspector) { + if (inspector instanceof PrimitiveObjectInspector) { + return toPrimitiveConstDesc(colInfo, inspector); } - } - if (inspector instanceof ConstantObjectInspector && inspector instanceof MapObjectInspector) { - ObjectInspector keyOI = ((MapObjectInspector)inspector).getMapKeyObjectInspector(); - ObjectInspector valueOI = ((MapObjectInspector)inspector).getMapValueObjectInspector(); - if (keyOI instanceof PrimitiveObjectInspector && valueOI instanceof PrimitiveObjectInspector) { - return toMapConstDesc(colInfo, inspector, keyOI, valueOI); + + Object inputConstantValue = ((ConstantObjectInspector) inspector).getWritableConstantValue(); + if (inputConstantValue == null) { + return createExprNodeConstantDesc(colInfo, null); } - } - if (inspector instanceof ConstantObjectInspector && inspector instanceof StructObjectInspector) { - boolean allPrimitive = true; - List fields = ((StructObjectInspector)inspector).getAllStructFieldRefs(); - for (StructField field : fields) { - allPrimitive &= field.getFieldObjectInspector() instanceof PrimitiveObjectInspector; + + if (inspector instanceof ListObjectInspector) { + ObjectInspector listElementOI = ((ListObjectInspector) inspector).getListElementObjectInspector(); + if (listElementOI instanceof PrimitiveObjectInspector) { + PrimitiveObjectInspector poi = (PrimitiveObjectInspector) listElementOI; + return createExprNodeConstantDesc(colInfo, toListConstant((List) inputConstantValue, poi)); + } } - if (allPrimitive) { - return toStructConstDesc(colInfo, inspector, fields); + if (inspector instanceof MapObjectInspector) { + ObjectInspector keyOI = ((MapObjectInspector)inspector).getMapKeyObjectInspector(); + ObjectInspector valueOI = ((MapObjectInspector)inspector).getMapValueObjectInspector(); + if (keyOI instanceof PrimitiveObjectInspector && valueOI instanceof PrimitiveObjectInspector) { + return createExprNodeConstantDesc(colInfo, toMapConstant((Map) inputConstantValue, keyOI, valueOI)); + } + } + if (inspector instanceof StructObjectInspector) { + boolean allPrimitive = true; + List fields = ((StructObjectInspector)inspector).getAllStructFieldRefs(); + for (StructField field : fields) { + allPrimitive &= field.getFieldObjectInspector() instanceof PrimitiveObjectInspector; + } + if (allPrimitive) { + return createExprNodeConstantDesc(colInfo, toStructConstDesc( + (List) ((ConstantObjectInspector) inspector).getWritableConstantValue(), fields)); + } } } // non-constant or non-primitive constants @@ -145,6 +155,13 @@ protected ExprNodeDesc toExpr(ColumnInfo colInfo, RowResolver rowResolver, int o return column; } + private static ExprNodeConstantDesc createExprNodeConstantDesc(ColumnInfo colInfo, Object constantValue) { + ExprNodeConstantDesc constantExpr = new ExprNodeConstantDesc(colInfo.getType(), constantValue); + constantExpr.setFoldedFromCol(colInfo.getInternalName()); + constantExpr.setFoldedFromTab(colInfo.getTabAlias()); + return constantExpr; + } + private static ExprNodeConstantDesc toPrimitiveConstDesc(ColumnInfo colInfo, ObjectInspector inspector) { PrimitiveObjectInspector poi = (PrimitiveObjectInspector) inspector; Object constant = ((ConstantObjectInspector) inspector).getWritableConstantValue(); @@ -155,50 +172,33 @@ private static ExprNodeConstantDesc toPrimitiveConstDesc(ColumnInfo colInfo, Obj return constantExpr; } - private static ExprNodeConstantDesc toListConstDesc(ColumnInfo colInfo, ObjectInspector inspector, - ObjectInspector listElementOI) { - PrimitiveObjectInspector poi = (PrimitiveObjectInspector)listElementOI; - List values = (List)((ConstantObjectInspector) inspector).getWritableConstantValue(); - List constant = new ArrayList(); - for (Object o : values) { + private static List toListConstant(List constantValue, PrimitiveObjectInspector poi) { + List constant = new ArrayList<>(constantValue.size()); + for (Object o : constantValue) { constant.add(poi.getPrimitiveJavaObject(o)); } - - ExprNodeConstantDesc constantExpr = new ExprNodeConstantDesc(colInfo.getType(), constant); - constantExpr.setFoldedFromCol(colInfo.getInternalName()); - constantExpr.setFoldedFromTab(colInfo.getTabAlias()); - return constantExpr; + return constant; } - private static ExprNodeConstantDesc toMapConstDesc(ColumnInfo colInfo, ObjectInspector inspector, - ObjectInspector keyOI, ObjectInspector valueOI) { - PrimitiveObjectInspector keyPoi = (PrimitiveObjectInspector)keyOI; - PrimitiveObjectInspector valuePoi = (PrimitiveObjectInspector)valueOI; - Map values = (Map)((ConstantObjectInspector) inspector).getWritableConstantValue(); - Map constant = new LinkedHashMap(); - for (Map.Entry e : values.entrySet()) { + private static Map toMapConstant( + Map constantValue, ObjectInspector keyOI, ObjectInspector valueOI) { + PrimitiveObjectInspector keyPoi = (PrimitiveObjectInspector) keyOI; + PrimitiveObjectInspector valuePoi = (PrimitiveObjectInspector) valueOI; + Map constant = new LinkedHashMap<>(constantValue.size()); + for (Map.Entry e : constantValue.entrySet()) { constant.put(keyPoi.getPrimitiveJavaObject(e.getKey()), valuePoi.getPrimitiveJavaObject(e.getValue())); } - - ExprNodeConstantDesc constantExpr = new ExprNodeConstantDesc(colInfo.getType(), constant); - constantExpr.setFoldedFromCol(colInfo.getInternalName()); - constantExpr.setFoldedFromTab(colInfo.getTabAlias()); - return constantExpr; + return constant; } - private static ExprNodeConstantDesc toStructConstDesc(ColumnInfo colInfo, ObjectInspector inspector, - List fields) { - List values = (List)((ConstantObjectInspector) inspector).getWritableConstantValue(); - List constant = new ArrayList(); - for (int i = 0; i < values.size(); i++) { - Object value = values.get(i); + private static List toStructConstDesc(List constantValue, List fields) { + List constant = new ArrayList<>(constantValue.size()); + for (int i = 0; i < constantValue.size(); i++) { + Object value = constantValue.get(i); PrimitiveObjectInspector fieldPoi = (PrimitiveObjectInspector) fields.get(i).getFieldObjectInspector(); constant.add(fieldPoi.getPrimitiveJavaObject(value)); } - ExprNodeConstantDesc constantExpr = new ExprNodeConstantDesc(colInfo.getType(), constant); - constantExpr.setFoldedFromCol(colInfo.getInternalName()); - constantExpr.setFoldedFromTab(colInfo.getTabAlias()); - return constantExpr; + return constant; } /** diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java index 9837b325230b..318d0c87792c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java @@ -139,6 +139,9 @@ public class TypeCheckProcFactory { serdeConstants.INTERVAL_DAY_TIME_TYPE_NAME); CONVERSION_FUNCTION_TEXT_MAP.put(HiveParser.TOK_DECIMAL, serdeConstants.DECIMAL_TYPE_NAME); + CONVERSION_FUNCTION_TEXT_MAP.put(HiveParser.TOK_MAP, "toMap"); + CONVERSION_FUNCTION_TEXT_MAP.put(HiveParser.TOK_LIST, "toArray"); + CONVERSION_FUNCTION_TEXT_MAP.put(HiveParser.TOK_STRUCT, "toStruct"); WINDOWING_TOKENS = new HashSet(); WINDOWING_TOKENS.add(HiveParser.KW_OVER); @@ -1134,6 +1137,10 @@ private TypeInfo getTypeInfo(ASTNode funcNameNode) throws SemanticException { return timestampLocalTZTypeInfo; case HiveParser.TOK_DECIMAL: return ParseUtils.getDecimalTypeTypeInfo(funcNameNode); + case HiveParser.TOK_MAP: + case HiveParser.TOK_LIST: + case HiveParser.TOK_STRUCT: + return ParseUtils.getComplexTypeTypeInfo(funcNameNode); default: return null; } @@ -1409,7 +1416,9 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // Return nulls for conversion operators if (CONVERSION_FUNCTION_TEXT_MAP.keySet().contains(expr.getType()) || expr.getToken().getType() == HiveParser.CharSetName - || expr.getToken().getType() == HiveParser.CharSetLiteral) { + || expr.getToken().getType() == HiveParser.CharSetLiteral + || expr.getType() == HiveParser.TOK_TABCOL + || expr.getType() == HiveParser.TOK_TABCOLLIST) { return null; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java index f5e3828e2cda..6709dc319e86 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java @@ -173,6 +173,9 @@ public String getExprString() { if (typeInfo.getCategory() == Category.PRIMITIVE) { return getFormatted(typeInfo, value); } else if (typeInfo.getCategory() == Category.STRUCT) { + if (getWritableObjectInspector().getWritableConstantValue() == null) { + return getFormatted(typeInfo, value); + } StringBuilder sb = new StringBuilder(); sb.append("const struct("); List items = (List) getWritableObjectInspector().getWritableConstantValue(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToArray.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToArray.java new file mode 100644 index 000000000000..915188a363a5 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToArray.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.SettableUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; + +@Description(name = "toArray", value = "_FUNC_(x) - converts it's parameter to _FUNC_" + + "Currently only null literal is supported.") +public class GenericUDFToArray extends GenericUDF implements SettableUDF { + private ListTypeInfo typeInfo; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + return TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(typeInfo); + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + return null; + } + + @Override + public String getDisplayString(String[] children) { + return String.format("toArray(%s)", String.join(",", children)); + } + + @Override + public void setTypeInfo(TypeInfo typeInfo) throws UDFArgumentException { + this.typeInfo = (ListTypeInfo) typeInfo; + } + + @Override + public TypeInfo getTypeInfo() { + return typeInfo; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToMap.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToMap.java new file mode 100644 index 000000000000..85d534ccbbff --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToMap.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.SettableUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; + +/** + * GenericUDFMap. + * + */ +@Description(name = "toMap", value = "_FUNC_(x) - converts it's parameter to _FUNC_" + + "Currently only null literal is supported.") +public class GenericUDFToMap extends GenericUDF implements SettableUDF { + private MapTypeInfo typeInfo; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + return TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(typeInfo); + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + return null; + } + + @Override + public String getDisplayString(String[] children) { + return String.format("toMap(%s)", String.join(",", children)); + } + + @Override + public void setTypeInfo(TypeInfo typeInfo) throws UDFArgumentException { + this.typeInfo = (MapTypeInfo) typeInfo; + } + + @Override + public TypeInfo getTypeInfo() { + return typeInfo; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToStruct.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToStruct.java new file mode 100644 index 000000000000..f59d11ce7730 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFToStruct.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.SettableUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; + +@Description(name = "toStruct", value = "_FUNC_(x) - converts it's parameter to _FUNC_" + + "Currently only null literal is supported.") +public class GenericUDFToStruct extends GenericUDF implements SettableUDF { + private StructTypeInfo typeInfo; + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + return TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(typeInfo); + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + return null; + } + + @Override + public String getDisplayString(String[] children) { + return String.format("toStruct(%s)", String.join(",", children)); } + + @Override + public void setTypeInfo(TypeInfo typeInfo) throws UDFArgumentException { + this.typeInfo = (StructTypeInfo) typeInfo; + } + + @Override + public TypeInfo getTypeInfo() { + return typeInfo; + } +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TestASTConverter.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TestASTConverter.java index ddea68429f82..3c48447d51e2 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TestASTConverter.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TestASTConverter.java @@ -36,11 +36,40 @@ import static java.util.Arrays.asList; import static java.util.Collections.singletonList; +import static org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.convertType; import static org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.emptyPlan; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.Is.is; class TestASTConverter { + + @Test + void testConvertTypeWhenInputIsStruct() { + List fields = asList( + new RelDataTypeFieldImpl("a", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), + new RelDataTypeFieldImpl("b", 1, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.CHAR, 30)), + new RelDataTypeFieldImpl("c", 2, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.NULL))); + + RelDataType dataType = new RelRecordType(fields); + + ASTNode tree = convertType(dataType); + assertThat(tree.dump(), is(EXPECTED_STRUCT_TREE)); + } + + private static final String EXPECTED_STRUCT_TREE = "\n" + + "TOK_STRUCT\n" + + " TOK_TABCOLLIST\n" + + " TOK_TABCOL\n" + + " a\n" + + " TOK_INT\n" + + " TOK_TABCOL\n" + + " b\n" + + " TOK_CHAR\n" + + " 30\n" + + " TOK_TABCOL\n" + + " c\n" + + " TOK_NULL\n"; + @Test void testEmptyPlanWhenInputSchemaIsEmpty() { RelRecordType dataType = new RelRecordType(Collections.emptyList()); @@ -54,9 +83,9 @@ void testEmptyPlanWhenInputSchemaIsEmpty() { @Test void testEmptyPlan() { List fields = asList( - new RelDataTypeFieldImpl("a", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), - new RelDataTypeFieldImpl("b", 1, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.CHAR, 30)), - new RelDataTypeFieldImpl("c", 2, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.NULL))); + new RelDataTypeFieldImpl("a", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), + new RelDataTypeFieldImpl("b", 1, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.CHAR, 30)), + new RelDataTypeFieldImpl("c", 2, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.NULL))); RelDataType dataType = new RelRecordType(fields); ASTNode tree = emptyPlan(dataType); @@ -65,96 +94,81 @@ void testEmptyPlan() { } private static final String EXPECTED_TREE = "\n" + - "TOK_QUERY\n" + - " TOK_INSERT\n" + - " TOK_DESTINATION\n" + - " TOK_DIR\n" + - " TOK_TMP_FILE\n" + - " TOK_SELECT\n" + - " TOK_SELEXPR\n" + - " TOK_FUNCTION\n" + - " TOK_INT\n" + - " TOK_NULL\n" + - " a\n" + - " TOK_SELEXPR\n" + - " TOK_FUNCTION\n" + - " TOK_CHAR\n" + - " 30\n" + - " TOK_NULL\n" + - " b\n" + - " TOK_SELEXPR\n" + - " TOK_NULL\n" + - " c\n" + - " TOK_LIMIT\n" + - " 0\n" + - " 0\n"; + "TOK_QUERY\n" + + " TOK_INSERT\n" + + " TOK_DESTINATION\n" + + " TOK_DIR\n" + + " TOK_TMP_FILE\n" + + " TOK_SELECT\n" + + " TOK_SELEXPR\n" + + " TOK_FUNCTION\n" + + " TOK_INT\n" + + " TOK_NULL\n" + + " a\n" + + " TOK_SELEXPR\n" + + " TOK_FUNCTION\n" + + " TOK_CHAR\n" + + " 30\n" + + " TOK_NULL\n" + + " b\n" + + " TOK_SELEXPR\n" + + " TOK_NULL\n" + + " c\n" + + " TOK_LIMIT\n" + + " 0\n" + + " 0\n"; @Test - void testEmptyPlanWithComplexTypes() { + void testEmptyPlanWithNestedComplexTypes() { List nestedStructFields = asList( - new RelDataTypeFieldImpl("nf1", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), - new RelDataTypeFieldImpl("nf2", 1, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.CHAR, 30))); + new RelDataTypeFieldImpl("nf1", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), + new RelDataTypeFieldImpl("nf2", 1, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.CHAR, 30))); List structFields = asList( - new RelDataTypeFieldImpl("f1", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), - new RelDataTypeFieldImpl("farray", 1, - new ArraySqlType(new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), true)), - new RelDataTypeFieldImpl("fmap", 2, new MapSqlType( - new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), - new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), true)), - new RelDataTypeFieldImpl("fstruct", 3, - new RelRecordType(nestedStructFields))); + new RelDataTypeFieldImpl("f1", 0, new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER)), + new RelDataTypeFieldImpl("farray", 1, + new ArraySqlType(new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), true)), + new RelDataTypeFieldImpl("fmap", 2, new MapSqlType( + new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), + new BasicSqlType(new HiveTypeSystemImpl(), SqlTypeName.INTEGER), true)), + new RelDataTypeFieldImpl("fstruct", 3, + new RelRecordType(nestedStructFields))); List fields = singletonList(new RelDataTypeFieldImpl("a", 0, new RelRecordType(structFields))); RelDataType dataType = new RelRecordType(fields); - ASTNode tree = emptyPlan(dataType); + ASTNode tree = convertType(dataType); assertThat(tree.dump(), is(EXPECTED_COMPLEX_TREE)); } private static final String EXPECTED_COMPLEX_TREE = "\n" + - "TOK_QUERY\n" + - " TOK_INSERT\n" + - " TOK_DESTINATION\n" + - " TOK_DIR\n" + - " TOK_TMP_FILE\n" + - " TOK_SELECT\n" + - " TOK_SELEXPR\n" + - " TOK_FUNCTION\n" + - " named_struct\n" + - " f1\n" + - " TOK_FUNCTION\n" + - " TOK_INT\n" + - " TOK_NULL\n" + - " farray\n" + - " TOK_FUNCTION\n" + - " array\n" + - " TOK_FUNCTION\n" + - " TOK_INT\n" + - " TOK_NULL\n" + - " fmap\n" + - " TOK_FUNCTION\n" + - " map\n" + - " TOK_FUNCTION\n" + - " TOK_INT\n" + - " TOK_NULL\n" + - " TOK_FUNCTION\n" + - " TOK_INT\n" + - " TOK_NULL\n" + - " fstruct\n" + - " TOK_FUNCTION\n" + - " named_struct\n" + - " nf1\n" + - " TOK_FUNCTION\n" + - " TOK_INT\n" + - " TOK_NULL\n" + - " nf2\n" + - " TOK_FUNCTION\n" + - " TOK_CHAR\n" + - " 30\n" + - " TOK_NULL\n" + - " a\n" + - " TOK_LIMIT\n" + - " 0\n" + - " 0\n"; + "TOK_STRUCT\n" + + " TOK_TABCOLLIST\n" + + " TOK_TABCOL\n" + + " a\n" + + " TOK_STRUCT\n" + + " TOK_TABCOLLIST\n" + + " TOK_TABCOL\n" + + " f1\n" + + " TOK_INT\n" + + " TOK_TABCOL\n" + + " farray\n" + + " TOK_LIST\n" + + " TOK_INT\n" + + " TOK_TABCOL\n" + + " fmap\n" + + " TOK_MAP\n" + + " TOK_INT\n" + + " TOK_INT\n" + + " TOK_TABCOL\n" + + " fstruct\n" + + " TOK_STRUCT\n" + + " TOK_TABCOLLIST\n" + + " TOK_TABCOL\n" + + " nf1\n" + + " TOK_INT\n" + + " TOK_TABCOL\n" + + " nf2\n" + + " TOK_CHAR\n" + + " 30\n"; } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprNodeDescExprFactory.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprNodeDescExprFactory.java new file mode 100644 index 000000000000..b97c2261ee03 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestExprNodeDescExprFactory.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.parse.type; + +import junit.framework.TestCase; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; + +import java.util.HashMap; + +import static java.util.Arrays.asList; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +public class TestExprNodeDescExprFactory extends TestCase { + + public void testToExprWhenColumnIsPrimitive() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("decimal(3,2)"); + DecimalTypeInfo typeInfo = new DecimalTypeInfo(3, 2); + columnInfo.setObjectinspector(PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + typeInfo, new HiveDecimalWritable(HiveDecimal.create(6.4)))); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("6.4")); + } + + public void testToExprWhenColumnIsPrimitiveNullValue() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("decimal(3,2)"); + DecimalTypeInfo typeInfo = new DecimalTypeInfo(3, 2); + columnInfo.setObjectinspector(PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + typeInfo, null)); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("null")); + } + + public void testToExprWhenColumnIsList() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("array"); + DecimalTypeInfo typeInfo = new DecimalTypeInfo(3, 2); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantListObjectInspector( + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(typeInfo), + asList( + new HiveDecimalWritable(HiveDecimal.create(5d)), + new HiveDecimalWritable(HiveDecimal.create(0.4)), + null))); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("Const array [5, 0.4, null]")); + } + + public void testToExprWhenColumnIsListWithNullValue() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("array"); + DecimalTypeInfo typeInfo = new DecimalTypeInfo(3, 2); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantListObjectInspector( + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(typeInfo), null)); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("Const array null")); + } + + public void testToExprWhenColumnIsMap() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("map"); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantMapObjectInspector( + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + PrimitiveObjectInspector.PrimitiveCategory.INT), + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + PrimitiveObjectInspector.PrimitiveCategory.STRING), + new HashMap() {{ put(new IntWritable(4), new Text("foo")); put(null, null); }})); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("Const map {null=null, 4=foo}")); + } + + public void testToExprWhenColumnIsMapWithNullValue() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("map"); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantMapObjectInspector( + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + PrimitiveObjectInspector.PrimitiveCategory.INT), + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + PrimitiveObjectInspector.PrimitiveCategory.STRING), + null)); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("Const map null")); + } + + public void testToExprWhenColumnIsStruct() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("struct"); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantStructObjectInspector( + asList("f1", "f2"), + asList(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.INT), + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING)), + asList(new IntWritable(4), new Text("foo")))); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("const struct(4,'foo')")); + } + + public void testToExprWhenColumnIsStructWithNullFields() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("struct"); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantStructObjectInspector( + asList("f1", "f2"), + asList(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.INT), + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING)), + asList(null, null))); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("const struct(null,null)")); + } + + public void testToExprWhenColumnIsStructWithNullValue() throws SemanticException { + ExprNodeDescExprFactory exprFactory = new ExprNodeDescExprFactory(); + ColumnInfo columnInfo = new ColumnInfo(); + columnInfo.setTypeName("struct"); + columnInfo.setObjectinspector(ObjectInspectorFactory.getStandardConstantStructObjectInspector( + asList("f1", "f2"), + asList(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.INT), + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.STRING)), + null)); + + ExprNodeDesc exprNodeDesc = exprFactory.toExpr(columnInfo, null, 0); + + assertThat(exprNodeDesc.getExprString(), is("null")); + } + +} diff --git a/ql/src/test/queries/clientpositive/cast_null_to_complex.q b/ql/src/test/queries/clientpositive/cast_null_to_complex.q new file mode 100644 index 000000000000..323e4bda0fc3 --- /dev/null +++ b/ql/src/test/queries/clientpositive/cast_null_to_complex.q @@ -0,0 +1,13 @@ +SET hive.cli.print.header=true; + +explain cbo +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>); +explain +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>); +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>); + + +create table t1 as +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>); + +describe formatted t1; diff --git a/ql/src/test/queries/clientpositive/empty_result_ctas.q b/ql/src/test/queries/clientpositive/empty_result_ctas.q index 0a1fc91c11aa..9437e0c1e6ab 100644 --- a/ql/src/test/queries/clientpositive/empty_result_ctas.q +++ b/ql/src/test/queries/clientpositive/empty_result_ctas.q @@ -3,3 +3,15 @@ SET hive.cli.print.header=true; CREATE TABLE T1 (c_primitive int, c_array array, c_nested array, f3:array>>); CREATE TABLE T2 AS SELECT * FROM T1 LIMIT 0; DESCRIBE FORMATTED t2; + +-- empty source table +CREATE TABLE T3 AS SELECT * FROM T1; +DESCRIBE FORMATTED t3; + +create table table1 (a string, b string); +create table table2 (complex_column array, `values`:array>>>>); + +-- empty result subquery +create table table3 as with t1 as (select * from table1), t2 as (select * from table2 where 1=0) select t1.*, t2.* from t1 left join t2; + +describe formatted table3; diff --git a/ql/src/test/results/clientpositive/llap/analyze_npe.q.out b/ql/src/test/results/clientpositive/llap/analyze_npe.q.out index df9097d44cc3..c63f23c4fb42 100644 --- a/ql/src/test/results/clientpositive/llap/analyze_npe.q.out +++ b/ql/src/test/results/clientpositive/llap/analyze_npe.q.out @@ -114,6 +114,7 @@ STAGE PLANS: Filter Operator predicate: c1 is null (type: boolean) Select Operator + expressions: null (type: struct) outputColumnNames: _col0 ListSink @@ -139,7 +140,7 @@ STAGE PLANS: Filter Operator predicate: c1 is null (type: boolean) Select Operator - expressions: null (type: void) + expressions: Const map null (type: map) outputColumnNames: _col0 ListSink @@ -165,7 +166,7 @@ STAGE PLANS: Filter Operator predicate: c1 is null (type: boolean) Select Operator - expressions: null (type: void) + expressions: Const array null (type: array) outputColumnNames: _col0 ListSink @@ -191,7 +192,7 @@ STAGE PLANS: Filter Operator predicate: c1 is null (type: boolean) Select Operator - expressions: null (type: void) + expressions: null (type: struct) outputColumnNames: _col0 ListSink diff --git a/ql/src/test/results/clientpositive/llap/cast_null_to_complex.q.out b/ql/src/test/results/clientpositive/llap/cast_null_to_complex.q.out new file mode 100644 index 000000000000..d6db733df7ad --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/cast_null_to_complex.q.out @@ -0,0 +1,107 @@ +PREHOOK: query: explain cbo +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: explain cbo +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +Explain +CBO PLAN: +HiveProject(_o__c0=[null:(INTEGER, VARCHAR(2147483647) CHARACTER SET "UTF-16LE") MAP], _o__c1=[null:(INTEGER, VARCHAR(2147483647) CHARACTER SET "UTF-16LE") MAP ARRAY], _o__c2=[null:INTEGER], _o__c3=[null:RecordType((INTEGER, VARCHAR(2147483647) CHARACTER SET "UTF-16LE") MAP ARRAY f1, RecordType(DOUBLE a, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" b) f2)]) + HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) + +PREHOOK: query: explain +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: explain +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +Explain +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: _dummy_table + Row Limit Per Split: 1 + Select Operator + expressions: Const map null (type: map), Const array> null (type: array>), null (type: int), null (type: struct>,f2:struct>) + outputColumnNames: _col0, _col1, _col2, _col3 + ListSink + +PREHOOK: query: select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +_c0 _c1 _c2 _c3 +NULL NULL NULL NULL +PREHOOK: query: create table t1 as +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1 as +select cast(null as map), cast(null as array>), cast(null as int), cast(null as struct>, f2:struct>) +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t1._c0 SIMPLE [] +POSTHOOK: Lineage: t1._c1 SIMPLE [] +POSTHOOK: Lineage: t1._c2 SIMPLE [] +POSTHOOK: Lineage: t1._c3 SIMPLE [] +_c0 _c1 _c2 _c3 +PREHOOK: query: describe formatted t1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@t1 +POSTHOOK: query: describe formatted t1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@t1 +col_name data_type comment +# col_name data_type comment +_c0 map +_c1 array> +_c2 int +_c3 struct>,f2:struct> + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + bucketing_version 2 + numFiles 1 + numRows 1 + rawDataSize 11 + totalSize 12 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/ql/src/test/results/clientpositive/llap/empty_result_ctas.q.out b/ql/src/test/results/clientpositive/llap/empty_result_ctas.q.out index 57cc1b7e8221..27eba3c3b628 100644 --- a/ql/src/test/results/clientpositive/llap/empty_result_ctas.q.out +++ b/ql/src/test/results/clientpositive/llap/empty_result_ctas.q.out @@ -18,8 +18,8 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Input: default@t1 POSTHOOK: Output: database:default POSTHOOK: Output: default@T2 -POSTHOOK: Lineage: t2.c_array EXPRESSION [] -POSTHOOK: Lineage: t2.c_nested EXPRESSION [] +POSTHOOK: Lineage: t2.c_array SIMPLE [] +POSTHOOK: Lineage: t2.c_nested SIMPLE [] POSTHOOK: Lineage: t2.c_primitive SIMPLE [] t1.c_primitive t1.c_array t1.c_nested PREHOOK: query: DESCRIBE FORMATTED t2 @@ -49,6 +49,126 @@ Table Parameters: totalSize 0 #### A masked pattern was here #### +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: CREATE TABLE T3 AS SELECT * FROM T1 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@t1 +PREHOOK: Output: database:default +PREHOOK: Output: default@T3 +POSTHOOK: query: CREATE TABLE T3 AS SELECT * FROM T1 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@t1 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T3 +POSTHOOK: Lineage: t3.c_array SIMPLE [(t1)t1.FieldSchema(name:c_array, type:array, comment:null), ] +POSTHOOK: Lineage: t3.c_nested SIMPLE [(t1)t1.FieldSchema(name:c_nested, type:array,f3:array>>, comment:null), ] +POSTHOOK: Lineage: t3.c_primitive SIMPLE [(t1)t1.FieldSchema(name:c_primitive, type:int, comment:null), ] +t1.c_primitive t1.c_array t1.c_nested +PREHOOK: query: DESCRIBE FORMATTED t3 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@t3 +POSTHOOK: query: DESCRIBE FORMATTED t3 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@t3 +col_name data_type comment +# col_name data_type comment +c_primitive int +c_array array +c_nested array,f3:array>> + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + bucketing_version 2 + numFiles 0 + numRows 0 + rawDataSize 0 + totalSize 0 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: create table table1 (a string, b string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table1 +POSTHOOK: query: create table table1 (a string, b string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table1 +PREHOOK: query: create table table2 (complex_column array, `values`:array>>>>) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table2 +POSTHOOK: query: create table table2 (complex_column array, `values`:array>>>>) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table2 +PREHOOK: query: create table table3 as with t1 as (select * from table1), t2 as (select * from table2 where 1=0) select t1.*, t2.* from t1 left join t2 +PREHOOK: type: CREATETABLE_AS_SELECT +PREHOOK: Input: default@table1 +PREHOOK: Input: default@table2 +PREHOOK: Output: database:default +PREHOOK: Output: default@table3 +POSTHOOK: query: create table table3 as with t1 as (select * from table1), t2 as (select * from table2 where 1=0) select t1.*, t2.* from t1 left join t2 +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@table1 +POSTHOOK: Input: default@table2 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table3 +POSTHOOK: Lineage: table3.a SIMPLE [(table1)table1.FieldSchema(name:a, type:string, comment:null), ] +POSTHOOK: Lineage: table3.b SIMPLE [(table1)table1.FieldSchema(name:b, type:string, comment:null), ] +POSTHOOK: Lineage: table3.complex_column SIMPLE [] +t1.a t1.b t2.complex_column +PREHOOK: query: describe formatted table3 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@table3 +POSTHOOK: query: describe formatted table3 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@table3 +col_name data_type comment +# col_name data_type comment +a string +b string +complex_column array,values:array>>>> + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + bucketing_version 2 + numFiles 0 + numRows 0 + rawDataSize 0 + totalSize 0 +#### A masked pattern was here #### + # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat: org.apache.hadoop.mapred.TextInputFormat diff --git a/ql/src/test/results/clientpositive/llap/show_functions.q.out b/ql/src/test/results/clientpositive/llap/show_functions.q.out index 3eab5bdb7ccc..e5a249647112 100644 --- a/ql/src/test/results/clientpositive/llap/show_functions.q.out +++ b/ql/src/test/results/clientpositive/llap/show_functions.q.out @@ -454,6 +454,9 @@ to_date to_epoch_milli to_unix_timestamp to_utc_timestamp +toarray +tomap +tostruct translate trim trunc @@ -1081,6 +1084,9 @@ to_date to_epoch_milli to_unix_timestamp to_utc_timestamp +toarray +tomap +tostruct translate trim trunc From 81c89cdb7b01617dc1ab847a52805c16c9a5621a Mon Sep 17 00:00:00 2001 From: veghlaci05 Date: Mon, 18 Dec 2023 12:59:43 +0100 Subject: [PATCH 102/179] HIVE-27481: TxnHandler cleanup (Laszlo Vegh, reviewed by Denys Kuzmenko, Krisztian Kasa, Zoltan Ratkai, Laszlo Bodor) --- .../compaction/IcebergCompactionService.java | 2 +- .../TestReplicationScenariosAcidTables.java | 2 +- .../hive/ql/txn/compactor/TestCompactor.java | 5 +- .../txn/compactor/TestCrudCompactorOnTez.java | 2 +- .../apache/hadoop/hive/ql/io/AcidUtils.java | 2 +- .../ql/txn/compactor/CompactorContext.java | 2 +- .../ql/txn/compactor/CompactorFactory.java | 4 +- .../ql/txn/compactor/CompactorThread.java | 2 +- .../hive/ql/txn/compactor/CompactorUtil.java | 2 +- .../hive/ql/txn/compactor/Initiator.java | 2 +- .../hive/ql/txn/compactor/InitiatorBase.java | 2 +- .../hive/ql/txn/compactor/MRCompactor.java | 2 +- .../hive/ql/txn/compactor/MergeCompactor.java | 2 +- .../compactor/MetaStoreCompactorThread.java | 2 +- .../hive/ql/txn/compactor/QueryCompactor.java | 2 +- .../txn/compactor/RemoteCompactorThread.java | 2 +- .../ql/txn/compactor/RemoteCompactorUtil.java | 2 +- .../hive/ql/txn/compactor/StatsUpdater.java | 3 +- .../hadoop/hive/ql/txn/compactor/Worker.java | 8 +- .../compactor/handler/AbortedTxnCleaner.java | 2 +- .../compactor/handler/CompactionCleaner.java | 2 +- .../ql/txn/compactor/handler/TaskHandler.java | 3 +- .../service/AcidCompactionService.java | 4 +- .../compactor/service/CompactionService.java | 2 +- .../txn/TestCompactionTxnHandler.java | 7 +- .../hive/metastore/txn/TestTxnHandler.java | 129 +- .../hadoop/hive/ql/TestTxnCommands3.java | 2 +- .../hive/ql/TxnCommandsBaseForTests.java | 2 +- .../hive/ql/txn/compactor/CompactorTest.java | 2 +- .../hive/ql/txn/compactor/TestCleaner.java | 2 +- .../txn/compactor/TestCompactionMetrics.java | 4 +- .../TestMRCompactorJobQueueConfiguration.java | 2 +- .../handler/TestAbortedTxnCleaner.java | 2 +- .../hive/metastore/conf/MetastoreConf.java | 2 +- .../hadoop/hive/metastore/txn/TxnQueries.java | 88 - .../hive/metastore/DatabaseProduct.java | 25 +- .../hadoop/hive/metastore/HMSHandler.java | 1 + .../events/CommitCompactionEvent.java | 3 +- .../metastore/metrics/AcidMetricLogger.java | 10 +- .../metastore/metrics/AcidMetricService.java | 10 +- .../txn/CompactionMetricsDataConverter.java | 1 + .../metastore/txn/CompactionTxnHandler.java | 139 +- .../metastore/txn/DefaultTxnLockManager.java | 126 + ...dler.java => TransactionalRetryProxy.java} | 91 +- .../hadoop/hive/metastore/txn/TxnHandler.java | 6733 ++--------------- .../hive/metastore/txn/TxnLockManager.java | 85 + .../hadoop/hive/metastore/txn/TxnStore.java | 116 +- .../hive/metastore/txn/TxnStoreMutex.java | 209 + .../hadoop/hive/metastore/txn/TxnUtils.java | 112 +- .../txn/{ => entities}/CompactionInfo.java | 5 +- .../{ => entities}/CompactionMetricsData.java | 2 +- .../txn/{ => entities}/CompactionState.java | 22 +- .../hive/metastore/txn/entities/LockInfo.java | 147 + .../txn/{ => entities}/MetricsInfo.java | 2 +- .../metastore/txn/{ => entities}/OpenTxn.java | 8 +- .../txn/{ => entities}/OpenTxnList.java | 11 +- .../txn/{ => entities}/OperationType.java | 2 +- .../txn/{ => entities}/TxnStatus.java | 2 +- .../txn/jdbc/ConditionalCommand.java | 43 + .../txn/jdbc/InClauseBatchCommand.java | 75 + .../txn/jdbc/MultiDataSourceJdbcResource.java | 180 +- .../txn/jdbc/NoPoolConnectionPool.java | 145 + .../txn/jdbc/ParameterizedBatchCommand.java | 61 + .../metastore/txn/jdbc/RollbackException.java | 40 + .../txn/jdbc/TransactionContext.java | 54 +- .../txn/jdbc/TransactionContextManager.java | 58 +- .../txn/jdbc/TransactionalFunction.java | 8 +- .../AddWriteIdsToMinHistoryCommand.java | 87 + .../AddWriteIdsToTxnToWriteIdCommand.java | 67 + .../DeleteInvalidOpenTxnsCommand.java | 33 + .../DeleteReplTxnMapEntryCommand.java | 54 + .../InsertCompactionInfoCommand.java | 6 +- .../InsertCompactionRequestCommand.java | 93 + .../InsertCompletedTxnComponentsCommand.java | 70 + .../jdbc/commands/InsertHiveLocksCommand.java | 97 + .../commands/InsertTxnComponentsCommand.java | 185 + .../RemoveCompactionMetricsDataCommand.java | 4 +- ...DuplicateCompleteTxnComponentsCommand.java | 2 +- .../RemoveTxnsFromMinHistoryLevelCommand.java | 47 + .../RemoveWriteIdsFromMinHistoryCommand.java | 49 + .../functions/AbortCompactionFunction.java | 188 + .../txn/jdbc/functions/AbortTxnFunction.java | 105 + .../txn/jdbc/functions/AbortTxnsFunction.java | 209 + .../functions/AcquireTxnLockFunction.java | 49 + .../AllocateTableWriteIdsFunction.java | 348 + .../txn/jdbc/functions/CheckLockFunction.java | 342 + .../CleanTxnToWriteIdTableFunction.java | 13 +- .../functions/CleanupRecordsFunction.java | 186 + .../txn/jdbc/functions/CommitTxnFunction.java | 636 ++ .../txn/jdbc/functions/CompactFunction.java | 123 + .../jdbc/functions/EnqueueLockFunction.java | 118 + .../functions/EnsureValidTxnFunction.java | 70 + .../FindPotentialCompactionsFunction.java | 8 +- .../GenerateCompactionQueueIdFunction.java | 65 + ...terializationInvalidationInfoFunction.java | 180 + .../GetValidWriteIdsForTableFunction.java | 155 + .../functions/GetValidWriteIdsFunction.java | 103 + .../jdbc/functions/HeartbeatLockFunction.java | 61 + .../jdbc/functions/HeartbeatTxnFunction.java | 73 + .../functions/HeartbeatTxnRangeFunction.java | 104 + .../LockMaterializationRebuildFunction.java | 92 + .../functions}/MarkCleanedFunction.java | 34 +- .../MinOpenTxnIdWaterMarkFunction.java | 67 + .../functions}/NextCompactionFunction.java | 4 +- .../txn/jdbc/functions/OnRenameFunction.java | 159 + .../txn/jdbc/functions/OpenTxnsFunction.java | 233 + .../functions/PerformTimeoutsFunction.java | 203 + .../PurgeCompactionHistoryFunction.java | 25 +- .../ReleaseMaterializationRebuildLocks.java | 78 + .../ReplTableWriteIdStateFunction.java | 158 + ...pCompactionMetricsDataPerTypeFunction.java | 11 +- .../functions/UpdataDatabasePropFunction.java | 72 + .../UpdateCompactionMetricsDataFunction.java | 6 +- .../txn/jdbc/queries/AbortTxnInfoHandler.java | 131 + .../queries}/AbortedTxnHandler.java | 6 +- .../CheckFailedCompactionsHandler.java | 4 +- .../queries}/CompactionCandidateHandler.java | 4 +- .../CompactionMetricsDataHandler.java | 4 +- .../jdbc/queries/CountOpenTxnsHandler.java | 63 + .../txn/jdbc/queries/DbTimeHandler.java | 51 + .../queries}/FindColumnsWithStatsHandler.java | 4 +- .../txn/jdbc/queries/FindTxnStateHandler.java | 63 + .../queries}/GetCompactionInfoHandler.java | 8 +- .../jdbc/queries/GetDatabaseIdHandler.java | 59 + .../jdbc/queries/GetHighWaterMarkHandler.java | 55 + ...tLatestCommittedCompactionInfoHandler.java | 100 + .../txn/jdbc/queries/GetLocksByLockId.java | 81 + .../GetMaxAllocatedTableWriteIdHandler.java | 61 + .../queries/GetOpenTxnTypeAndLockHandler.java | 71 + .../jdbc/queries/GetOpenTxnsListHandler.java | 126 + .../jdbc/queries/GetTxnDbsUpdatedHandler.java | 62 + .../txn/jdbc/queries/GetWriteIdsHandler.java | 81 + .../queries/LatestTxnIdInConflictHandler.java | 79 + .../txn/jdbc/queries/MetricsInfoHandler.java | 93 + .../queries}/MinUncommittedTxnIdHandler.java | 4 +- ...OpenTxnTimeoutLowBoundaryTxnIdHandler.java | 64 + .../queries}/ReadyToCleanAbortHandler.java | 6 +- .../queries}/ReadyToCleanHandler.java | 15 +- .../txn/jdbc/queries/ShowCompactHandler.java | 185 + .../txn/jdbc/queries/ShowLocksHandler.java | 179 + .../queries/TablesWithAbortedTxnsHandler.java | 71 + .../jdbc/queries/TargetTxnIdListHandler.java | 64 + .../jdbc/queries/TxnIdForWriteIdHandler.java | 64 + .../RetryPropagation.java | 2 +- .../{retryhandling => retry}/SqlRetry.java | 2 +- .../SqlRetryCallProperties.java | 2 +- .../SqlRetryException.java} | 30 +- .../SqlRetryFunction.java | 5 +- .../SqlRetryHandler.java | 36 +- .../metastore/utils/StackThreadLocal.java | 57 + .../hive/metastore/DummyCustomRDBMS.java | 2 +- .../txn/retry/TestSqlRetryHandler.java | 43 + .../apache/hive/streaming/TestStreaming.java | 12 +- 153 files changed, 9220 insertions(+), 6732 deletions(-) delete mode 100644 standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ProxyTxnHandler.java => TransactionalRetryProxy.java} (61%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/CompactionInfo.java (98%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/CompactionMetricsData.java (98%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/CompactionState.java (81%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/MetricsInfo.java (98%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/OpenTxn.java (84%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/OpenTxnList.java (89%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/OperationType.java (97%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => entities}/TxnStatus.java (97%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ConditionalCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/InClauseBatchCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/NoPoolConnectionPool.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ParameterizedBatchCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/RollbackException.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToMinHistoryCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToTxnToWriteIdCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteInvalidOpenTxnsCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteReplTxnMapEntryCommand.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/commands}/InsertCompactionInfoCommand.java (96%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionRequestCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompletedTxnComponentsCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertHiveLocksCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertTxnComponentsCommand.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/commands}/RemoveCompactionMetricsDataCommand.java (94%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/commands}/RemoveDuplicateCompleteTxnComponentsCommand.java (98%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveTxnsFromMinHistoryLevelCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveWriteIdsFromMinHistoryCommand.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortCompactionFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnsFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AcquireTxnLockFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AllocateTableWriteIdsFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CheckLockFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/CleanTxnToWriteIdTableFunction.java (90%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CleanupRecordsFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CommitTxnFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CompactFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnqueueLockFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnsureValidTxnFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/FindPotentialCompactionsFunction.java (87%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GenerateCompactionQueueIdFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetMaterializationInvalidationInfoFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsForTableFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatLockFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnRangeFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/LockMaterializationRebuildFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/MarkCleanedFunction.java (90%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MinOpenTxnIdWaterMarkFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/NextCompactionFunction.java (97%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OpenTxnsFunction.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PerformTimeoutsFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/PurgeCompactionHistoryFunction.java (93%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReplTableWriteIdStateFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/TopCompactionMetricsDataPerTypeFunction.java (87%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdataDatabasePropFunction.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/functions}/UpdateCompactionMetricsDataFunction.java (92%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortTxnInfoHandler.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/AbortedTxnHandler.java (94%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/CheckFailedCompactionsHandler.java (96%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/CompactionCandidateHandler.java (96%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/CompactionMetricsDataHandler.java (95%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CountOpenTxnsHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/DbTimeHandler.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/FindColumnsWithStatsHandler.java (95%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindTxnStateHandler.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/GetCompactionInfoHandler.java (96%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetDatabaseIdHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetHighWaterMarkHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLatestCommittedCompactionInfoHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLocksByLockId.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetMaxAllocatedTableWriteIdHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnTypeAndLockHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnsListHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetTxnDbsUpdatedHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetWriteIdsHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MetricsInfoHandler.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/MinUncommittedTxnIdHandler.java (96%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/OpenTxnTimeoutLowBoundaryTxnIdHandler.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/ReadyToCleanAbortHandler.java (97%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{impl => jdbc/queries}/ReadyToCleanHandler.java (92%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowCompactHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowLocksHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TablesWithAbortedTxnsHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TargetTxnIdListHandler.java create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TxnIdForWriteIdHandler.java rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{retryhandling => retry}/RetryPropagation.java (97%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{retryhandling => retry}/SqlRetry.java (96%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{retryhandling => retry}/SqlRetryCallProperties.java (98%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ContextNode.java => retry/SqlRetryException.java} (58%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{retryhandling => retry}/SqlRetryFunction.java (89%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{retryhandling => retry}/SqlRetryHandler.java (91%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/StackThreadLocal.java create mode 100644 standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/retry/TestSqlRetryHandler.java diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java index e0ff9782c690..5c985a55e571 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java @@ -19,7 +19,7 @@ package org.apache.iceberg.mr.hive.compaction; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext; import org.apache.hadoop.hive.ql.txn.compactor.CompactorPipeline; import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java index 93fd5f0cbc9f..63ba2aad1888 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java @@ -40,7 +40,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore; diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java index fd6e111df0ac..1083bef80e9b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; @@ -52,9 +52,6 @@ import org.apache.hadoop.hive.ql.io.orc.Reader; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.txn.compactor.Cleaner; -import org.apache.hadoop.hive.ql.txn.compactor.FSRemover; -import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache; import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandler; import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandlerFactory; import org.apache.hive.streaming.HiveStreamingConnection; diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java index 295bbdd09f79..1342b25916b4 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java @@ -50,7 +50,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.Driver; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java index febe8a2b1e90..4c63a74d853e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java @@ -92,7 +92,7 @@ import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.txn.CompactionState; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionState; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java index dece9ed939c1..7d16d64db50d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.ql.io.AcidDirectory; /** diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java index 2184bcd5072c..84ce492ae610 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.AcidUtils; @@ -70,7 +70,7 @@ private CompactorFactory() { * @param compactionInfo provides insight about the type of compaction, must be not null. * @return {@link QueryCompactor} or null. */ - public CompactorPipeline getCompactorPipeline(Table table, HiveConf configuration, CompactionInfo compactionInfo,IMetaStoreClient msc) + public CompactorPipeline getCompactorPipeline(Table table, HiveConf configuration, CompactionInfo compactionInfo, IMetaStoreClient msc) throws HiveException { if (AcidUtils.isFullAcidTable(table.getParameters())) { if (!"tez".equalsIgnoreCase(HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE)) || diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java index dda8b219f908..09296293f0ac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.thrift.TException; import org.slf4j.Logger; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java index 6daf665dd369..c8d86c7ea276 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.metastore.utils.StringableMap; import org.apache.hadoop.hive.ql.io.AcidDirectory; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java index 6396aa8c30d8..58cb478bbe82 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.metrics.PerfLogger; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java index 369dc69fc309..8f632dbd398c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.io.AcidDirectory; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java index f9044fa3d296..5bf1c0bffb14 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java @@ -47,7 +47,7 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.StringableMap; import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; import org.apache.hadoop.hive.ql.io.AcidInputFormat; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java index 295ebe564178..7e5a3608e462 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.ql.io.AcidDirectory; import org.apache.hadoop.hive.ql.io.AcidOutputFormat; import org.apache.hadoop.hive.ql.io.AcidUtils; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java index bfd59c6fdc71..0878330cc3e7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.metrics.Metrics; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java index ee0d9c9a7d74..384a386dfb6b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.StringableMap; import org.apache.hadoop.hive.ql.DriverUtils; import org.apache.hadoop.hive.ql.io.AcidDirectory; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java index 819e39579c51..f95834ac23d6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; import org.apache.thrift.TException; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java index 676a11891a0d..c7f20a93605a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java @@ -23,7 +23,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java index 50d04f8b1b29..ce0998158dc3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java @@ -21,8 +21,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.CompactionType; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.ql.DriverUtils; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.stats.StatsUtils; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java index 8f75c06d54b7..b39236ef8b9a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java @@ -28,6 +28,10 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; +import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg; +import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus; +import org.apache.hadoop.hive.ql.io.AcidDirectory; +import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionService; import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionExecutorFactory; @@ -36,7 +40,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.session.SessionState; import java.io.IOException; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java index acd4519d0fe3..2314ce4d2e4b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.metrics.PerfLogger; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java index c2be268b6f9a..721e3ea2266e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.metrics.PerfLogger; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java index d530b7efa34c..f4d0a5adc15f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.ql.io.AcidDirectory; @@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java index 4fb6775eda53..b4143f9dd184 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java @@ -36,10 +36,10 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg; -import org.apache.hadoop.hive.metastore.txn.TxnStatus; import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.io.AcidDirectory; import org.apache.hadoop.hive.ql.io.AcidUtils; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java index c896ffc77cf1..814ed9cd046b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.ql.txn.compactor.CompactorFactory; import org.apache.thrift.TException; import org.slf4j.Logger; diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java index 38c44f7722fe..d26f3774af70 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hive.common.util.HiveVersionInfo; import org.junit.After; @@ -463,7 +464,7 @@ public void testMarkFailed() throws Exception { txnHandler.markFailed(ci); fail("The first call to markFailed() must have failed as this call did " + "not throw the expected exception"); - } catch (MetaException e) { + } catch (IllegalStateException e) { // This is expected assertTrue(e.getMessage().contains("No record with CQ_ID=")); } @@ -753,7 +754,7 @@ public void testFindPotentialCompactions() throws Exception { LockResponse res = txnHandler.lock(req); assertTrue(res.getState() == LockState.ACQUIRED); txnHandler.commitTxn(new CommitTxnRequest(txnid)); - assertEquals(0, txnHandler.numLocksInLockTable()); + assertEquals(0, txnHandler.getNumLocks()); Set potentials = txnHandler.findPotentialCompactions(100, -1L); assertEquals(2, potentials.size()); @@ -1014,7 +1015,7 @@ public void testFindPotentialCompactions_limitFetchSize() throws Exception { LockResponse res = txnHandler.lock(req); assertSame(res.getState(), LockState.ACQUIRED); txnHandler.commitTxn(new CommitTxnRequest(txnId)); - assertEquals(0, txnHandler.numLocksInLockTable()); + assertEquals(0, txnHandler.getNumLocks()); Set potentials = txnHandler.findPotentialCompactions(100, -1L); assertEquals(1, potentials.size()); diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java index ee810c2117e5..4f9f1d8e2fcb 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java @@ -95,7 +95,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -107,7 +106,6 @@ import static junit.framework.Assert.fail; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.TABLE_IS_TRANSACTIONAL; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES; -import static org.apache.hadoop.hive.metastore.utils.LockTypeUtil.getEncoding; /** * Tests for TxnHandler. @@ -1033,7 +1031,7 @@ public void testMultipleLock() throws Exception { res = txnHandler.checkLock(new CheckLockRequest(lockid)); assertTrue(res.getState() == LockState.ACQUIRED); txnHandler.unlock(new UnlockRequest(lockid)); - assertEquals(0, txnHandler.numLocksInLockTable()); + assertEquals(0, txnHandler.getNumLocks()); } @Test @@ -1088,7 +1086,7 @@ public void testUnlockOnCommit() throws Exception { LockResponse res = txnHandler.lock(req); assertTrue(res.getState() == LockState.ACQUIRED); txnHandler.commitTxn(new CommitTxnRequest(txnid)); - assertEquals(0, txnHandler.numLocksInLockTable()); + assertEquals(0, txnHandler.getNumLocks()); } @Test @@ -1104,7 +1102,7 @@ public void testUnlockOnAbort() throws Exception { LockResponse res = txnHandler.lock(req); assertTrue(res.getState() == LockState.ACQUIRED); txnHandler.abortTxn(new AbortTxnRequest(txnid)); - assertEquals(0, txnHandler.numLocksInLockTable()); + assertEquals(0, txnHandler.getNumLocks()); } @Test @@ -1462,113 +1460,6 @@ public void showLocks() throws Exception { for (int i = 0; i < saw.length; i++) assertTrue("Didn't see lock id " + i, saw[i]); } - @Test - @Ignore("Wedges Derby") - public void deadlockDetected() throws Exception { - LOG.debug("Starting deadlock test"); - - if (txnHandler instanceof TxnHandler) { - final TxnHandler tHndlr = (TxnHandler)txnHandler; - Connection conn = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE); - try { - Statement stmt = conn.createStatement(); - long now = tHndlr.getDbTime(conn); - stmt.executeUpdate("INSERT INTO \"TXNS\" (\"TXN_ID\", \"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", " + - "txn_user, txn_host) values (1, 'o', " + now + ", " + now + ", 'shagy', " + - "'scooby.com')"); - stmt.executeUpdate("INSERT INTO \"HIVE_LOCKS\" (\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", " + - "\"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", " + - "\"HL_USER\", \"HL_HOST\") VALUES (1, 1, 1, 'MYDB', 'MYTABLE', 'MYPARTITION', '" + - tHndlr.LOCK_WAITING + "', '" + getEncoding(LockType.EXCLUSIVE) + "', " + now + ", 'fred', " + - "'scooby.com')"); - conn.commit(); - } finally { - tHndlr.closeDbConn(conn); - } - - final AtomicBoolean sawDeadlock = new AtomicBoolean(); - - final Connection conn1 = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE); - final Connection conn2 = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE); - try { - - for (int i = 0; i < 5; i++) { - Thread t1 = new Thread() { - @Override - public void run() { - try { - try { - updateTxns(conn1); - updateLocks(conn1); - Thread.sleep(1000); - conn1.commit(); - LOG.debug("no exception, no deadlock"); - } catch (SQLException e) { - try { - tHndlr.checkRetryable(e, "thread t1"); - LOG.debug("Got an exception, but not a deadlock, SQLState is " + - e.getSQLState() + " class of exception is " + e.getClass().getName() + - " msg is <" + e.getMessage() + ">"); - } catch (TxnHandler.RetryException de) { - LOG.debug("Forced a deadlock, SQLState is " + e.getSQLState() + " class of " + - "exception is " + e.getClass().getName() + " msg is <" + e - .getMessage() + ">"); - sawDeadlock.set(true); - } - } - conn1.rollback(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - }; - - Thread t2 = new Thread() { - @Override - public void run() { - try { - try { - updateLocks(conn2); - updateTxns(conn2); - Thread.sleep(1000); - conn2.commit(); - LOG.debug("no exception, no deadlock"); - } catch (SQLException e) { - try { - tHndlr.checkRetryable(e, "thread t2"); - LOG.debug("Got an exception, but not a deadlock, SQLState is " + - e.getSQLState() + " class of exception is " + e.getClass().getName() + - " msg is <" + e.getMessage() + ">"); - } catch (TxnHandler.RetryException de) { - LOG.debug("Forced a deadlock, SQLState is " + e.getSQLState() + " class of " + - "exception is " + e.getClass().getName() + " msg is <" + e - .getMessage() + ">"); - sawDeadlock.set(true); - } - } - conn2.rollback(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - }; - - t1.start(); - t2.start(); - t1.join(); - t2.join(); - if (sawDeadlock.get()) break; - } - assertTrue(sawDeadlock.get()); - } finally { - conn1.rollback(); - tHndlr.closeDbConn(conn1); - conn2.rollback(); - tHndlr.closeDbConn(conn2); - } - } - } - /** * This cannnot be run against Derby (thus in UT) but it can run against MySQL. * 1. add to metastore/pom.xml @@ -1662,20 +1553,6 @@ public void uncaughtException(Thread t, Throwable e) { } } - @Test - public void testRetryableRegex() throws Exception { - SQLException sqlException = new SQLException("ORA-08177: can't serialize access for this transaction", "72000"); - // Note that we have 3 regex'es below - conf.setVar(HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX, "^Deadlock detected, roll back,.*08177.*,.*08178.*"); - boolean result = TxnHandler.isRetryable(conf, sqlException); - Assert.assertTrue("regex should be retryable", result); - - sqlException = new SQLException("This error message, has comma in it"); - conf.setVar(HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX, ".*comma.*"); - result = TxnHandler.isRetryable(conf, sqlException); - Assert.assertTrue("regex should be retryable", result); - } - private List replOpenTxnForTest(long startId, int numTxn, String replPolicy) throws Exception { conf.setIntVar(HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH, numTxn); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java index 741078028918..c5a2639bdafc 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java index 03c4d4f23da3..3a9b0cb754cd 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java @@ -134,8 +134,8 @@ void setUpInternal() throws Exception { MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true); TestTxnDbUtil.setConfValues(hiveConf); - txnHandler = TxnUtils.getTxnStore(hiveConf); TestTxnDbUtil.prepDb(hiveConf); + txnHandler = TxnUtils.getTxnStore(hiveConf); File f = new File(getWarehouseDir()); if (f.exists()) { FileUtil.fullyDelete(f); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java index ba90d8549d19..38484534b772 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java @@ -61,7 +61,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java index 7eb573dedace..d435a07c2dfb 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java index 24c625139ece..b33f8917c74f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java @@ -51,7 +51,8 @@ import org.apache.hadoop.hive.metastore.metrics.AcidMetricService; import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.TxnHandler; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.ThrowingTxnHandler; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; @@ -665,6 +666,7 @@ public void testDBMetrics() throws Exception { Table t = newTable(dbName, tblName, false); MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_LEVEL, false); + TxnHandler.ConfVars.setUseMinHistoryLevel(false); long start = System.currentTimeMillis(); burnThroughTransactions(t.getDbName(), t.getTableName(), 24, new HashSet<>(Arrays.asList(22L, 23L, 24L)), null); openTxn(TxnType.REPL_CREATED); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java index 8a709b5a168a..e775b4567155 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.StringableMap; import org.apache.hadoop.mapred.JobConf; import org.junit.jupiter.params.ParameterizedTest; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index e0a2a1f5ed32..8f6814d4890f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.io.AcidUtils; diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index a37501a8d116..ffa3d6a06c54 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -1628,7 +1628,7 @@ public enum ConfVars { "Time before an open transaction operation should persist, otherwise it is considered invalid and rolled back"), TXN_USE_MIN_HISTORY_LEVEL("metastore.txn.use.minhistorylevel", "hive.txn.use.minhistorylevel", true, "Set this to false, for the TxnHandler and Cleaner to not use MIN_HISTORY_LEVEL table and take advantage of openTxn optimisation.\n" - + "If the table is dropped HMS will switch this flag to false."), + + "If the table is dropped HMS will switch this flag to false, any other value changes need a restart to take effect."), TXN_USE_MIN_HISTORY_WRITE_ID("metastore.txn.use.minhistorywriteid", "hive.txn.use.minhistorywriteid", false, "Set this to true, to avoid global minOpenTxn check in Cleaner.\n" + "If the table is dropped HMS will switch this flag to false."), diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java deleted file mode 100644 index f3f0e5d939b9..000000000000 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.metastore.txn; - -public class TxnQueries { - public static final String SHOW_COMPACTION_ORDERBY_CLAUSE = - " ORDER BY CASE " + - " WHEN \"CC_END\" > \"CC_START\" and \"CC_END\" > \"CC_COMMIT_TIME\" " + - " THEN \"CC_END\" " + - " WHEN \"CC_START\" > \"CC_COMMIT_TIME\" " + - " THEN \"CC_START\" " + - " ELSE \"CC_COMMIT_TIME\" " + - " END desc ," + - " \"CC_ENQUEUE_TIME\" asc"; - - public static final String SHOW_COMPACTION_QUERY = - "SELECT XX.* FROM ( SELECT " + - " \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\", " + - " \"CQ_STATE\" AS \"CC_STATE\", \"CQ_TYPE\" AS \"CC_TYPE\", \"CQ_WORKER_ID\" AS \"CC_WORKER_ID\", " + - " \"CQ_START\" AS \"CC_START\", -1 \"CC_END\", \"CQ_RUN_AS\" AS \"CC_RUN_AS\", " + - " \"CQ_HADOOP_JOB_ID\" AS \"CC_HADOOP_JOB_ID\", \"CQ_ID\" AS \"CC_ID\", \"CQ_ERROR_MESSAGE\" AS \"CC_ERROR_MESSAGE\", " + - " \"CQ_ENQUEUE_TIME\" AS \"CC_ENQUEUE_TIME\", \"CQ_WORKER_VERSION\" AS \"CC_WORKER_VERSION\", " + - " \"CQ_INITIATOR_ID\" AS \"CC_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" AS \"CC_INITIATOR_VERSION\", " + - " \"CQ_CLEANER_START\" AS \"CC_CLEANER_START\", \"CQ_POOL_NAME\" AS \"CC_POOL_NAME\", \"CQ_TXN_ID\" AS \"CC_TXN_ID\", " + - " \"CQ_NEXT_TXN_ID\" AS \"CC_NEXT_TXN_ID\", \"CQ_COMMIT_TIME\" AS \"CC_COMMIT_TIME\", " + - " \"CQ_HIGHEST_WRITE_ID\" AS \"CC_HIGHEST_WRITE_ID\" " + - "FROM " + - " \"COMPACTION_QUEUE\" " + - "UNION ALL " + - "SELECT " + - " \"CC_DATABASE\" , \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", \"CC_WORKER_ID\", " + - " \"CC_START\", \"CC_END\", \"CC_RUN_AS\", \"CC_HADOOP_JOB_ID\", \"CC_ID\", \"CC_ERROR_MESSAGE\", " + - " \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\", " + - " -1 , \"CC_POOL_NAME\", \"CC_TXN_ID\", \"CC_NEXT_TXN_ID\", \"CC_COMMIT_TIME\", " + - " \"CC_HIGHEST_WRITE_ID\"" + - "FROM " + - " \"COMPLETED_COMPACTIONS\" ) XX "; - - - public static final String SELECT_COMPACTION_QUEUE_BY_COMPID = - "SELECT XX.* FROM ( SELECT " + - " \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\", " + - " \"CQ_STATE\" AS \"CC_STATE\", \"CQ_TYPE\" AS \"CC_TYPE\", \"CQ_TBLPROPERTIES\" AS \"CC_TBLPROPERTIES\", \"CQ_WORKER_ID\" AS \"CC_WORKER_ID\", " + - " \"CQ_START\" AS \"CC_START\", \"CQ_RUN_AS\" AS \"CC_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" AS \"CC_HIGHEST_WRITE_ID\", \"CQ_META_INFO\" AS \"CC_META_INFO\"," + - " \"CQ_HADOOP_JOB_ID\" AS \"CC_HADOOP_JOB_ID\", \"CQ_ERROR_MESSAGE\" AS \"CC_ERROR_MESSAGE\", \"CQ_ENQUEUE_TIME\" AS \"CC_ENQUEUE_TIME\"," + - " \"CQ_WORKER_VERSION\" AS \"CC_WORKER_VERSION\", \"CQ_INITIATOR_ID\" AS \"CC_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" AS \"CC_INITIATOR_VERSION\", " + - " \"CQ_RETRY_RETENTION\" AS \"CC_RETRY_RETENTION\", \"CQ_NEXT_TXN_ID\" AS \"CC_NEXT_TXN_ID\", \"CQ_TXN_ID\" AS \"CC_TXN_ID\", " + - " \"CQ_COMMIT_TIME\" AS \"CC_COMMIT_TIME\", \"CQ_POOL_NAME\" AS \"CC_POOL_NAME\", " + - " \"CQ_NUMBER_OF_BUCKETS\" AS \"CC_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\" AS \"CC_ORDER_BY\" " + - " FROM " + - " \"COMPACTION_QUEUE\" " + - " UNION ALL " + - " SELECT " + - " \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", " + - " \"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_RUN_AS\", " + - " \"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", " + - " \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\", " + - " -1 , \"CC_NEXT_TXN_ID\", \"CC_TXN_ID\", \"CC_NEXT_TXN_ID\", \"CC_POOL_NAME\", " + - " \"CC_NUMBER_OF_BUCKETS\", \"CC_ORDER_BY\" " + - " FROM " + - " \"COMPLETED_COMPACTIONS\") XX "; - - - public static final String INSERT_INTO_COMPLETED_COMPACTION = - "INSERT INTO \"COMPLETED_COMPACTIONS\" " + - " (\"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", " + - " \"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_END\", \"CC_RUN_AS\", " + - " \"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", " + - " \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\"," + - " \"CC_NEXT_TXN_ID\", \"CC_TXN_ID\", \"CC_COMMIT_TIME\", \"CC_POOL_NAME\", \"CC_NUMBER_OF_BUCKETS\", " + - " \"CC_ORDER_BY\") " + - " VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)"; -} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java index afee07974204..6e04bf0d6f78 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.util.ReflectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -205,10 +206,11 @@ public boolean isDeadlock(SQLException e) { /** * Is the given exception a table not found exception - * @param e Exception + * @param t Exception * @return */ - public boolean isTableNotExistsError(SQLException e) { + public boolean isTableNotExistsError(Throwable t) { + SQLException e = TxnUtils.getSqlException(t); return (isPOSTGRES() && "42P01".equalsIgnoreCase(e.getSQLState())) || (isMYSQL() && "42S02".equalsIgnoreCase(e.getSQLState())) || (isORACLE() && "42000".equalsIgnoreCase(e.getSQLState()) && e.getMessage().contains("ORA-00942")) @@ -558,41 +560,42 @@ public boolean supportsGetGeneratedKeys() throws MetaException { } } - public boolean isDuplicateKeyError(SQLException ex) { + public boolean isDuplicateKeyError(Throwable t) { + SQLException sqlEx = TxnUtils.getSqlException(t); switch (dbType) { case DERBY: case CUSTOM: // ANSI SQL - if("23505".equals(ex.getSQLState())) { + if("23505".equals(sqlEx.getSQLState())) { return true; } break; case MYSQL: //https://dev.mysql.com/doc/refman/5.5/en/error-messages-server.html - if((ex.getErrorCode() == 1022 || ex.getErrorCode() == 1062 || ex.getErrorCode() == 1586) - && "23000".equals(ex.getSQLState())) { + if((sqlEx.getErrorCode() == 1022 || sqlEx.getErrorCode() == 1062 || sqlEx.getErrorCode() == 1586) + && "23000".equals(sqlEx.getSQLState())) { return true; } break; case SQLSERVER: //2627 is unique constaint violation incl PK, 2601 - unique key - if ((ex.getErrorCode() == 2627 || ex.getErrorCode() == 2601) && "23000".equals(ex.getSQLState())) { + if ((sqlEx.getErrorCode() == 2627 || sqlEx.getErrorCode() == 2601) && "23000".equals(sqlEx.getSQLState())) { return true; } break; case ORACLE: - if(ex.getErrorCode() == 1 && "23000".equals(ex.getSQLState())) { + if(sqlEx.getErrorCode() == 1 && "23000".equals(sqlEx.getSQLState())) { return true; } break; case POSTGRES: //http://www.postgresql.org/docs/8.1/static/errcodes-appendix.html - if("23505".equals(ex.getSQLState())) { + if("23505".equals(sqlEx.getSQLState())) { return true; } break; default: - String msg = ex.getMessage() + - " (SQLState=" + ex.getSQLState() + ", ErrorCode=" + ex.getErrorCode() + ")"; + String msg = sqlEx.getMessage() + + " (SQLState=" + sqlEx.getSQLState() + ", ErrorCode=" + sqlEx.getErrorCode() + ")"; throw new IllegalArgumentException("Unexpected DB type: " + dbType + "; " + msg); } return false; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index 1399553e40bb..a9dbb03d989e 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -62,6 +62,7 @@ import org.apache.hadoop.hive.metastore.properties.PropertyMap; import org.apache.hadoop.hive.metastore.properties.PropertyStore; import org.apache.hadoop.hive.metastore.txn.*; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; import org.apache.hadoop.hive.metastore.utils.FileUtils; import org.apache.hadoop.hive.metastore.utils.FilterUtils; import org.apache.hadoop.hive.metastore.utils.HdfsUtils; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java index f728f9a0b11d..265a43d6756d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java @@ -22,8 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hive.metastore.IHMSHandler; import org.apache.hadoop.hive.metastore.api.CompactionType; -import org.apache.hadoop.hive.metastore.api.TxnType; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; /** * CommitCompactionEvent diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java index 35450a4fe60d..4fbfced83284 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java @@ -23,8 +23,8 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData; -import org.apache.hadoop.hive.metastore.txn.MetricsInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData; +import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.slf4j.Logger; @@ -33,9 +33,9 @@ import java.util.List; import java.util.concurrent.TimeUnit; -import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_DELTAS; -import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS; -import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS; +import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_DELTAS; +import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS; +import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS; /** * diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java index cc29af053f24..d80f84219eea 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java @@ -32,8 +32,8 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData; -import org.apache.hadoop.hive.metastore.txn.MetricsInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData; +import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.thrift.TException; @@ -78,9 +78,9 @@ import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.OLDEST_OPEN_REPL_TXN_ID; import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.OLDEST_READY_FOR_CLEANING_AGE; import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.TABLES_WITH_X_ABORTED_TXNS; -import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_DELTAS; -import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS; -import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS; +import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_DELTAS; +import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS; +import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS; /** * Collect and publish ACID and compaction related metrics. diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java index e579311d3687..531ae67fcfaa 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionMetricsDataStruct; import org.apache.hadoop.hive.metastore.api.CompactionMetricsMetricType; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData; public class CompactionMetricsDataConverter { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index 6bde27a47597..d3b6091574a2 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -17,45 +17,42 @@ */ package org.apache.hadoop.hive.metastore.txn; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.classification.RetrySemantics; -import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier; import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.FindNextCompactRequest; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; -import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider; -import org.apache.hadoop.hive.metastore.events.CommitCompactionEvent; -import org.apache.hadoop.hive.metastore.messaging.EventMessage; -import org.apache.hadoop.hive.metastore.txn.impl.CleanTxnToWriteIdTableFunction; -import org.apache.hadoop.hive.metastore.txn.impl.FindPotentialCompactionsFunction; -import org.apache.hadoop.hive.metastore.txn.impl.NextCompactionFunction; -import org.apache.hadoop.hive.metastore.txn.impl.ReadyToCleanAbortHandler; -import org.apache.hadoop.hive.metastore.txn.impl.CheckFailedCompactionsHandler; -import org.apache.hadoop.hive.metastore.txn.impl.CompactionMetricsDataHandler; -import org.apache.hadoop.hive.metastore.txn.impl.FindColumnsWithStatsHandler; -import org.apache.hadoop.hive.metastore.txn.impl.GetCompactionInfoHandler; -import org.apache.hadoop.hive.metastore.txn.impl.InsertCompactionInfoCommand; -import org.apache.hadoop.hive.metastore.txn.impl.MarkCleanedFunction; -import org.apache.hadoop.hive.metastore.txn.impl.PurgeCompactionHistoryFunction; -import org.apache.hadoop.hive.metastore.txn.impl.ReadyToCleanHandler; -import org.apache.hadoop.hive.metastore.txn.impl.RemoveCompactionMetricsDataCommand; -import org.apache.hadoop.hive.metastore.txn.impl.RemoveDuplicateCompleteTxnComponentsCommand; -import org.apache.hadoop.hive.metastore.txn.impl.TopCompactionMetricsDataPerTypeFunction; -import org.apache.hadoop.hive.metastore.txn.impl.UpdateCompactionMetricsDataFunction; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionState; +import org.apache.hadoop.hive.metastore.txn.entities.OperationType; +import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus; +import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertCompactionInfoCommand; +import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveCompactionMetricsDataCommand; +import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveDuplicateCompleteTxnComponentsCommand; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.CleanTxnToWriteIdTableFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.FindPotentialCompactionsFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.GenerateCompactionQueueIdFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.MarkCleanedFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.MinOpenTxnIdWaterMarkFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.NextCompactionFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.PurgeCompactionHistoryFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.TopCompactionMetricsDataPerTypeFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.UpdateCompactionMetricsDataFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.CheckFailedCompactionsHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.CompactionMetricsDataHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.FindColumnsWithStatsHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetCompactionInfoHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.OpenTxnTimeoutLowBoundaryTxnIdHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.ReadyToCleanAbortHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.ReadyToCleanHandler; import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.dao.DataAccessException; -import org.springframework.jdbc.UncategorizedSQLException; import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.Statement; import java.sql.Types; import java.util.List; import java.util.Optional; @@ -73,26 +70,9 @@ class CompactionTxnHandler extends TxnHandler { private static final Logger LOG = LoggerFactory.getLogger(CompactionTxnHandler.class.getName()); - private static boolean initialized = false; - public CompactionTxnHandler() { } - @Override - public void setConf(Configuration conf) { - super.setConf(conf); - synchronized (CompactionTxnHandler.class) { - if (!initialized) { - int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.HIVE_COMPACTOR_CONNECTION_POOLING_MAX_CONNECTIONS); - try (DataSourceProvider.DataSourceNameConfigurator configurator = - new DataSourceProvider.DataSourceNameConfigurator(conf, "compactor")) { - jdbcResource.registerDataSource(POOL_COMPACTOR, setupJdbcConnectionPool(conf, maxPoolSize)); - initialized = true; - } - } - } - } - /** * This will look through the completed_txn_components table and look for partitions or tables * that may be ready for compaction. Also, look through txns and txn_components tables for @@ -143,7 +123,7 @@ public CompactionInfo findNextToCompact(FindNextCompactRequest rqst) throws Meta if (rqst == null) { throw new MetaException("FindNextCompactRequest is null"); } - long poolTimeout = MetastoreConf.getTimeVar(conf, ConfVars.COMPACTOR_WORKER_POOL_TIMEOUT, TimeUnit.MILLISECONDS); + long poolTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_POOL_TIMEOUT, TimeUnit.MILLISECONDS); return new NextCompactionFunction(rqst, getDbTime(), poolTimeout).execute(jdbcResource); } @@ -172,7 +152,7 @@ public void markCompacted(CompactionInfo info) throws MetaException { @Override @RetrySemantics.ReadOnly public List findReadyToClean(long minOpenTxnWaterMark, long retentionTime) throws MetaException { - return jdbcResource.execute(new ReadyToCleanHandler(conf, useMinHistoryWriteId, minOpenTxnWaterMark, retentionTime)); + return jdbcResource.execute(new ReadyToCleanHandler(conf, minOpenTxnWaterMark, retentionTime)); } @Override @@ -227,7 +207,7 @@ private void setCleanerStart(CompactionInfo info, Long timestamp) throws MetaExc @RetrySemantics.CannotRetry public void markCleaned(CompactionInfo info) throws MetaException { LOG.debug("Running markCleaned with CompactionInfo: {}", info); - new MarkCleanedFunction(info, conf).execute(jdbcResource); + new MarkCleanedFunction(info).execute(jdbcResource); } /** @@ -237,7 +217,7 @@ public void markCleaned(CompactionInfo info) throws MetaException { @Override @RetrySemantics.SafeToRetry public void cleanTxnToWriteIdTable() throws MetaException { - new CleanTxnToWriteIdTableFunction(useMinHistoryLevel, findMinTxnIdSeenOpen()).execute(jdbcResource); + new CleanTxnToWriteIdTableFunction(findMinTxnIdSeenOpen()).execute(jdbcResource); } @Override @@ -264,7 +244,7 @@ public void cleanEmptyAbortedAndCommittedTxns() throws MetaException { * 2. never deletes the maximum txnId even if it is before the TXN_OPENTXN_TIMEOUT window */ try { - long lowWaterMark = getOpenTxnTimeoutLowBoundaryTxnId(jdbcResource.getConnection()); + long lowWaterMark = jdbcResource.execute(new OpenTxnTimeoutLowBoundaryTxnIdHandler(openTxnTimeOutMillis)); jdbcResource.execute( "DELETE FROM \"TXNS\" WHERE \"TXN_ID\" NOT IN (SELECT \"TC_TXNID\" FROM \"TXN_COMPONENTS\") " + "AND (\"TXN_STATE\" = :abortedState OR \"TXN_STATE\" = :committedState) AND \"TXN_ID\" < :txnId", @@ -273,7 +253,7 @@ public void cleanEmptyAbortedAndCommittedTxns() throws MetaException { .addValue("abortedState", TxnStatus.ABORTED.getSqlConst(), Types.CHAR) .addValue("committedState", TxnStatus.COMMITTED.getSqlConst(), Types.CHAR), null); - } catch (SQLException e) { + } catch (DataAccessException e) { throw new MetaException("Unable to get the txn id: " + SqlRetryHandler.getMessage(e)); } } @@ -390,7 +370,7 @@ public void updateCompactorState(CompactionInfo ci, long compactionTxnId) throws @Override @RetrySemantics.SafeToRetry public void purgeCompactionHistory() throws MetaException { - new PurgeCompactionHistoryFunction(conf).execute(jdbcResource); + new PurgeCompactionHistoryFunction().execute(jdbcResource); } /** @@ -431,7 +411,7 @@ private void updateStatus(CompactionInfo ci) throws MetaException { if (ciActual.id == 0) { //The failure occurred before we even made an entry in COMPACTION_QUEUE //generate ID so that we can make an entry in COMPLETED_COMPACTIONS - ciActual.id = generateCompactionQueueId(); + ciActual.id = new GenerateCompactionQueueIdFunction().execute(jdbcResource); //this is not strictly accurate, but 'type' cannot be null. if (ciActual.type == null) { ciActual.type = CompactionType.MINOR; @@ -480,7 +460,7 @@ public void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws Meta * compactions for any resource. */ try (TxnStore.MutexAPI.LockHandle ignored = getMutexAPI().acquireLock(MUTEX_KEY.CompactionScheduler.name())) { - long id = generateCompactionQueueId(); + long id = new GenerateCompactionQueueIdFunction().execute(jdbcResource); int updCnt = jdbcResource.execute( "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " + " \"CQ_TYPE\", \"CQ_STATE\", \"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", \"CQ_COMMIT_TIME\") " + @@ -527,14 +507,10 @@ public void setHadoopJobId(String hadoopJobId, long id) throws MetaException { @Override @RetrySemantics.Idempotent public long findMinOpenTxnIdForCleaner() throws MetaException { - if (useMinHistoryWriteId) { + if (ConfVars.useMinHistoryWriteId()) { return Long.MAX_VALUE; } - try { - return getMinOpenTxnIdWaterMark(jdbcResource.getConnection()); - } catch (SQLException e) { - throw new UncategorizedSQLException(null, null, e); - } + return new MinOpenTxnIdWaterMarkFunction(openTxnTimeOutMillis).execute(jdbcResource); } /** @@ -546,7 +522,7 @@ public long findMinOpenTxnIdForCleaner() throws MetaException { @RetrySemantics.Idempotent @Deprecated public long findMinTxnIdSeenOpen() { - if (!useMinHistoryLevel || useMinHistoryWriteId) { + if (!ConfVars.useMinHistoryLevel() || ConfVars.useMinHistoryWriteId()) { return Long.MAX_VALUE; } try { @@ -554,53 +530,20 @@ public long findMinTxnIdSeenOpen() { new MapSqlParameterSource(), Long.class); return minId == null ? Long.MAX_VALUE : minId; } catch (DataAccessException e) { - if (e.getCause() instanceof SQLException) { - if (dbProduct.isTableNotExistsError((SQLException) e.getCause())) { - useMinHistoryLevel = false; - return Long.MAX_VALUE; - } + if (dbProduct.isTableNotExistsError(e)) { + ConfVars.setUseMinHistoryLevel(false); + return Long.MAX_VALUE; } LOG.error("Unable to execute findMinTxnIdSeenOpen", e); throw e; } } - @Override - protected void updateWSCommitIdAndCleanUpMetadata(Statement stmt, long txnid, TxnType txnType, - Long commitId, long tempId) throws SQLException, MetaException { - super.updateWSCommitIdAndCleanUpMetadata(stmt, txnid, txnType, commitId, tempId); - - if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) { - stmt.executeUpdate("UPDATE \"COMPACTION_QUEUE\" SET \"CQ_NEXT_TXN_ID\" = " + commitId + ", \"CQ_COMMIT_TIME\" = " + - getEpochFn(dbProduct) + " WHERE \"CQ_TXN_ID\" = " + txnid); - } - } - @Override public Optional getCompactionByTxnId(long txnId) throws MetaException { return Optional.ofNullable(jdbcResource.execute(new GetCompactionInfoHandler(txnId, true))); } - @Override - protected void createCommitNotificationEvent(Connection conn, long txnid, TxnType txnType) - throws MetaException, SQLException { - super.createCommitNotificationEvent(conn, txnid, txnType); - if (transactionalListeners != null) { - //Please note that TxnHandler and CompactionTxnHandler are using different DataSources (to have different pools). - //This call must use the same transaction and connection as TxnHandler.commitTxn(), therefore we are passing the - //datasource wrapper comming from TxnHandler. Without this, the getCompactionByTxnId(long txnId) call would be - //executed using a different connection obtained from CompactionTxnHandler's own datasourceWrapper. - CompactionInfo compactionInfo = getCompactionByTxnId(txnid).orElse(null); - if (compactionInfo != null) { - MetaStoreListenerNotifier - .notifyEventWithDirectSql(transactionalListeners, EventMessage.EventType.COMMIT_COMPACTION, - new CommitCompactionEvent(txnid, compactionInfo), conn, sqlGenerator); - } else { - LOG.warn("No compaction queue record found for Compaction type transaction commit. txnId:" + txnid); - } - } - } - @Override public boolean updateCompactionMetricsData(CompactionMetricsData data) throws MetaException { return new UpdateCompactionMetricsDataFunction(data).execute(jdbcResource); @@ -609,7 +552,7 @@ public boolean updateCompactionMetricsData(CompactionMetricsData data) throws Me @Override public List getTopCompactionMetricsDataPerType(int limit) throws MetaException { - return new TopCompactionMetricsDataPerTypeFunction(limit, sqlGenerator).execute(jdbcResource); + return new TopCompactionMetricsDataPerTypeFunction(limit).execute(jdbcResource); } @Override diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java new file mode 100644 index 000000000000..39b760d016ee --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; +import org.apache.hadoop.hive.metastore.api.UnlockRequest; +import org.apache.hadoop.hive.metastore.txn.entities.LockInfo; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.CheckLockFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.EnqueueLockFunction; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetLocksByLockId; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.ShowLocksHandler; +import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource; +import org.apache.hadoop.hive.metastore.txn.jdbc.RollbackException; +import org.apache.hadoop.hive.metastore.utils.JavaUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; + +import java.sql.Types; +import java.util.List; + +import static org.apache.hadoop.hive.metastore.txn.entities.LockInfo.LOCK_WAITING; + +public class DefaultTxnLockManager implements TxnLockManager { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultTxnLockManager.class); + + private final MultiDataSourceJdbcResource jdbcResource; + + public DefaultTxnLockManager(MultiDataSourceJdbcResource jdbcResource) { + this.jdbcResource = jdbcResource; + } + + @Override + public long enqueueLock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException { + return new EnqueueLockFunction(rqst).execute(jdbcResource); + } + + @Override + public LockResponse checkLock(long extLockId, long txnId, boolean zeroWaitReadEnabled, boolean isExclusiveCTAS) + throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException { + return new CheckLockFunction(extLockId, txnId, zeroWaitReadEnabled, isExclusiveCTAS).execute(jdbcResource); + } + + @Override + public void unlock(UnlockRequest rqst) throws TxnOpenException, MetaException { + long extLockId = rqst.getLockid(); + /** + * This method is logically like commit for read-only auto commit queries. + * READ_COMMITTED since this only has 1 delete statement and no new entries with the + * same hl_lock_ext_id can be added, i.e. all rows with a given hl_lock_ext_id are + * created in a single atomic operation. + * Theoretically, this competes with {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)} + * but hl_lock_ext_id is not known until that method returns. + * Also competes with {@link #checkLock(org.apache.hadoop.hive.metastore.api.CheckLockRequest)} + * but using SERIALIZABLE doesn't materially change the interaction. + * If "delete" stmt misses, additional logic is best effort to produce meaningful error msg. + */ + //hl_txnid <> 0 means it's associated with a transaction + int rc = jdbcResource.getJdbcTemplate().update("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = :extLockId " + + " AND (\"HL_TXNID\" = 0 OR (\"HL_TXNID\" <> 0 AND \"HL_LOCK_STATE\" = :state))", + new MapSqlParameterSource() + .addValue("extLockId", extLockId) + .addValue("state", Character.toString(LOCK_WAITING), Types.CHAR)); + //(hl_txnid <> 0 AND hl_lock_state = '" + LOCK_WAITING + "') is for multi-statement txns where + //some query attempted to lock (thus LOCK_WAITING state) but is giving up due to timeout for example + if (rc < 1) { + LOG.info("Failure to unlock any locks with extLockId={}.", extLockId); + List lockInfos = jdbcResource.execute(new GetLocksByLockId(extLockId, 1, jdbcResource.getSqlGenerator())); + if (CollectionUtils.isEmpty(lockInfos)) { + //didn't find any lock with extLockId but at ReadCommitted there is a possibility that + //it existed when above delete ran but it didn't have the expected state. + LOG.info("No lock in {} mode found for unlock({})", LOCK_WAITING, + JavaUtils.lockIdToString(rqst.getLockid())); + + //bail here to make the operation idempotent + throw new RollbackException(null); + } + LockInfo lockInfo = lockInfos.get(0); + if (TxnUtils.isValidTxn(lockInfo.getTxnId())) { + String msg = "Unlocking locks associated with transaction not permitted. " + lockInfo; + //if a lock is associated with a txn we can only "unlock" it if it's in WAITING state + // which really means that the caller wants to give up waiting for the lock + LOG.error(msg); + throw new TxnOpenException(msg); + } else { + //we didn't see this lock when running DELETE stmt above but now it showed up + //so should "should never happen" happened... + String msg = "Found lock in unexpected state " + lockInfo; + LOG.error(msg); + throw new MetaException(msg); + } + } + LOG.debug("Successfully unlocked at least 1 lock with extLockId={}", extLockId); + } + + @Override + public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException { + return jdbcResource.execute(new ShowLocksHandler(rqst)); + } + +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ProxyTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TransactionalRetryProxy.java similarity index 61% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ProxyTxnHandler.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TransactionalRetryProxy.java index 10a6e9b7654f..0b85b450f3c8 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ProxyTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TransactionalRetryProxy.java @@ -17,16 +17,19 @@ */ package org.apache.hadoop.hive.metastore.txn; +import org.apache.commons.lang3.ClassUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource; +import org.apache.hadoop.hive.metastore.txn.jdbc.RollbackException; import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryCallProperties; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetry; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryFunction; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetry; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryCallProperties; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryFunction; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler; +import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.springframework.dao.DataAccessException; import org.springframework.transaction.annotation.Transactional; import java.lang.reflect.InvocationHandler; @@ -37,45 +40,44 @@ import java.util.Arrays; /** - * Responsible for processing the following annotations: {@link SqlRetry} and {@link Transactional}. The annotated methods - * will be called accordingly: + * Responsible for proxying an interface, and calling its methods by processing the following annotations: + * {@link SqlRetry} and {@link Transactional}. The annotated methods will be called accordingly: *

    *
  • SQL errors in methods annotated with {@link SqlRetry} will be caught and the method will be re-executed
  • *
  • Methods annotated with {@link Transactional} will be executed after creating a transaction, and all operations done - * via {@link MultiDataSourceJdbcResource}, {@link org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedQuery}, - * {@link org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand} and - * {@link org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler} will use the created transaction.
  • + * via {@link MultiDataSourceJdbcResource} will use the created transaction. *
  • In case a method is annotated with both annotations, the transaction will be inside the retry-call. This means * in case of SQL errors and retries, the transaction will be rolled back and a new one will be created for each retry * attempt.
  • *
* Not annotated methods are called directly. */ -public class ProxyTxnHandler implements InvocationHandler { +public class TransactionalRetryProxy implements InvocationHandler { - private static final Logger LOG = LoggerFactory.getLogger(ProxyTxnHandler.class); + private static final Logger LOG = LoggerFactory.getLogger(TransactionalRetryProxy.class); /** * Gets the proxy interface for the given {@link TxnStore}. * - * @param realStore The real {@link TxnStore} to proxy. * @param sqlRetryHandler Responsible to re-execute the methods in case of failure. + * @param interfaceObject The real object to proxy. * @return Returns the proxy object capable of retrying the failed calls automatically and transparently. */ - public static TxnStore getProxy(TxnStore realStore, SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResourceHandler) { - ProxyTxnHandler handler = new ProxyTxnHandler(realStore, sqlRetryHandler, jdbcResourceHandler); - return (TxnStore) Proxy.newProxyInstance( - ProxyTxnHandler.class.getClassLoader(), - new Class[]{ TxnStore.class }, + public static T getProxy(SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResourceHandler, T interfaceObject) { + TransactionalRetryProxy handler = new TransactionalRetryProxy<>(interfaceObject, sqlRetryHandler, jdbcResourceHandler); + //noinspection unchecked + return (T) Proxy.newProxyInstance( + TransactionalRetryProxy.class.getClassLoader(), + ClassUtils.getAllInterfaces(interfaceObject.getClass()).toArray(new Class[0]), handler); } + private final T interfaceObject; private final SqlRetryHandler sqlRetryHandler; - private final TxnStore realStore; private final MultiDataSourceJdbcResource jdbcResource; - private ProxyTxnHandler(TxnStore realStore, SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResource) { - this.realStore = realStore; + private TransactionalRetryProxy(T interfaceObject, SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResource) { + this.interfaceObject = interfaceObject; this.sqlRetryHandler = sqlRetryHandler; this.jdbcResource = jdbcResource; } @@ -95,7 +97,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl ThrowingSupplier functionToCall = () -> { try { - return method.invoke(realStore, args); + return method.invoke(interfaceObject, args); } catch (InvocationTargetException | UndeclaredThrowableException e) { throw e.getCause(); } @@ -108,23 +110,36 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl TransactionContext context = null; try { jdbcResource.bindDataSource(transactional); - context = jdbcResource.getTransactionManager().getTransaction(transactional.propagation().value()); + context = jdbcResource.getTransactionManager().getNewTransaction(transactional.propagation().value()); Object result = toCall.execute(); LOG.debug("Successfull method invocation within transactional context: {}, going to commit.", callerId); - jdbcResource.getTransactionManager().commit(context); + if (context.isRollbackOnly()) { + jdbcResource.getTransactionManager().rollback(context); + } else if (!context.isCompleted()) { + jdbcResource.getTransactionManager().commit(context); + } return result; + } catch (RollbackException e) { + if (context != null && !context.isCompleted()) { + jdbcResource.getTransactionManager().rollback(context); + } + return e.getResult(); } catch (Exception e) { - if (Arrays.stream(transactional.noRollbackFor()).anyMatch(ex -> ex.isInstance(e)) || - Arrays.stream(transactional.noRollbackForClassName()).anyMatch(exName -> exName.equals(e.getClass().getName()))) { - throw e; - } if (context != null) { - if (transactional.rollbackFor().length > 0 || transactional.rollbackForClassName().length > 0) { + if (transactional.noRollbackFor().length > 0 || transactional.noRollbackForClassName().length > 0) { + if (Arrays.stream(transactional.noRollbackFor()).anyMatch(ex -> ex.isInstance(e)) || + Arrays.stream(transactional.noRollbackForClassName()).anyMatch(exName -> exName.equals(e.getClass().getName()))) { + jdbcResource.getTransactionManager().commit(context); + } else { + jdbcResource.getTransactionManager().rollback(context); + } + } else if (transactional.rollbackFor().length > 0 || transactional.rollbackForClassName().length > 0) { if (Arrays.stream(transactional.rollbackFor()).anyMatch(ex -> ex.isInstance(e)) || Arrays.stream(transactional.rollbackForClassName()).anyMatch(exName -> exName.equals(e.getClass().getName()))) { jdbcResource.getTransactionManager().rollback(context); + } else { + jdbcResource.getTransactionManager().commit(context); } - throw e; } else { jdbcResource.getTransactionManager().rollback(context); } @@ -149,15 +164,23 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl LOG.debug("Successfull method invocation within retry context: {}", callerId); return result; } catch (IllegalAccessException | InvocationTargetException | UndeclaredThrowableException e) { - if (e.getCause() instanceof MetaException) { - throw (MetaException) e.getCause(); + if (e.getCause() instanceof TException) { + throw (TException) e.getCause(); } else if (e.getCause() instanceof RuntimeException) { throw (RuntimeException) e.getCause(); } else { throw new RuntimeException(e); } - } catch (Throwable e) { - throw new RuntimeException(e); + } catch (TException | DataAccessException e) { + throw e; + } catch (Throwable e) { + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } else if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException)e.getCause(); + } else { + throw new RuntimeException(e); + } } }; return sqlRetryHandler.executeWithRetry(properties, retryWrapper); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java index 8905a01490e0..9be5e475b6ed 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -17,76 +17,19 @@ */ package org.apache.hadoop.hive.metastore.txn; -import java.io.PrintWriter; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.sql.Connection; -import java.sql.Driver; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.SQLFeatureNotSupportedException; -import java.sql.Statement; -import java.sql.Timestamp; -import java.text.MessageFormat; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Function; -import java.util.function.Predicate; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import javax.sql.DataSource; - -import com.google.common.collect.ImmutableList; +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.time.StopWatch; -import org.apache.commons.lang3.ArrayUtils; -import org.apache.commons.lang3.NotImplementedException; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.MaterializationSnapshot; -import org.apache.hadoop.hive.common.TableName; -import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; -import org.apache.hadoop.hive.common.ValidReadTxnList; -import org.apache.hadoop.hive.common.ValidReaderWriteIdList; import org.apache.hadoop.hive.common.ValidTxnList; -import org.apache.hadoop.hive.common.ValidTxnWriteIdList; -import org.apache.hadoop.hive.common.ValidWriteIdList; import org.apache.hadoop.hive.common.classification.RetrySemantics; -import org.apache.hadoop.hive.common.repl.ReplConst; import org.apache.hadoop.hive.metastore.DatabaseProduct; -import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier; import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener; -import org.apache.hadoop.hive.metastore.LockTypeComparator; import org.apache.hadoop.hive.metastore.api.AbortCompactResponse; import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest; -import org.apache.hadoop.hive.metastore.api.AbortCompactionResponseElement; -import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException; import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest; import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions; @@ -94,14 +37,10 @@ import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse; import org.apache.hadoop.hive.metastore.api.CheckLockRequest; import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; -import org.apache.hadoop.hive.metastore.api.CompactionInfoStruct; import org.apache.hadoop.hive.metastore.api.CompactionRequest; import org.apache.hadoop.hive.metastore.api.CompactionResponse; -import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.CreationMetadata; -import org.apache.hadoop.hive.metastore.api.DataOperationType; import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoRequest; import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoResponse; import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; @@ -112,104 +51,101 @@ import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest; import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse; import org.apache.hadoop.hive.metastore.api.HiveObjectType; -import org.apache.hadoop.hive.metastore.api.LockComponent; import org.apache.hadoop.hive.metastore.api.LockRequest; import org.apache.hadoop.hive.metastore.api.LockResponse; -import org.apache.hadoop.hive.metastore.api.LockState; -import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.Materialization; import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdRequest; import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdResponse; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException; import org.apache.hadoop.hive.metastore.api.NoSuchLockException; import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.ReplLastIdInfo; import org.apache.hadoop.hive.metastore.api.ReplTblWriteIdStateRequest; import org.apache.hadoop.hive.metastore.api.SeedTableWriteIdsRequest; import org.apache.hadoop.hive.metastore.api.SeedTxnIdRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; -import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; import org.apache.hadoop.hive.metastore.api.TxnAbortedException; import org.apache.hadoop.hive.metastore.api.TxnOpenException; -import org.apache.hadoop.hive.metastore.api.TxnToWriteId; import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.api.UnlockRequest; import org.apache.hadoop.hive.metastore.api.UpdateTransactionalStatsRequest; -import org.apache.hadoop.hive.metastore.api.WriteEventInfo; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider; import org.apache.hadoop.hive.metastore.datasource.DataSourceProviderFactory; import org.apache.hadoop.hive.metastore.events.AbortTxnEvent; -import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent; -import org.apache.hadoop.hive.metastore.events.CommitTxnEvent; -import org.apache.hadoop.hive.metastore.events.ListenerEvent; -import org.apache.hadoop.hive.metastore.events.OpenTxnEvent; import org.apache.hadoop.hive.metastore.events.AcidWriteEvent; +import org.apache.hadoop.hive.metastore.events.ListenerEvent; import org.apache.hadoop.hive.metastore.messaging.EventMessage; import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.tools.SQLGenerator; -import org.apache.hadoop.hive.metastore.txn.impl.InsertCompactionInfoCommand; -import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryCallProperties; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionState; +import org.apache.hadoop.hive.metastore.txn.entities.LockInfo; +import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo; +import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus; +import org.apache.hadoop.hive.metastore.txn.jdbc.commands.*; +import org.apache.hadoop.hive.metastore.txn.jdbc.functions.*; +import org.apache.hadoop.hive.metastore.txn.jdbc.queries.*; import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryFunction; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler; -import org.apache.hadoop.hive.metastore.utils.FileUtils; +import org.apache.hadoop.hive.metastore.txn.jdbc.NoPoolConnectionPool; +import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryCallProperties; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryException; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler; import org.apache.hadoop.hive.metastore.utils.JavaUtils; -import org.apache.hadoop.hive.metastore.utils.LockTypeUtil; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; -import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.apache.hadoop.hive.metastore.utils.StringableMap; import org.apache.hadoop.util.StringUtils; +import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import com.google.common.base.Splitter; - -import static org.apache.commons.lang3.StringUtils.isNotBlank; -import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.commons.lang3.StringUtils.repeat; -import static org.apache.commons.lang3.StringUtils.EMPTY; -import static org.apache.commons.lang3.StringUtils.wrap; -import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn; -import static org.apache.hadoop.hive.metastore.txn.TxnUtils.executeQueriesInBatchNoCount; -import static org.apache.hadoop.hive.metastore.txn.TxnUtils.executeQueriesInBatch; -import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getFullTableName; -import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; -import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier; -import static org.springframework.transaction.TransactionDefinition.PROPAGATION_REQUIRED; - -import com.google.common.annotations.VisibleForTesting; import org.springframework.dao.DataAccessException; import org.springframework.jdbc.UncategorizedSQLException; import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; +import javax.sql.DataSource; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.text.MessageFormat; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiPredicate; + +import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; + /** * A handler to answer transaction related calls that come into the metastore * server. - * + *

* Note on log messages: Please include txnid:X and lockid info using * {@link JavaUtils#txnIdToString(long)} * and {@link JavaUtils#lockIdToString(long)} in all messages. * The txnid:X and lockid:Y matches how Thrift object toString() methods are generated, * so keeping the format consistent makes grep'ing the logs much easier. - * + *

* Note on HIVE_LOCKS.hl_last_heartbeat. * For locks that are part of transaction, we set this 0 (would rather set it to NULL but * Currently the DB schema has this NOT NULL) and only update/read heartbeat from corresponding * transaction in TXNS. - * + *

* In general there can be multiple metastores where this logic can execute, thus the DB is * used to ensure proper mutexing of operations. * Select ... For Update (or equivalent: either MsSql with(updlock) or actual Update stmt) is @@ -221,22 +157,21 @@ * This allows almost all operations to run at READ_COMMITTED and minimizes DB deadlocks. * 3. checkLock() - this is mutexted entirely since we must ensure that while we check if some lock * can be granted, no other (strictly speaking "earlier") lock can change state. - * - * The exception to his is Derby which doesn't support proper S4U. Derby is always running embedded + *

+ * The exception to this is Derby which doesn't support proper S4U. Derby is always running embedded * (this is the only supported configuration for Derby) * in the same JVM as HiveMetaStoreHandler thus we use JVM wide lock to properly sequnce the operations. - * - * {@link #derbyLock} + *

* If we ever decide to run remote Derby server, according to * https://db.apache.org/derby/docs/10.0/manuals/develop/develop78.html all transactions will be * seriazlied, so that would also work though has not been tested. - * + *

* General design note: * It's imperative that any operation on a txn (e.g. commit), ensure (atomically) that this txn is * still valid and active. In the code this is usually achieved at the same time the txn record * is locked for some operation. - * + *

* Note on retry logic: * Metastore has retry logic in both {@link org.apache.hadoop.hive.metastore.RetryingMetaStoreClient} * and {@link org.apache.hadoop.hive.metastore.RetryingHMSHandler}. The retry logic there is very @@ -248,136 +183,82 @@ * the metstore call stack should have logic not to retry. There are {@link RetrySemantics} * annotations to document the behavior. */ +@SuppressWarnings("SqlSourceToSinkFlow") @InterfaceAudience.Private @InterfaceStability.Evolving -abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI { - - private static final String TXN_TMP_STATE = "_"; - private static final String DEFAULT_POOL_NAME = "default"; +public abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI { - // Lock states - static final protected char LOCK_ACQUIRED = 'a'; - static final protected char LOCK_WAITING = 'w'; + + public final static class ConfVars { + + private ConfVars() {} - private static final int ALLOWED_REPEATED_DEADLOCKS = 10; - private static final Logger LOG = LoggerFactory.getLogger(TxnHandler.class.getName()); + // Whether to use min_history_level table or not. + // At startup we read it from the config, but set it to false if min_history_level does nto exists. + private boolean useMinHistoryLevel; + private boolean useMinHistoryWriteId; - private static DataSource connPool; - private static DataSource connPoolMutex; + public boolean useMinHistoryLevel() { + return useMinHistoryLevel; + } - private static final String MANUAL_RETRY = "ManualRetry"; + public void setUseMinHistoryLevel(boolean useMinHistoryLevel) { + this.useMinHistoryLevel = useMinHistoryLevel; + } - // Query definitions - private static final String HIVE_LOCKS_INSERT_QRY = "INSERT INTO \"HIVE_LOCKS\" ( " + - "\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", " + - "\"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", \"HL_USER\", \"HL_HOST\", \"HL_AGENT_INFO\") " + - "VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s, ?, ?, ?)"; - private static final String TXN_COMPONENTS_INSERT_QUERY = "INSERT INTO \"TXN_COMPONENTS\" (" + - "\"TC_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_OPERATION_TYPE\", \"TC_WRITEID\")" + - " VALUES (?, ?, ?, ?, ?, ?)"; - private static final String TXN_COMPONENTS_DP_DELETE_QUERY = "DELETE FROM \"TXN_COMPONENTS\" " + - "WHERE \"TC_TXNID\" = ? AND \"TC_DATABASE\" = ? AND \"TC_TABLE\" = ? AND \"TC_PARTITION\" IS NULL"; - private static final String INCREMENT_NEXT_LOCK_ID_QUERY = "UPDATE \"NEXT_LOCK_ID\" SET \"NL_NEXT\" = %s"; - private static final String UPDATE_HIVE_LOCKS_EXT_ID_QUERY = "UPDATE \"HIVE_LOCKS\" SET \"HL_LOCK_EXT_ID\" = %s " + - "WHERE \"HL_LOCK_EXT_ID\" = %s"; - private static final String SELECT_WRITE_ID_QUERY = "SELECT \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE" + - " \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND \"T2W_TXNID\" = ?"; - private static final String COMPL_TXN_COMPONENTS_INSERT_QUERY = "INSERT INTO \"COMPLETED_TXN_COMPONENTS\" " + - "(\"CTC_TXNID\"," + " \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", \"CTC_WRITEID\", \"CTC_UPDATE_DELETE\")" + - " VALUES (%s, ?, ?, ?, ?, %s)"; - private static final String TXNS_INSERT_QRY = "INSERT INTO \"TXNS\" " + - "(\"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", \"TXN_USER\", \"TXN_HOST\", \"TXN_TYPE\") " + - "VALUES(?,%s,%s,?,?,?)"; - private static final String SELECT_LOCKS_FOR_LOCK_ID_QUERY = "SELECT \"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", " + - "\"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_TXNID\" " + - "FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = ?"; - private static final String SELECT_TIMED_OUT_LOCKS_QUERY = "SELECT DISTINCT \"HL_LOCK_EXT_ID\" FROM \"HIVE_LOCKS\" " + - "WHERE \"HL_LAST_HEARTBEAT\" < %s - :timeout AND \"HL_TXNID\" = 0"; - private static final String TXN_TO_WRITE_ID_INSERT_QUERY = "INSERT INTO \"TXN_TO_WRITE_ID\" (\"T2W_TXNID\", " + - "\"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\") VALUES (?, ?, ?, ?)"; - private static final String MIN_HISTORY_WRITE_ID_INSERT_QUERY = "INSERT INTO \"MIN_HISTORY_WRITE_ID\" (\"MH_TXNID\", " + - "\"MH_DATABASE\", \"MH_TABLE\", \"MH_WRITEID\") VALUES (?, ?, ?, ?)"; - private static final String SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID = - "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?"; - private static final String SELECT_METRICS_INFO_QUERY = - "SELECT * FROM (SELECT COUNT(*) FROM \"TXN_TO_WRITE_ID\") \"TTWID\" CROSS JOIN (" + - "SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\") \"CTC\" CROSS JOIN (" + - "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" WHERE \"TXN_STATE\"='" + - TxnStatus.OPEN + "' AND \"TXN_TYPE\" = "+ TxnType.REPL_CREATED.getValue() +") \"TR\" CROSS JOIN (" + - "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" WHERE \"TXN_STATE\"='" + - TxnStatus.OPEN + "' AND \"TXN_TYPE\" != "+ TxnType.REPL_CREATED.getValue() +") \"T\" CROSS JOIN (" + - "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" WHERE \"TXN_STATE\"='" + - TxnStatus.ABORTED + "') \"A\" CROSS JOIN (" + - "SELECT COUNT(*), ({0} - MIN(\"HL_ACQUIRED_AT\"))/1000 FROM \"HIVE_LOCKS\") \"HL\" CROSS JOIN (" + - "SELECT ({0} - MIN(\"CQ_COMMIT_TIME\"))/1000 from \"COMPACTION_QUEUE\" WHERE " + - "\"CQ_STATE\"=''" + READY_FOR_CLEANING + "'') OLDEST_CLEAN"; - private static final String SELECT_TABLES_WITH_X_ABORTED_TXNS = - "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXN_COMPONENTS\" " + - "INNER JOIN \"TXNS\" ON \"TC_TXNID\" = \"TXN_ID\" WHERE \"TXN_STATE\" = " + TxnStatus.ABORTED + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" HAVING COUNT(\"TXN_ID\") > ?"; - - private static final String EXCL_CTAS_ERR_MSG = - "Failed to initiate a concurrent CTAS operation with the same table name, lockInfo : %s"; - private static final String ZERO_WAIT_READ_ERR_MSG = "Unable to acquire read lock due to an existing exclusive lock {%s}"; + public boolean useMinHistoryWriteId() { + return useMinHistoryWriteId; + } + public void setUseMinHistoryWriteId(boolean useMinHistoryWriteId) { + this.useMinHistoryWriteId = useMinHistoryWriteId; + } - protected List transactionalListeners; + public void init(BiPredicate tableCheck, Configuration conf){ + useMinHistoryWriteId = tableCheck.test("MIN_HISTORY_WRITE_ID", + MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_WRITE_ID)); + useMinHistoryLevel = tableCheck.test("MIN_HISTORY_LEVEL", + MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_LEVEL)); + + } + } + + private static final Logger LOG = LoggerFactory.getLogger(TxnHandler.class.getName()); + public static final TxnHandler.ConfVars ConfVars = new ConfVars(); // Maximum number of open transactions that's allowed private static volatile int maxOpenTxns = 0; // Whether number of open transactions reaches the threshold private static volatile boolean tooManyOpenTxns = false; + // Current number of open txns + private static AtomicInteger numOpenTxns; + + private static volatile boolean initialized = false; + private static DataSource connPool; + private static DataSource connPoolMutex; + protected static DataSource connPoolCompactor; + + protected static DatabaseProduct dbProduct; + protected static SQLGenerator sqlGenerator; + protected static long openTxnTimeOutMillis; /** * Number of consecutive deadlocks we have seen */ - private int deadlockCnt; - private long deadlockRetryInterval; protected Configuration conf; - protected static DatabaseProduct dbProduct; - protected static SQLGenerator sqlGenerator; - private static long openTxnTimeOutMillis; + protected List transactionalListeners; // (End user) Transaction timeout, in milliseconds. private long timeout; private long replicationTxnTimeout; - private int maxBatchSize; - private String identifierQuoteString; // quotes to use for quoting tables, where necessary - private long retryInterval; - private int retryLimit; - private int retryNum; - // Current number of open txns - private AtomicInteger numOpenTxns; - // Whether to use min_history_level table or not. - // At startup we read it from the config, but set it to false if min_history_level does nto exists. - static boolean useMinHistoryLevel; - static boolean useMinHistoryWriteId; - - private static SqlRetryHandler sqlRetryHandler; - protected static MultiDataSourceJdbcResource jdbcResource; + private MutexAPI mutexAPI; + private TxnLockManager txnLockManager; + private SqlRetryHandler sqlRetryHandler; + protected MultiDataSourceJdbcResource jdbcResource; - /** - * Derby specific concurrency control - */ - private static final ReentrantLock derbyLock = new ReentrantLock(true); - /** - * must be static since even in UT there may be > 1 instance of TxnHandler - * (e.g. via Compactor services) - */ - private final static ConcurrentHashMap derbyKey2Lock = new ConcurrentHashMap<>(); private static final String hostname = JavaUtils.hostname(); - // Private methods should never catch SQLException and then throw MetaException. The public - // methods depend on SQLException coming back so they can detect and handle deadlocks. Private - // methods should only throw MetaException when they explicitly know there's a logic error and - // they want to throw past the public methods. - // - // All public methods that write to the database have to check for deadlocks when a SQLException - // comes back and handle it if they see one. This has to be done with the connection pooling - // in mind. To do this they should call checkRetryable() AFTER rolling back the db transaction, - // and then they should catch RetryException and call themselves recursively. See commitTxn for an example. - public TxnHandler() { } @@ -385,123 +266,86 @@ public TxnHandler() { * This is logically part of c'tor and must be called prior to any other method. * Not physically part of c'tor due to use of reflection */ - public void setConf(Configuration conf){ + @Override + public void setConf(Configuration conf) { this.conf = conf; - int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS); - synchronized (TxnHandler.class) { - try (DataSourceProvider.DataSourceNameConfigurator configurator = - new DataSourceProvider.DataSourceNameConfigurator(conf, POOL_TX)) { - if (connPool == null) { - connPool = setupJdbcConnectionPool(conf, maxPoolSize); - } - if (connPoolMutex == null) { - configurator.resetName(POOL_MUTEX); - connPoolMutex = setupJdbcConnectionPool(conf, maxPoolSize); - } - } - if (dbProduct == null) { - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) { - determineDatabaseProduct(dbConn); - } catch (SQLException e) { - LOG.error("Unable to determine database product", e); - throw new RuntimeException(e); - } - } + if (!initialized) { + synchronized (TxnHandler.class) { + if (!initialized) { + try (DataSourceProvider.DataSourceNameConfigurator configurator = + new DataSourceProvider.DataSourceNameConfigurator(conf, POOL_TX)) { + int maxPoolSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS); + if (connPool == null) { + connPool = setupJdbcConnectionPool(conf, maxPoolSize); + } + if (connPoolMutex == null) { + configurator.resetName(POOL_MUTEX); + connPoolMutex = setupJdbcConnectionPool(conf, maxPoolSize); + } + if (connPoolCompactor == null) { + configurator.resetName(POOL_COMPACTOR); + connPoolCompactor = setupJdbcConnectionPool(conf, + MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CONNECTION_POOLING_MAX_CONNECTIONS)); + } + } + if (dbProduct == null) { + try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPool)) { + determineDatabaseProduct(dbConn); + } catch (SQLException e) { + LOG.error("Unable to determine database product", e); + throw new RuntimeException(e); + } + } + if (sqlGenerator == null) { + sqlGenerator = new SQLGenerator(dbProduct, conf); + } + + initJdbcResource(); - if (sqlGenerator == null) { - sqlGenerator = new SQLGenerator(dbProduct, conf); + try { + TxnHandler.ConfVars.init(this::checkIfTableIsUsable, conf); + } catch (Exception e) { + String msg = "Error during TxnHandler initialization, " + e.getMessage(); + LOG.error(msg); + throw e; + } + initialized = true; + } } - - if (jdbcResource == null) { - jdbcResource = new MultiDataSourceJdbcResource(dbProduct); - jdbcResource.registerDataSource(POOL_TX, connPool); - jdbcResource.registerDataSource(POOL_MUTEX, connPoolMutex); - } } - numOpenTxns = Metrics.getOrCreateGauge(MetricsConstants.NUM_OPEN_TXNS); + initJdbcResource(); - timeout = MetastoreConf.getTimeVar(conf, ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS); - replicationTxnTimeout = MetastoreConf.getTimeVar(conf, ConfVars.REPL_TXN_TIMEOUT, TimeUnit.MILLISECONDS); - retryInterval = MetastoreConf.getTimeVar(conf, ConfVars.HMS_HANDLER_INTERVAL, - TimeUnit.MILLISECONDS); - retryLimit = MetastoreConf.getIntVar(conf, ConfVars.HMS_HANDLER_ATTEMPTS); - deadlockRetryInterval = retryInterval / 10; - maxOpenTxns = MetastoreConf.getIntVar(conf, ConfVars.MAX_OPEN_TXNS); - maxBatchSize = MetastoreConf.getIntVar(conf, ConfVars.JDBC_MAX_BATCH_SIZE); + mutexAPI = new TxnStoreMutex(sqlGenerator, jdbcResource); - openTxnTimeOutMillis = MetastoreConf.getTimeVar(conf, ConfVars.TXN_OPENTXN_TIMEOUT, TimeUnit.MILLISECONDS); - - try { - useMinHistoryWriteId = checkIfTableIsUsable("MIN_HISTORY_WRITE_ID", - MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_WRITE_ID)); - - // override the config if table does not exists anymore - // this helps to roll out his feature when multiple HMS is accessing the same backend DB - useMinHistoryLevel = checkIfTableIsUsable("MIN_HISTORY_LEVEL", - MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_LEVEL)); - } catch (MetaException e) { - String msg = "Error during TxnHandler startup, " + e.getMessage(); - LOG.error(msg); - throw new RuntimeException(e); - } + numOpenTxns = Metrics.getOrCreateGauge(MetricsConstants.NUM_OPEN_TXNS); + + timeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS); + replicationTxnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.REPL_TXN_TIMEOUT, TimeUnit.MILLISECONDS); + maxOpenTxns = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.MAX_OPEN_TXNS); + openTxnTimeOutMillis = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_OPENTXN_TIMEOUT, TimeUnit.MILLISECONDS); try { transactionalListeners = MetaStoreServerUtils.getMetaStoreListeners( - TransactionalMetaStoreEventListener.class, - conf, MetastoreConf.getVar(conf, ConfVars.TRANSACTIONAL_EVENT_LISTENERS)); - } catch(MetaException e) { + TransactionalMetaStoreEventListener.class, + conf, MetastoreConf.getVar(conf, MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS)); + } catch (MetaException e) { String msg = "Unable to get transaction listeners, " + e.getMessage(); LOG.error(msg); throw new RuntimeException(e); } - sqlRetryHandler = new SqlRetryHandler(conf, jdbcResource.getDatabaseProduct()); - } - - /** - * Check if provided table is usable - * @return - * @throws MetaException - */ - private boolean checkIfTableIsUsable(String tableName, boolean configValue) throws MetaException { - if (!configValue) { - // don't check it if disabled - return false; - } - Connection dbConn = null; - boolean tableExists = true; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - try (Statement stmt = dbConn.createStatement()) { - // Dummy query to see if table exists - try (ResultSet rs = stmt.executeQuery("SELECT 1 FROM \"" + tableName + "\"")) { - rs.next(); - } - } - } catch (SQLException e) { - LOG.debug("Catching sql exception in " + tableName + " check", e); - if (dbProduct.isTableNotExistsError(e)) { - tableExists = false; - } else { - throw new MetaException( - "Unable to select from transaction database: " + getMessage(e) + StringUtils.stringifyException(e)); - } - } finally { - closeDbConn(dbConn); - } - return tableExists; + sqlRetryHandler = new SqlRetryHandler(conf, jdbcResource.getDatabaseProduct()); + txnLockManager = TransactionalRetryProxy.getProxy(sqlRetryHandler, jdbcResource, new DefaultTxnLockManager(jdbcResource)); } - + @Override - @RetrySemantics.ReadOnly public SqlRetryHandler getRetryHandler() { return sqlRetryHandler; } @Override - @RetrySemantics.ReadOnly public MultiDataSourceJdbcResource getJdbcResourceHolder() { return jdbcResource; } @@ -512,114 +356,21 @@ public Configuration getConf() { } @Override - @RetrySemantics.ReadOnly public GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException { - return getOpenTxnsList(true).toOpenTxnsInfoResponse(); + return jdbcResource.execute(new GetOpenTxnsListHandler(true, openTxnTimeOutMillis)) + .toOpenTxnsInfoResponse(); } @Override - @RetrySemantics.ReadOnly public GetOpenTxnsResponse getOpenTxns() throws MetaException { - return getOpenTxnsList(false).toOpenTxnsResponse(Arrays.asList(TxnType.READ_ONLY)); - } - - private GetOpenTxnsResponse getOpenTxns(Connection dbConn) throws MetaException { - return getOpenTxnsList(false, dbConn).toOpenTxnsResponse(Arrays.asList(TxnType.READ_ONLY)); + return jdbcResource.execute(new GetOpenTxnsListHandler(false, openTxnTimeOutMillis)) + .toOpenTxnsResponse(Collections.singletonList(TxnType.READ_ONLY)); } @Override - @RetrySemantics.ReadOnly public GetOpenTxnsResponse getOpenTxns(List excludeTxnTypes) throws MetaException { - return getOpenTxnsList(false).toOpenTxnsResponse(excludeTxnTypes); - } - - private OpenTxnList getOpenTxnsList(boolean infoFileds) throws MetaException { - Connection dbConn = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - return getOpenTxnsList(infoFileds, dbConn); - } catch (SQLException e) { - throw new MetaException( - "Unable to get a connection: " + getMessage(e) + StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - } - } - - private OpenTxnList getOpenTxnsList(boolean infoFields, Connection dbConn) throws MetaException { - try { - // We need to figure out the HighWaterMark and the list of open transactions. - Statement stmt = null; - ResultSet rs = null; - try { - /* - * This method need guarantees from - * {@link #openTxns(OpenTxnRequest)} and {@link #commitTxn(CommitTxnRequest)}. - * It will look at the TXNS table and find each transaction between the max(txn_id) as HighWaterMark - * and the max(txn_id) before the TXN_OPENTXN_TIMEOUT period as LowWaterMark. - * Every transaction that is not found between these will be considered as open, since it may appear later. - * openTxns must ensure, that no new transaction will be opened with txn_id below LWM and - * commitTxn must ensure, that no committed transaction will be removed before the time period expires. - */ - stmt = dbConn.createStatement(); - List txnInfos = new ArrayList<>(); - String txnsQuery = String.format(infoFields ? OpenTxn.OPEN_TXNS_INFO_QUERY : OpenTxn.OPEN_TXNS_QUERY, - getEpochFn(dbProduct)); - LOG.debug("Going to execute query<{}>", txnsQuery); - rs = stmt.executeQuery(txnsQuery); - /* - * We can use the maximum txn_id from the TXNS table as high water mark, since the commitTxn and the Initiator - * guarantees, that the transaction with the highest txn_id will never be removed from the TXNS table. - * If there is a pending openTxns, that is already acquired it's sequenceId but not yet committed the insert - * into the TXNS table, will have either a lower txn_id than HWM and will be listed in the openTxn list, - * or will have a higher txn_id and don't effect this getOpenTxns() call. - */ - long hwm = 0; - long openTxnLowBoundary = 0; - - while (rs.next()) { - long txnId = rs.getLong(1); - long age = rs.getLong(4); - hwm = txnId; - if (age < getOpenTxnTimeOutMillis()) { - // We will consider every gap as an open transaction from the previous txnId - openTxnLowBoundary++; - while (txnId > openTxnLowBoundary) { - // Add an empty open transaction for every missing value - txnInfos.add(new OpenTxn(openTxnLowBoundary, TxnStatus.OPEN, TxnType.DEFAULT)); - LOG.debug("Open transaction added for missing value in TXNS {}", - JavaUtils.txnIdToString(openTxnLowBoundary)); - openTxnLowBoundary++; - } - } else { - openTxnLowBoundary = txnId; - } - TxnStatus state = TxnStatus.fromString(rs.getString(2)); - if (state == TxnStatus.COMMITTED) { - // This is only here, to avoid adding this txnId as possible gap - continue; - } - OpenTxn txnInfo = new OpenTxn(txnId, state, TxnType.findByValue(rs.getInt(3))); - if (infoFields) { - txnInfo.setUser(rs.getString(5)); - txnInfo.setHost(rs.getString(6)); - txnInfo.setStartedTime(rs.getLong(7)); - txnInfo.setLastHeartBeatTime(rs.getLong(8)); - } - txnInfos.add(txnInfo); - } - LOG.debug("Got OpenTxnList with hwm: {} and openTxnList size {}.", hwm, txnInfos.size()); - return new OpenTxnList(hwm, txnInfos); - } catch (SQLException e) { - checkRetryable(e, "getOpenTxnsList"); - throw new MetaException( - "Unable to select from transaction database: " + getMessage(e) + StringUtils.stringifyException(e)); - } finally { - close(rs, stmt, null); - } - } catch (RetryException e) { - return getOpenTxnsList(infoFields, dbConn); - } + return jdbcResource.execute(new GetOpenTxnsListHandler(false, openTxnTimeOutMillis)) + .toOpenTxnsResponse(excludeTxnTypes); } /** @@ -627,7 +378,6 @@ private OpenTxnList getOpenTxnsList(boolean infoFields, Connection dbConn) throw * Worst case, it will leave an open txn which will timeout. */ @Override - @RetrySemantics.Idempotent public OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException { if (!tooManyOpenTxns && numOpenTxns.get() >= maxOpenTxns) { tooManyOpenTxns = true; @@ -648,260 +398,75 @@ public OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException { throw new MetaException("Invalid input for number of txns: " + numTxns); } - try { - Connection dbConn = null; - Statement stmt = null; - try { - /* - * To make {@link #getOpenTxns()}/{@link #getOpenTxnsInfo()} work correctly, this operation must ensure - * that looking at the TXNS table every open transaction could be identified below a given High Water Mark. - * One way to do it, would be to serialize the openTxns call with a S4U lock, but that would cause - * performance degradation with high transaction load. - * To enable parallel openTxn calls, we define a time period (TXN_OPENTXN_TIMEOUT) and consider every - * transaction missing from the TXNS table in that period open, and prevent opening transaction outside - * the period. - * Example: At t[0] there is one open transaction in the TXNS table, T[1]. - * T[2] acquires the next sequence at t[1] but only commits into the TXNS table at t[10]. - * T[3] acquires its sequence at t[2], and commits into the TXNS table at t[3]. - * Then T[3] calculates it’s snapshot at t[4] and puts T[1] and also T[2] in the snapshot’s - * open transaction list. T[1] because it is presented as open in TXNS, - * T[2] because it is a missing sequence. - * - * In the current design, there can be several metastore instances running in a given Warehouse. - * This makes ideas like reserving a range of IDs to save trips to DB impossible. For example, - * a client may go to MS1 and start a transaction with ID 500 to update a particular row. - * Now the same client will start another transaction, except it ends up on MS2 and may get - * transaction ID 400 and update the same row. Now the merge that happens to materialize the snapshot - * on read will thing the version of the row from transaction ID 500 is the latest one. - * - * Longer term we can consider running Active-Passive MS (at least wrt to ACID operations). This - * set could support a write-through cache for added performance. - */ - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - /* - * The openTxn and commitTxn must be mutexed, when committing a not read only transaction. - * This is achieved by requesting a shared table lock here, and an exclusive one at commit. - * Since table locks are working in Derby, we don't need the lockInternal call here. - * Example: Suppose we have two transactions with update like x = x+1. - * We have T[3,3] that was using a value from a snapshot with T[2,2]. If we allow committing T[3,3] - * and opening T[4] parallel it is possible, that T[4] will be using the value from a snapshot with T[2,2], - * and we will have a lost update problem - */ - acquireTxnLock(stmt, true); - // Measure the time from acquiring the sequence value, till committing in the TXNS table - StopWatch generateTransactionWatch = new StopWatch(); - generateTransactionWatch.start(); - - List txnIds = openTxns(dbConn, rqst); + /* + * To make {@link #getOpenTxns()}/{@link #getOpenTxnsInfo()} work correctly, this operation must ensure + * that looking at the TXNS table every open transaction could be identified below a given High Water Mark. + * One way to do it, would be to serialize the openTxns call with a S4U lock, but that would cause + * performance degradation with high transaction load. + * To enable parallel openTxn calls, we define a time period (TXN_OPENTXN_TIMEOUT) and consider every + * transaction missing from the TXNS table in that period open, and prevent opening transaction outside + * the period. + * Example: At t[0] there is one open transaction in the TXNS table, T[1]. + * T[2] acquires the next sequence at t[1] but only commits into the TXNS table at t[10]. + * T[3] acquires its sequence at t[2], and commits into the TXNS table at t[3]. + * Then T[3] calculates it’s snapshot at t[4] and puts T[1] and also T[2] in the snapshot’s + * open transaction list. T[1] because it is presented as open in TXNS, + * T[2] because it is a missing sequence. + * + * In the current design, there can be several metastore instances running in a given Warehouse. + * This makes ideas like reserving a range of IDs to save trips to DB impossible. For example, + * a client may go to MS1 and start a transaction with ID 500 to update a particular row. + * Now the same client will start another transaction, except it ends up on MS2 and may get + * transaction ID 400 and update the same row. Now the merge that happens to materialize the snapshot + * on read will thing the version of the row from transaction ID 500 is the latest one. + * + * Longer term we can consider running Active-Passive MS (at least wrt to ACID operations). This + * set could support a write-through cache for added performance. + */ + /* + * The openTxn and commitTxn must be mutexed, when committing a not read only transaction. + * This is achieved by requesting a shared table lock here, and an exclusive one at commit. + * Since table locks are working in Derby, we don't need the lockInternal call here. + * Example: Suppose we have two transactions with update like x = x+1. + * We have T[3,3] that was using a value from a snapshot with T[2,2]. If we allow committing T[3,3] + * and opening T[4] parallel it is possible, that T[4] will be using the value from a snapshot with T[2,2], + * and we will have a lost update problem + */ + acquireTxnLock(true); + // Measure the time from acquiring the sequence value, till committing in the TXNS table + StopWatch generateTransactionWatch = new StopWatch(); + generateTransactionWatch.start(); - LOG.debug("Going to commit"); - dbConn.commit(); - generateTransactionWatch.stop(); - long elapsedMillis = generateTransactionWatch.getTime(TimeUnit.MILLISECONDS); - TxnType txnType = rqst.isSetTxn_type() ? rqst.getTxn_type() : TxnType.DEFAULT; - if (txnType != TxnType.READ_ONLY && elapsedMillis >= openTxnTimeOutMillis) { - /* - * The commit was too slow, we can not allow this to continue (except if it is read only, - * since that can not cause dirty reads). - * When calculating the snapshot for a given transaction, we look back for possible open transactions - * (that are not yet committed in the TXNS table), for TXN_OPENTXN_TIMEOUT period. - * We can not allow a write transaction, that was slower than TXN_OPENTXN_TIMEOUT to continue, - * because there can be other transactions running, that didn't considered this transactionId open, - * this could cause dirty reads. - */ - LOG.error("OpenTxnTimeOut exceeded commit duration {}, deleting transactionIds: {}", elapsedMillis, txnIds); - deleteInvalidOpenTransactions(dbConn, txnIds); - dbConn.commit(); - /* - * We do not throw RetryException directly, to not circumvent the max retry limit - */ - throw new SQLException("OpenTxnTimeOut exceeded", MANUAL_RETRY); - } - return new OpenTxnsResponse(txnIds); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "openTxns(" + rqst + ")"); - throw new MetaException("Unable to select from transaction database " + StringUtils.stringifyException(e)); - } finally { - close(null, stmt, dbConn); - } - } catch (RetryException e) { - return openTxns(rqst); - } - } + List txnIds = new OpenTxnsFunction(rqst, openTxnTimeOutMillis, transactionalListeners).execute(jdbcResource); - private List openTxns(Connection dbConn, OpenTxnRequest rqst) - throws SQLException, MetaException { - int numTxns = rqst.getNum_txns(); - // Make sure the user has not requested an insane amount of txns. - int maxTxns = MetastoreConf.getIntVar(conf, ConfVars.TXN_MAX_OPEN_BATCH); - if (numTxns > maxTxns) { - numTxns = maxTxns; - } - List insertPreparedStmts = null; + LOG.debug("Going to commit"); + jdbcResource.getTransactionManager().getActiveTransaction().createSavepoint(); + generateTransactionWatch.stop(); + long elapsedMillis = generateTransactionWatch.getTime(TimeUnit.MILLISECONDS); TxnType txnType = rqst.isSetTxn_type() ? rqst.getTxn_type() : TxnType.DEFAULT; - boolean isReplayedReplTxn = txnType == TxnType.REPL_CREATED; - boolean isHiveReplTxn = rqst.isSetReplPolicy() && txnType == TxnType.DEFAULT; - try { - if (isReplayedReplTxn) { - assert rqst.isSetReplPolicy(); - List targetTxnIdList = getTargetTxnIdList(rqst.getReplPolicy(), rqst.getReplSrcTxnIds(), dbConn); - - if (!targetTxnIdList.isEmpty()) { - if (targetTxnIdList.size() != rqst.getReplSrcTxnIds().size()) { - LOG.warn("target txn id number {} is not matching with source txn id number {}", - targetTxnIdList, rqst.getReplSrcTxnIds()); - } - LOG.info("Target transactions {} are present for repl policy : {} and Source transaction id : {}", - targetTxnIdList.toString(), rqst.getReplPolicy(), rqst.getReplSrcTxnIds().toString()); - return targetTxnIdList; - } - } - - long minOpenTxnId = 0; - if (useMinHistoryLevel) { - minOpenTxnId = getMinOpenTxnIdWaterMark(dbConn); - } - - List txnIds = new ArrayList<>(numTxns); + if (txnType != TxnType.READ_ONLY && elapsedMillis >= openTxnTimeOutMillis) { /* - * The getGeneratedKeys are not supported in every dbms, after executing a multi line insert. - * But it is supported in every used dbms for single line insert, even if the metadata says otherwise. - * If the getGeneratedKeys are not supported first we insert a random batchId in the TXN_META_INFO field, - * then the keys are selected beck with that batchid. + * The commit was too slow, we can not allow this to continue (except if it is read only, + * since that can not cause dirty reads). + * When calculating the snapshot for a given transaction, we look back for possible open transactions + * (that are not yet committed in the TXNS table), for TXN_OPENTXN_TIMEOUT period. + * We can not allow a write transaction, that was slower than TXN_OPENTXN_TIMEOUT to continue, + * because there can be other transactions running, that didn't considered this transactionId open, + * this could cause dirty reads. */ - boolean genKeySupport = dbProduct.supportsGetGeneratedKeys(); - genKeySupport = genKeySupport || (numTxns == 1); - - String insertQuery = String.format(TXNS_INSERT_QRY, getEpochFn(dbProduct), getEpochFn(dbProduct)); - LOG.debug("Going to execute insert <{}>", insertQuery); - try (PreparedStatement ps = dbConn.prepareStatement(insertQuery, new String[] {"TXN_ID"})) { - String state = genKeySupport ? TxnStatus.OPEN.getSqlConst() : TXN_TMP_STATE; - if (numTxns == 1) { - ps.setString(1, state); - ps.setString(2, rqst.getUser()); - ps.setString(3, rqst.getHostname()); - ps.setInt(4, txnType.getValue()); - txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(dbConn, genKeySupport, ps, false)); - } else { - for (int i = 0; i < numTxns; ++i) { - ps.setString(1, state); - ps.setString(2, rqst.getUser()); - ps.setString(3, rqst.getHostname()); - ps.setInt(4, txnType.getValue()); - ps.addBatch(); - - if ((i + 1) % maxBatchSize == 0) { - txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(dbConn, genKeySupport, ps, true)); - } - } - if (numTxns % maxBatchSize != 0) { - txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(dbConn, genKeySupport, ps, true)); - } - } - } - - assert txnIds.size() == numTxns; - - addTxnToMinHistoryLevel(dbConn, txnIds, minOpenTxnId); - - if (isReplayedReplTxn) { - List rowsRepl = new ArrayList<>(numTxns); - List params = Collections.singletonList(rqst.getReplPolicy()); - List> paramsList = new ArrayList<>(numTxns); - for (int i = 0; i < numTxns; i++) { - rowsRepl.add("?," + rqst.getReplSrcTxnIds().get(i) + "," + txnIds.get(i)); - paramsList.add(params); - } - - insertPreparedStmts = sqlGenerator.createInsertValuesPreparedStmt(dbConn, - "\"REPL_TXN_MAP\" (\"RTM_REPL_POLICY\", \"RTM_SRC_TXN_ID\", \"RTM_TARGET_TXN_ID\")", rowsRepl, - paramsList); - for (PreparedStatement pst : insertPreparedStmts) { - pst.execute(); - } - } + LOG.error("OpenTxnTimeOut exceeded commit duration {}, deleting transactionIds: {}", elapsedMillis, txnIds); - if (transactionalListeners != null && !isHiveReplTxn) { - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.OPEN_TXN, new OpenTxnEvent(txnIds, txnType), dbConn, sqlGenerator); - } - return txnIds; - } finally { - if (insertPreparedStmts != null) { - for (PreparedStatement pst : insertPreparedStmts) { - pst.close(); - } - } - } - } - - private List executeTxnInsertBatchAndExtractGeneratedKeys(Connection dbConn, boolean genKeySupport, - PreparedStatement ps, boolean batch) throws SQLException { - List txnIds = new ArrayList<>(); - if (batch) { - ps.executeBatch(); - } else { - // For slight performance advantage we do not use the executeBatch, when we only have one row - ps.execute(); - } - if (genKeySupport) { - try (ResultSet generatedKeys = ps.getGeneratedKeys()) { - while (generatedKeys.next()) { - txnIds.add(generatedKeys.getLong(1)); - } - } - } else { - try (PreparedStatement pstmt = - dbConn.prepareStatement("SELECT \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = ?")) { - pstmt.setString(1, TXN_TMP_STATE); - try (ResultSet rs = pstmt.executeQuery()) { - while (rs.next()) { - txnIds.add(rs.getLong(1)); - } - } - } - try (PreparedStatement pstmt = dbConn - .prepareStatement("UPDATE \"TXNS\" SET \"TXN_STATE\" = ? WHERE \"TXN_STATE\" = ?")) { - pstmt.setString(1, TxnStatus.OPEN.getSqlConst()); - pstmt.setString(2, TXN_TMP_STATE); - pstmt.executeUpdate(); + if (!txnIds.isEmpty()) { + deleteInvalidOpenTransactions(txnIds); } - } - return txnIds; - } - private void deleteInvalidOpenTransactions(Connection dbConn, List txnIds) throws MetaException { - if (txnIds.size() == 0) { - return; + /* + * We cannot throw SQLException directly, as it is not in the throws clause + */ + throw new SqlRetryException("OpenTxnTimeOut exceeded"); } - try { - Statement stmt = null; - try { - stmt = dbConn.createStatement(); - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - prefix.append("DELETE FROM \"TXNS\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnIds, "\"TXN_ID\"", false, false); - executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize); - LOG.info("Removed transactions: ({}) from TXNS", txnIds); - - removeTxnsFromMinHistoryLevel(dbConn, txnIds); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "deleteInvalidOpenTransactions(" + txnIds + ")"); - throw new MetaException("Unable to select from transaction database " + StringUtils.stringifyException(e)); - } finally { - closeStmt(stmt); - } - } catch (RetryException ex) { - deleteInvalidOpenTransactions(dbConn, txnIds); - } + return new OpenTxnsResponse(txnIds); } @Override @@ -914,518 +479,89 @@ public void setOpenTxnTimeOutMillis(long openTxnTimeOutMillis) { TxnHandler.openTxnTimeOutMillis = openTxnTimeOutMillis; } - protected long getOpenTxnTimeoutLowBoundaryTxnId(Connection dbConn) throws MetaException, SQLException { - long maxTxnId; - String s = - "SELECT MAX(\"TXN_ID\") FROM \"TXNS\" WHERE \"TXN_STARTED\" < (" + getEpochFn(dbProduct) + " - " - + openTxnTimeOutMillis + ")"; - try (Statement stmt = dbConn.createStatement()) { - LOG.debug("Going to execute query <{}>", s); - try (ResultSet maxTxnIdRs = stmt.executeQuery(s)) { - maxTxnIdRs.next(); - maxTxnId = maxTxnIdRs.getLong(1); - if (maxTxnIdRs.wasNull()) { - /* - * TXNS always contains at least one transaction, - * the row where txnid = (select max(txnid) where txn_started < epoch - TXN_OPENTXN_TIMEOUT) is never deleted - */ - throw new MetaException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)"); - } - } + @Override + public long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException { + List targetTxnIds =jdbcResource.execute(new TargetTxnIdListHandler(replPolicy, Collections.singletonList(sourceTxnId))); + if (targetTxnIds.isEmpty()) { + LOG.info("Txn {} not present for repl policy {}", sourceTxnId, replPolicy); + return -1; } - return maxTxnId; + assert (targetTxnIds.size() == 1); + return targetTxnIds.get(0); } - private long getHighWaterMark(Statement stmt) throws SQLException, MetaException { - String s = "SELECT MAX(\"TXN_ID\") FROM \"TXNS\""; - LOG.debug("Going to execute query <{}>", s); - long maxOpenTxnId; - try (ResultSet maxOpenTxnIdRs = stmt.executeQuery(s)) { - maxOpenTxnIdRs.next(); - maxOpenTxnId = maxOpenTxnIdRs.getLong(1); - if (maxOpenTxnIdRs.wasNull()) { - /* - * TXNS always contains at least one transaction, - * the row where txnid = (select max(txnid) where txn_started < epoch - TXN_OPENTXN_TIMEOUT) is never deleted - */ - throw new MetaException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)"); + @Override + public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException { + TxnType txnType = new AbortTxnFunction(rqst).execute(jdbcResource); + if (txnType != null) { + if (transactionalListeners != null && (!rqst.isSetReplPolicy() || !TxnType.DEFAULT.equals(rqst.getTxn_type()))) { + List dbsUpdated = getTxnDbsUpdated(rqst.getTxnid()); + MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, EventMessage.EventType.ABORT_TXN, + new AbortTxnEvent(rqst.getTxnid(), txnType, null, dbsUpdated), jdbcResource.getConnection(), sqlGenerator); } } - return maxOpenTxnId; } - private List getTargetTxnIdList(String replPolicy, List sourceTxnIdList, Connection dbConn) - throws SQLException { - PreparedStatement pst = null; - ResultSet rs = null; + @Override + public void abortTxns(AbortTxnsRequest rqst) throws MetaException { + List txnIds = rqst.getTxn_ids(); + TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE; + if (rqst.isSetErrorCode()) { + txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode()); + } + + List queries = new ArrayList<>(); + StringBuilder prefix = + new StringBuilder("SELECT \"TXN_ID\", \"TXN_TYPE\" from \"TXNS\" where \"TXN_STATE\" = ") + .append(TxnStatus.OPEN) + .append(" and \"TXN_TYPE\" != ").append(TxnType.READ_ONLY.getValue()).append(" and "); + + TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), + txnIds, "\"TXN_ID\"", false, false); + + Connection dbConn = jdbcResource.getConnection(); try { - List inQueries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - List targetTxnIdList = new ArrayList<>(); - prefix.append("SELECT \"RTM_TARGET_TXN_ID\" FROM \"REPL_TXN_MAP\" WHERE "); - suffix.append(" AND \"RTM_REPL_POLICY\" = ?"); - TxnUtils.buildQueryWithINClause(conf, inQueries, prefix, suffix, sourceTxnIdList, - "\"RTM_SRC_TXN_ID\"", false, false); - List params = Arrays.asList(replPolicy); - for (String query : inQueries) { - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute select <" + query.replace("?", "{}") + ">", quoteString(replPolicy)); + Map nonReadOnlyTxns = new HashMap<>(); + for (String query : queries) { + LOG.debug("Going to execute query <{}>", query); + try (Statement stmt = dbConn.createStatement(); ResultSet rs = stmt.executeQuery(sqlGenerator.addForUpdateClause(query))) { + while (rs.next()) { + TxnType txnType = TxnType.findByValue(rs.getInt(2)); + nonReadOnlyTxns.put(rs.getLong(1), txnType); + } } - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, params); - rs = pst.executeQuery(); - while (rs.next()) { - targetTxnIdList.add(rs.getLong(1)); + } + int numAborted = new AbortTxnsFunction(txnIds, false, false, false, txnErrorMsg).execute(jdbcResource); + if (numAborted != txnIds.size()) { + LOG.warn( + "Abort Transactions command only aborted {} out of {} transactions. It's possible that the other" + + " {} transactions have been aborted or committed, or the transaction ids are invalid.", + numAborted, txnIds.size(), (txnIds.size() - numAborted)); + } + + if (transactionalListeners != null) { + for (Long txnId : txnIds) { + List dbsUpdated = getTxnDbsUpdated(txnId); + MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, + EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId, + nonReadOnlyTxns.getOrDefault(txnId, TxnType.READ_ONLY), null, dbsUpdated), dbConn, sqlGenerator); } - closeStmt(pst); } - LOG.debug("targetTxnid for srcTxnId " + sourceTxnIdList.toString() + " is " + targetTxnIdList.toString()); - return targetTxnIdList; - } finally { - closeStmt(pst); - close(rs); - } - } - - @Override - @RetrySemantics.Idempotent - public long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException { - try { - Connection dbConn = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - List targetTxnIds = getTargetTxnIdList(replPolicy, Collections.singletonList(sourceTxnId), dbConn); - if (targetTxnIds.isEmpty()) { - LOG.info("Txn {} not present for repl policy {}", sourceTxnId, replPolicy); - return -1; - } - assert (targetTxnIds.size() == 1); - return targetTxnIds.get(0); - } catch (SQLException e) { - checkRetryable(e, "getTargetTxnId(" + replPolicy + sourceTxnId + ")"); - throw new MetaException("Unable to get target transaction id " - + StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - unlockInternal(); - } - } catch (RetryException e) { - return getTargetTxnId(replPolicy, sourceTxnId); - } - } - - private Set getDbNamesForReplayedTxns(Connection dbConn, List targetTxnIds) throws SQLException { - Set dbNames = new HashSet<>(); - if (targetTxnIds.isEmpty()) { - return dbNames; - } - PreparedStatement pst = null; - ResultSet rs = null; - try { - List inQueries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - prefix.append("SELECT \"RTM_REPL_POLICY\" FROM \"REPL_TXN_MAP\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, inQueries, prefix, new StringBuilder(), targetTxnIds, - "\"RTM_TARGET_TXN_ID\"", false, false); - for (String query : inQueries) { - LOG.debug("Going to execute select <{}>", query); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, null); - rs = pst.executeQuery(); - while (rs.next()) { - dbNames.add(MetaStoreUtils.getDbNameFromReplPolicy(rs.getString(1))); - } - } - return dbNames; - } finally { - closeStmt(pst); - close(rs); - } - } - - private void deleteReplTxnMapEntry(Connection dbConn, long sourceTxnId, String replPolicy) throws SQLException { - String s = "DELETE FROM \"REPL_TXN_MAP\" WHERE \"RTM_SRC_TXN_ID\" = " + sourceTxnId + " AND \"RTM_REPL_POLICY\" = ?"; - try (PreparedStatement pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, Arrays.asList(replPolicy))) { - LOG.info("Going to execute <" + s.replace("?", "{}") + ">", quoteString(replPolicy)); - pst.executeUpdate(); - } - } - - @Override - @RetrySemantics.Idempotent - public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException { - long txnid = rqst.getTxnid(); - TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE; - long sourceTxnId = -1; - boolean isReplayedReplTxn = TxnType.REPL_CREATED.equals(rqst.getTxn_type()); - boolean isHiveReplTxn = rqst.isSetReplPolicy() && TxnType.DEFAULT.equals(rqst.getTxn_type()); - try { - Connection dbConn = null; - Statement stmt = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - - if (isReplayedReplTxn) { - assert (rqst.isSetReplPolicy()); - sourceTxnId = rqst.getTxnid(); - List targetTxnIds = getTargetTxnIdList(rqst.getReplPolicy(), - Collections.singletonList(sourceTxnId), dbConn); - if (targetTxnIds.isEmpty()) { - // Idempotent case where txn was already closed or abort txn event received without - // corresponding open txn event. - LOG.info("Target txn id is missing for source txn id : {} and repl policy {}", sourceTxnId, - rqst.getReplPolicy()); - return; - } - assert targetTxnIds.size() == 1; - txnid = targetTxnIds.get(0); - } - - TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid); - if (txnType == null) { - TxnStatus status = findTxnState(txnid, stmt); - if (status == TxnStatus.ABORTED) { - if (isReplayedReplTxn) { - // in case of replication, idempotent is taken care by getTargetTxnId - LOG.warn("Invalid state ABORTED for transactions started using replication replay task"); - deleteReplTxnMapEntry(dbConn, sourceTxnId, rqst.getReplPolicy()); - } - LOG.info("abortTxn({}) requested by it is already {}", JavaUtils.txnIdToString(txnid), TxnStatus.ABORTED); - return; - } - raiseTxnUnexpectedState(status, txnid); - } - - if (isReplayedReplTxn) { - txnErrorMsg = TxnErrorMsg.ABORT_REPLAYED_REPL_TXN; - } else if (isHiveReplTxn) { - txnErrorMsg = TxnErrorMsg.ABORT_DEFAULT_REPL_TXN; - } else if (rqst.isSetErrorCode()) { - txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode()); - } - - abortTxns(dbConn, Collections.singletonList(txnid), true, isReplayedReplTxn, txnErrorMsg); - - if (isReplayedReplTxn) { - deleteReplTxnMapEntry(dbConn, sourceTxnId, rqst.getReplPolicy()); - } - - if (transactionalListeners != null && !isHiveReplTxn) { - List dbsUpdated = getTxnDbsUpdated(txnid, dbConn); - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.ABORT_TXN, - new AbortTxnEvent(txnid, txnType, null, dbsUpdated), dbConn, sqlGenerator); - } - - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "abortTxn(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " - + StringUtils.stringifyException(e)); - } finally { - close(null, stmt, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - abortTxn(rqst); - } - } - - @Override - @RetrySemantics.Idempotent - public void abortTxns(AbortTxnsRequest rqst) throws MetaException { - List txnIds = rqst.getTxn_ids(); - TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE; - if (rqst.isSetErrorCode()) { - txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode()); - } - try { - Connection dbConn = null; - Statement stmt = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - - List queries = new ArrayList<>(); - StringBuilder prefix = - new StringBuilder("SELECT \"TXN_ID\", \"TXN_TYPE\" from \"TXNS\" where \"TXN_STATE\" = ") - .append(TxnStatus.OPEN) - .append(" and \"TXN_TYPE\" != ").append(TxnType.READ_ONLY.getValue()).append(" and "); - - TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), - txnIds, "\"TXN_ID\"", false, false); - - Map nonReadOnlyTxns = new HashMap<>(); - for (String query : queries) { - LOG.debug("Going to execute query <{}>", query); - try (ResultSet rs = stmt.executeQuery(sqlGenerator.addForUpdateClause(query))) { - while (rs.next()) { - TxnType txnType = TxnType.findByValue(rs.getInt(2)); - nonReadOnlyTxns.put(rs.getLong(1), txnType); - } - } - } - int numAborted = abortTxns(dbConn, txnIds, false, false, txnErrorMsg); - if (numAborted != txnIds.size()) { - LOG.warn( - "Abort Transactions command only aborted {} out of {} transactions. It's possible that the other" - + " {} transactions have been aborted or committed, or the transaction ids are invalid.", - numAborted, txnIds.size(), (txnIds.size() - numAborted)); - } - - if (transactionalListeners != null){ - for (Long txnId : txnIds) { - List dbsUpdated = getTxnDbsUpdated(txnId, dbConn); - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId, - nonReadOnlyTxns.getOrDefault(txnId, TxnType.READ_ONLY), null, dbsUpdated), dbConn, sqlGenerator); - } - } - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "abortTxns(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " - + StringUtils.stringifyException(e)); - } finally { - closeStmt(stmt); - closeDbConn(dbConn); - } - } catch (RetryException e) { - abortTxns(rqst); - } - } - - private long getDatabaseId(Connection dbConn, String database, String catalog) throws SQLException, MetaException { - ResultSet rs = null; - PreparedStatement pst = null; - try { - String query = "select \"DB_ID\" from \"DBS\" where \"NAME\" = ? and \"CTLG_NAME\" = ?"; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, Arrays.asList(database, catalog)); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">", - quoteString(database), quoteString(catalog)); - } - rs = pst.executeQuery(); - if (!rs.next()) { - throw new MetaException("DB with name " + database + " does not exist in catalog " + catalog); - } - return rs.getLong(1); - } finally { - close(rs); - closeStmt(pst); - } - } - - private void updateDatabaseProp(Connection dbConn, String database, - long dbId, String prop, String propValue) throws SQLException { - ResultSet rs = null; - PreparedStatement pst = null; - try { - String query = "SELECT \"PARAM_VALUE\" FROM \"DATABASE_PARAMS\" WHERE \"PARAM_KEY\" = " + - "'" + prop + "' AND \"DB_ID\" = " + dbId; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, null); - rs = pst.executeQuery(); - query = null; - if (!rs.next()) { - query = "INSERT INTO \"DATABASE_PARAMS\" VALUES ( " + dbId + " , '" + prop + "' , ? )"; - } else if (!rs.getString(1).equals(propValue)) { - query = "UPDATE \"DATABASE_PARAMS\" SET \"PARAM_VALUE\" = ? WHERE \"DB_ID\" = " + dbId + - " AND \"PARAM_KEY\" = '" + prop + "'"; - } - closeStmt(pst); - if (query == null) { - LOG.info("Database property: {} with value: {} already updated for db: {}", prop, propValue, database); - return; - } - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, Arrays.asList(propValue)); - if (LOG.isDebugEnabled()) { - LOG.debug("Updating " + prop + " for db: " + database + " <" + query.replace("?", "{}") + ">", propValue); - } - if (pst.executeUpdate() != 1) { - //only one row insert or update should happen - throw new RuntimeException("DATABASE_PARAMS is corrupted for database: " + database); - } - } finally { - close(rs); - closeStmt(pst); - } - } - - private void markDbAsReplIncompatible(Connection dbConn, String database) throws SQLException, MetaException { - Statement stmt = null; - try { - stmt = dbConn.createStatement(); - String catalog = MetaStoreUtils.getDefaultCatalog(conf); - String s = sqlGenerator.getDbProduct().getPrepareTxnStmt(); - if (s != null) { - stmt.execute(s); - } - long dbId = getDatabaseId(dbConn, database, catalog); - updateDatabaseProp(dbConn, database, dbId, ReplConst.REPL_INCOMPATIBLE, ReplConst.TRUE); - } finally { - closeStmt(stmt); - } - } - - private void updateReplId(Connection dbConn, ReplLastIdInfo replLastIdInfo) throws SQLException, MetaException { - PreparedStatement pst = null; - PreparedStatement pstInt = null; - ResultSet rs = null; - ResultSet prs = null; - Statement stmt = null; - String query; - List params; - String lastReplId = Long.toString(replLastIdInfo.getLastReplId()); - String catalog = replLastIdInfo.isSetCatalog() ? normalizeIdentifier(replLastIdInfo.getCatalog()) : - MetaStoreUtils.getDefaultCatalog(conf); - String db = normalizeIdentifier(replLastIdInfo.getDatabase()); - String table = replLastIdInfo.isSetTable() ? normalizeIdentifier(replLastIdInfo.getTable()) : null; - List partList = replLastIdInfo.isSetPartitionList() ? replLastIdInfo.getPartitionList() : null; - - try { - stmt = dbConn.createStatement(); - - String s = sqlGenerator.getDbProduct().getPrepareTxnStmt(); - if (s != null) { - stmt.execute(s); - } - - long dbId = getDatabaseId(dbConn, db, catalog); - - // not used select for update as it will be updated by single thread only from repl load - updateDatabaseProp(dbConn, db, dbId, ReplConst.REPL_TARGET_TABLE_PROPERTY, lastReplId); - - if (table == null) { - // if only database last repl id to be updated. - return; - } - - query = "SELECT \"TBL_ID\" FROM \"TBLS\" WHERE \"TBL_NAME\" = ? AND \"DB_ID\" = " + dbId; - params = Arrays.asList(table); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">", quoteString(table)); - } - - rs = pst.executeQuery(); - if (!rs.next()) { - throw new MetaException("Table with name " + table + " does not exist in db " + catalog + "." + db); - } - long tblId = rs.getLong(1); - rs.close(); - pst.close(); - - // select for update is not required as only one task will update this during repl load. - rs = stmt.executeQuery("SELECT \"PARAM_VALUE\" FROM \"TABLE_PARAMS\" WHERE \"PARAM_KEY\" = " + - "'repl.last.id' AND \"TBL_ID\" = " + tblId); - if (!rs.next()) { - query = "INSERT INTO \"TABLE_PARAMS\" VALUES ( " + tblId + " , 'repl.last.id' , ? )"; - } else { - query = "UPDATE \"TABLE_PARAMS\" SET \"PARAM_VALUE\" = ? WHERE \"TBL_ID\" = " + tblId + - " AND \"PARAM_KEY\" = 'repl.last.id'"; - } - rs.close(); - - params = Arrays.asList(lastReplId); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Updating repl id for table <" + query.replace("?", "{}") + ">", lastReplId); - } - if (pst.executeUpdate() != 1) { - //only one row insert or update should happen - throw new RuntimeException("TABLE_PARAMS is corrupted for table " + table); - } - pst.close(); - - if (partList == null || partList.isEmpty()) { - return; - } - - List questions = new ArrayList<>(); - for(int i = 0; i < partList.size(); ++i) { - questions.add("?"); - } - - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - prefix.append("SELECT \"PART_ID\" FROM \"PARTITIONS\" WHERE \"TBL_ID\" = " + tblId + " and "); - - // Populate the complete query with provided prefix and suffix - List counts = TxnUtils.buildQueryWithINClauseStrings(conf, queries, prefix, suffix, - questions, "\"PART_NAME\"", true, false); - int totalCount = 0; - assert queries.size() == counts.size(); - params = Arrays.asList(lastReplId); - for (int i = 0; i < queries.size(); i++) { - query = queries.get(i); - int partCount = counts.get(i); - - LOG.debug("Going to execute query {} with partitions {}", query, - partList.subList(totalCount, (totalCount + partCount))); - pst = dbConn.prepareStatement(query); - for (int j = 0; j < partCount; j++) { - pst.setString(j + 1, partList.get(totalCount + j)); - } - totalCount += partCount; - prs = pst.executeQuery(); - while (prs.next()) { - long partId = prs.getLong(1); - rs = stmt.executeQuery("SELECT \"PARAM_VALUE\" FROM \"PARTITION_PARAMS\" WHERE \"PARAM_KEY\" " + - " = 'repl.last.id' AND \"PART_ID\" = " + partId); - if (!rs.next()) { - query = "INSERT INTO \"PARTITION_PARAMS\" VALUES ( " + partId + " , 'repl.last.id' , ? )"; - } else { - query = "UPDATE \"PARTITION_PARAMS\" SET \"PARAM_VALUE\" = ? " + - " WHERE \"PART_ID\" = " + partId + " AND \"PARAM_KEY\" = 'repl.last.id'"; - } - rs.close(); - - pstInt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Updating repl id for part <" + query.replace("?", "{}") + ">", lastReplId); - } - if (pstInt.executeUpdate() != 1) { - //only one row insert or update should happen - throw new RuntimeException("PARTITION_PARAMS is corrupted for partition " + partId); - } - partCount--; - pstInt.close(); - } - if (partCount != 0) { - throw new MetaException(partCount + " Number of partition among " + partList + " does not exist in table " + - catalog + "." + db + "." + table); - } - prs.close(); - pst.close(); - } - } finally { - closeStmt(stmt); - close(rs); - close(prs); - closeStmt(pst); - closeStmt(pstInt); + } catch (SQLException e) { + throw new UncategorizedSQLException(null, null, e); } } /** * Concurrency/isolation notes: - * This is mutexed with {@link #openTxns(OpenTxnRequest)} and other {@link #commitTxn(CommitTxnRequest)} + * This is mutexed with {@link #openTxns(OpenTxnRequest)} and other commitTxn(CommitTxnRequest) * operations using select4update on NEXT_TXN_ID. Also, mutexes on TXNS table for specific txnid:X * see more notes below. * In order to prevent lost updates, we need to determine if any 2 transactions overlap. Each txn * is viewed as an interval [M,N]. M is the txnid and N is taken from the same NEXT_TXN_ID sequence * so that we can compare commit time of txn T with start time of txn S. This sequence can be thought of - * as a logical time counter. If S.commitTime < T.startTime, T and S do NOT overlap. - * + * as a logical time counter. If S.commitTime < T.startTime, T and S do NOT overlap. + *

* Motivating example: * Suppose we have multi-statement transactions T and S both of which are attempting x = x + 1 * In order to prevent lost update problem, then the non-overlapping txns must lock in the snapshot @@ -1442,248 +578,8 @@ private void updateReplId(Connection dbConn, ReplLastIdInfo replLastIdInfo) thro * 'x' would be updated to the same value by both, i.e. lost update. */ @Override - @RetrySemantics.Idempotent("No-op if already committed") public void commitTxn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException { - char isUpdateDelete = 'N'; - long txnid = rqst.getTxnid(); - long sourceTxnId = -1; - - boolean isReplayedReplTxn = TxnType.REPL_CREATED.equals(rqst.getTxn_type()); - boolean isHiveReplTxn = rqst.isSetReplPolicy() && TxnType.DEFAULT.equals(rqst.getTxn_type()); - //start a new transaction - jdbcResource.bindDataSource(POOL_TX); - try (TransactionContext context = jdbcResource.getTransactionManager().getTransaction(PROPAGATION_REQUIRED)) { - Connection dbConn = null; - Statement stmt = null; - Long commitId = null; - try { - lockInternal(); - //make sure we are using the connection bound to the transaction, so obtain it via DataSourceUtils.getConnection() - dbConn = jdbcResource.getConnection(); - stmt = dbConn.createStatement(); - - if (rqst.isSetReplLastIdInfo()) { - updateReplId(dbConn, rqst.getReplLastIdInfo()); - } - - if (isReplayedReplTxn) { - assert (rqst.isSetReplPolicy()); - sourceTxnId = rqst.getTxnid(); - List targetTxnIds = getTargetTxnIdList(rqst.getReplPolicy(), - Collections.singletonList(sourceTxnId), dbConn); - if (targetTxnIds.isEmpty()) { - // Idempotent case where txn was already closed or commit txn event received without - // corresponding open txn event. - LOG.info("Target txn id is missing for source txn id : {} and repl policy {}", sourceTxnId, - rqst.getReplPolicy()); - return; - } - assert targetTxnIds.size() == 1; - txnid = targetTxnIds.get(0); - } - - /** - * Runs at READ_COMMITTED with S4U on TXNS row for "txnid". S4U ensures that no other - * operation can change this txn (such acquiring locks). While lock() and commitTxn() - * should not normally run concurrently (for same txn) but could due to bugs in the client - * which could then corrupt internal transaction manager state. Also competes with abortTxn(). - */ - TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid); - if (txnType == null) { - //if here, txn was not found (in expected state) - TxnStatus actualTxnStatus = findTxnState(txnid, stmt); - if (actualTxnStatus == TxnStatus.COMMITTED) { - if (isReplayedReplTxn) { - // in case of replication, idempotent is taken care by getTargetTxnId - LOG.warn("Invalid state COMMITTED for transactions started using replication replay task"); - } - /** - * This makes the operation idempotent - * (assume that this is most likely due to retry logic) - */ - LOG.info("Nth commitTxn({}) msg", JavaUtils.txnIdToString(txnid)); - return; - } - raiseTxnUnexpectedState(actualTxnStatus, txnid); - } - - String conflictSQLSuffix = "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" IN (" + - OperationType.UPDATE + "," + OperationType.DELETE + ")"; - long tempCommitId = generateTemporaryId(); - - if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) { - acquireTxnLock(stmt, false); - commitId = getHighWaterMark(stmt); - - } else if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn) { - String writeSetInsertSql = "INSERT INTO \"WRITE_SET\" (\"WS_DATABASE\", \"WS_TABLE\", \"WS_PARTITION\"," + - " \"WS_TXNID\", \"WS_COMMIT_ID\", \"WS_OPERATION_TYPE\")" + - " SELECT DISTINCT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_TXNID\", " + tempCommitId + ", \"TC_OPERATION_TYPE\" "; - - if (isUpdateOrDelete(stmt, conflictSQLSuffix)) { - isUpdateDelete = 'Y'; - //if here it means currently committing txn performed update/delete and we should check WW conflict - /** - * "select distinct" is used below because - * 1. once we get to multi-statement txns, we only care to record that something was updated once - * 2. if {@link #addDynamicPartitions(AddDynamicPartitions)} is retried by caller it may create - * duplicate entries in TXN_COMPONENTS - * but we want to add a PK on WRITE_SET which won't have unique rows w/o this distinct - * even if it includes all of its columns - * - * First insert into write_set using a temporary commitID, which will be updated in a separate call, - * see: {@link #updateWSCommitIdAndCleanUpMetadata(Statement, long, TxnType, Long, long)}}. - * This should decrease the scope of the S4U lock on the next_txn_id table. - */ - Object undoWriteSetForCurrentTxn = context.getTransactionStatus().createSavepoint(); - stmt.executeUpdate(writeSetInsertSql + (useMinHistoryLevel ? conflictSQLSuffix : - "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" <> " + OperationType.COMPACT)); - - /** - * This S4U will mutex with other commitTxn() and openTxns(). - * -1 below makes txn intervals look like [3,3] [4,4] if all txns are serial - * Note: it's possible to have several txns have the same commit id. Suppose 3 txns start - * at the same time and no new txns start until all 3 commit. - * We could've incremented the sequence for commitId as well but it doesn't add anything functionally. - */ - acquireTxnLock(stmt, false); - commitId = getHighWaterMark(stmt); - - if (!rqst.isExclWriteEnabled()) { - /** - * see if there are any overlapping txns that wrote the same element, i.e. have a conflict - * Since entire commit operation is mutexed wrt other start/commit ops, - * committed.ws_commit_id <= current.ws_commit_id for all txns - * thus if committed.ws_commit_id < current.ws_txnid, transactions do NOT overlap - * For example, [17,20] is committed, [6,80] is being committed right now - these overlap - * [17,20] committed and [21,21] committing now - these do not overlap. - * [17,18] committed and [18,19] committing now - these overlap (here 18 started while 17 was still running) - */ - try (ResultSet rs = checkForWriteConflict(stmt, txnid)) { - if (rs.next()) { - //found a conflict, so let's abort the txn - String committedTxn = "[" + JavaUtils.txnIdToString(rs.getLong(1)) + "," + rs.getLong(2) + "]"; - StringBuilder resource = new StringBuilder(rs.getString(3)).append("/").append(rs.getString(4)); - String partitionName = rs.getString(5); - if (partitionName != null) { - resource.append('/').append(partitionName); - } - String msg = "Aborting [" + JavaUtils.txnIdToString(txnid) + "," + commitId + "]" + " due to a write conflict on " + resource + - " committed by " + committedTxn + " " + rs.getString(7) + "/" + rs.getString(8); - //remove WRITE_SET info for current txn since it's about to abort - context.getTransactionStatus().rollbackToSavepoint(undoWriteSetForCurrentTxn); - LOG.info(msg); - //todo: should make abortTxns() write something into TXNS.TXN_META_INFO about this - if (abortTxns(dbConn, Collections.singletonList(txnid), false, isReplayedReplTxn, - TxnErrorMsg.ABORT_WRITE_CONFLICT) != 1) { - throw new IllegalStateException(msg + " FAILED!"); - } - jdbcResource.getTransactionManager().commit(context); - throw new TxnAbortedException(msg); - } - } - } - } else if (!useMinHistoryLevel) { - stmt.executeUpdate(writeSetInsertSql + "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + - " AND \"TC_OPERATION_TYPE\" <> " + OperationType.COMPACT); - commitId = getHighWaterMark(stmt); - } - } else { - /* - * current txn didn't update/delete anything (may have inserted), so just proceed with commit - * - * We only care about commit id for write txns, so for RO (when supported) txns we don't - * have to mutex on NEXT_TXN_ID. - * Consider: if RO txn is after a W txn, then RO's openTxns() will be mutexed with W's - * commitTxn() because both do S4U on NEXT_TXN_ID and thus RO will see result of W txn. - * If RO < W, then there is no reads-from relationship. - * In replication flow we don't expect any write write conflict as it should have been handled at source. - */ - assert true; - } - - - if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn && !MetaStoreServerUtils.isCompactionTxn(txnType)) { - moveTxnComponentsToCompleted(stmt, txnid, isUpdateDelete); - } else if (isReplayedReplTxn) { - if (rqst.isSetWriteEventInfos()) { - String sql = String.format(COMPL_TXN_COMPONENTS_INSERT_QUERY, txnid, quoteChar(isUpdateDelete)); - try (PreparedStatement pstmt = dbConn.prepareStatement(sql)) { - int insertCounter = 0; - for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) { - pstmt.setString(1, writeEventInfo.getDatabase()); - pstmt.setString(2, writeEventInfo.getTable()); - pstmt.setString(3, writeEventInfo.getPartition()); - pstmt.setLong(4, writeEventInfo.getWriteId()); - - pstmt.addBatch(); - insertCounter++; - if (insertCounter % maxBatchSize == 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", sql, maxBatchSize); - pstmt.executeBatch(); - } - } - if (insertCounter % maxBatchSize != 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", sql, insertCounter % maxBatchSize); - pstmt.executeBatch(); - } - } - } - deleteReplTxnMapEntry(dbConn, sourceTxnId, rqst.getReplPolicy()); - } - updateWSCommitIdAndCleanUpMetadata(stmt, txnid, txnType, commitId, tempCommitId); - removeTxnsFromMinHistoryLevel(dbConn, ImmutableList.of(txnid)); - removeWriteIdsFromMinHistory(dbConn, ImmutableList.of(txnid)); - if (rqst.isSetKeyValue()) { - updateKeyValueAssociatedWithTxn(rqst, stmt); - } - - if (!isHiveReplTxn) { - createCommitNotificationEvent(dbConn, txnid , txnType); - } - - LOG.debug("Going to commit"); - jdbcResource.getTransactionManager().commit(context); - - if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) { - Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_COMMITTED_TXNS).inc(); - } - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - jdbcResource.getTransactionManager().rollback(context); - checkRetryable(e, "commitTxn(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " - + StringUtils.stringifyException(e)); - } finally { - closeStmt(stmt); - unlockInternal(); - } - } catch (RetryException e) { - commitTxn(rqst); - } finally { - jdbcResource.unbindDataSource(); - } - } - - /** - * Create Notifiaction Events on txn commit - * @param txnid committed txn - * @param txnType transaction type - * @throws MetaException ex - */ - protected void createCommitNotificationEvent(Connection conn, long txnid, TxnType txnType) - throws MetaException, SQLException { - if (transactionalListeners != null) { - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, txnType), conn, sqlGenerator); - } - } - - private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throws SQLException, MetaException { - try (ResultSet rs = stmt.executeQuery(sqlGenerator.addLimitClause(1, - "\"TC_OPERATION_TYPE\" " + conflictSQLSuffix))) { - return rs.next(); - } + new CommitTxnFunction(rqst, transactionalListeners).execute(jdbcResource); } /** @@ -1694,4085 +590,364 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw * @return max Id for the conflicting transaction, if any, otherwise -1 * @throws MetaException */ - @RetrySemantics.ReadOnly + @Override public long getLatestTxnIdInConflict(long txnid) throws MetaException { - try { - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - Statement stmt = dbConn.createStatement()) { - - String writeConflictQuery = "SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" + - " FROM \"WRITE_SET\" \"COMMITTED\"" + - " INNER JOIN (" + - " SELECT DISTINCT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_TXNID\"" + - " FROM \"TXN_COMPONENTS\"" + - " WHERE \"TC_TXNID\" = " + txnid + - " AND \"TC_OPERATION_TYPE\" IN (" + OperationType.UPDATE + "," + OperationType.DELETE + ")" + - " ) \"CUR\"" + - " ON \"COMMITTED\".\"WS_DATABASE\" = \"CUR\".\"TC_DATABASE\"" + - " AND \"COMMITTED\".\"WS_TABLE\" = \"CUR\".\"TC_TABLE\"" + - (useMinHistoryLevel ? "" : - " AND \"COMMITTED\".\"WS_OPERATION_TYPE\" != " + OperationType.INSERT) + - // For partitioned table we always track writes at partition level (never at table) - // and for non partitioned - always at table level, thus the same table should never - // have entries with partition key and w/o - " AND (\"COMMITTED\".\"WS_PARTITION\" = \"CUR\".\"TC_PARTITION\" OR" + - " \"CUR\".\"TC_PARTITION\" IS NULL) " + - // txns overlap - " WHERE \"CUR\".\"TC_TXNID\" <= \"COMMITTED\".\"WS_COMMIT_ID\""; - - LOG.debug("Going to execute query: <{}>", writeConflictQuery); - try (ResultSet rs = stmt.executeQuery(writeConflictQuery)) { - return rs.next() ? rs.getLong(1) : -1; - } - } catch (SQLException e) { - checkRetryable(e, "getLatestTxnIdInConflict"); - throw new MetaException(StringUtils.stringifyException(e)); - } - } catch (RetryException e) { - return getLatestTxnIdInConflict(txnid); - } - } - - /** - * Returns the databases updated by txnId. - * Queries TXN_TO_WRITE_ID using txnId. - * - * @param txnId - * @throws MetaException - */ - private List getTxnDbsUpdated(long txnId, Connection dbConn) throws MetaException { - try { - try (Statement stmt = dbConn.createStatement()) { - - String query = "SELECT DISTINCT \"T2W_DATABASE\" " + - " FROM \"TXN_TO_WRITE_ID\" \"COMMITTED\"" + - " WHERE \"T2W_TXNID\" = " + txnId; - - LOG.debug("Going to execute query: <{}>", query); - try (ResultSet rs = stmt.executeQuery(query)) { - List dbsUpdated = new ArrayList(); - while (rs.next()) { - dbsUpdated.add(rs.getString(1)); - } - return dbsUpdated; - } - } catch (SQLException e) { - checkRetryable(e, "getTxnDbsUpdated"); - throw new MetaException(StringUtils.stringifyException(e)); - } - } catch (RetryException e) { - return getTxnDbsUpdated(txnId, dbConn); - } - } - - - private ResultSet checkForWriteConflict(Statement stmt, long txnid) throws SQLException, MetaException { - String writeConflictQuery = sqlGenerator.addLimitClause(1, "\"COMMITTED\".\"WS_TXNID\", \"COMMITTED\".\"WS_COMMIT_ID\", " + - "\"COMMITTED\".\"WS_DATABASE\", \"COMMITTED\".\"WS_TABLE\", \"COMMITTED\".\"WS_PARTITION\", " + - "\"CUR\".\"WS_COMMIT_ID\" \"CUR_WS_COMMIT_ID\", \"CUR\".\"WS_OPERATION_TYPE\" \"CUR_OP\", " + - "\"COMMITTED\".\"WS_OPERATION_TYPE\" \"COMMITTED_OP\" FROM \"WRITE_SET\" \"COMMITTED\" INNER JOIN \"WRITE_SET\" \"CUR\" " + - "ON \"COMMITTED\".\"WS_DATABASE\"=\"CUR\".\"WS_DATABASE\" AND \"COMMITTED\".\"WS_TABLE\"=\"CUR\".\"WS_TABLE\" " + - //For partitioned table we always track writes at partition level (never at table) - //and for non partitioned - always at table level, thus the same table should never - //have entries with partition key and w/o - "AND (\"COMMITTED\".\"WS_PARTITION\"=\"CUR\".\"WS_PARTITION\" OR (\"COMMITTED\".\"WS_PARTITION\" IS NULL AND \"CUR\".\"WS_PARTITION\" IS NULL)) " + - "WHERE \"CUR\".\"WS_TXNID\" <= \"COMMITTED\".\"WS_COMMIT_ID\"" + //txns overlap; could replace ws_txnid - // with txnid, though any decent DB should infer this - " AND \"CUR\".\"WS_TXNID\"=" + txnid + //make sure RHS of join only has rows we just inserted as - // part of this commitTxn() op - " AND \"COMMITTED\".\"WS_TXNID\" <> " + txnid + //and LHS only has committed txns - //U+U and U+D and D+D is a conflict and we don't currently track I in WRITE_SET at all - //it may seem like D+D should not be in conflict but consider 2 multi-stmt txns - //where each does "delete X + insert X, where X is a row with the same PK. This is - //equivalent to an update of X but won't be in conflict unless D+D is in conflict. - //The same happens when Hive splits U=I+D early so it looks like 2 branches of a - //multi-insert stmt (an Insert and a Delete branch). It also 'feels' - // un-serializable to allow concurrent deletes - " and (\"COMMITTED\".\"WS_OPERATION_TYPE\" IN(" + OperationType.UPDATE + - ", " + OperationType.DELETE + - ") AND \"CUR\".\"WS_OPERATION_TYPE\" IN(" + OperationType.UPDATE+ ", " - + OperationType.DELETE + "))"); - LOG.debug("Going to execute query: <{}>", writeConflictQuery); - return stmt.executeQuery(writeConflictQuery); - } - - private void moveTxnComponentsToCompleted(Statement stmt, long txnid, char isUpdateDelete) throws SQLException { - // Move the record from txn_components into completed_txn_components so that the compactor - // knows where to look to compact. - String s = "INSERT INTO \"COMPLETED_TXN_COMPONENTS\" (\"CTC_TXNID\", \"CTC_DATABASE\", " + - "\"CTC_TABLE\", \"CTC_PARTITION\", \"CTC_WRITEID\", \"CTC_UPDATE_DELETE\") SELECT \"TC_TXNID\"," + - " \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_WRITEID\", '" + isUpdateDelete + - "' FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" = " + txnid + - //we only track compactor activity in TXN_COMPONENTS to handle the case where the - //compactor txn aborts - so don't bother copying it to COMPLETED_TXN_COMPONENTS - " AND \"TC_OPERATION_TYPE\" <> " + OperationType.COMPACT; - LOG.debug("Going to execute insert <{}>", s); - - if ((stmt.executeUpdate(s)) < 1) { - //this can be reasonable for an empty txn START/COMMIT or read-only txn - //also an IUD with DP that didn't match any rows. - LOG.info("Expected to move at least one record from txn_components to " - + "completed_txn_components when committing txn! {}", JavaUtils.txnIdToString(txnid)); - } - } - - /** - * See overridden method in CompactionTxnHandler also. - */ - protected void updateWSCommitIdAndCleanUpMetadata(Statement stmt, long txnid, TxnType txnType, - Long commitId, long tempId) throws SQLException, MetaException { - List queryBatch = new ArrayList<>(5); - // update write_set with real commitId - if (commitId != null) { - queryBatch.add("UPDATE \"WRITE_SET\" SET \"WS_COMMIT_ID\" = " + commitId + - " WHERE \"WS_COMMIT_ID\" = " + tempId + " AND \"WS_TXNID\" = " + txnid); - } - // clean up txn related metadata - if (txnType != TxnType.READ_ONLY) { - queryBatch.add("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" = " + txnid); - } - queryBatch.add("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_TXNID\" = " + txnid); - // DO NOT remove the transaction from the TXN table, the cleaner will remove it when appropriate - queryBatch.add("UPDATE \"TXNS\" SET \"TXN_STATE\" = " + TxnStatus.COMMITTED + " WHERE \"TXN_ID\" = " + txnid); - if (txnType == TxnType.MATER_VIEW_REBUILD) { - queryBatch.add("DELETE FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE \"MRL_TXN_ID\" = " + txnid); - } - // execute all in one batch - executeQueriesInBatchNoCount(dbProduct, stmt, queryBatch, maxBatchSize); - } - - private void updateKeyValueAssociatedWithTxn(CommitTxnRequest rqst, Statement stmt) throws SQLException { - if (!rqst.getKeyValue().getKey().startsWith(TxnStore.TXN_KEY_START)) { - String errorMsg = "Error updating key/value in the sql backend with" - + " txnId=" + rqst.getTxnid() + "," - + " tableId=" + rqst.getKeyValue().getTableId() + "," - + " key=" + rqst.getKeyValue().getKey() + "," - + " value=" + rqst.getKeyValue().getValue() + "." - + " key should start with " + TXN_KEY_START + "."; - LOG.warn(errorMsg); - throw new IllegalArgumentException(errorMsg); - } - String s = "UPDATE \"TABLE_PARAMS\" SET" - + " \"PARAM_VALUE\" = " + quoteString(rqst.getKeyValue().getValue()) - + " WHERE \"TBL_ID\" = " + rqst.getKeyValue().getTableId() - + " AND \"PARAM_KEY\" = " + quoteString(rqst.getKeyValue().getKey()); - LOG.debug("Going to execute update <{}>", s); - int affectedRows = stmt.executeUpdate(s); - if (affectedRows != 1) { - String errorMsg = "Error updating key/value in the sql backend with" - + " txnId=" + rqst.getTxnid() + "," - + " tableId=" + rqst.getKeyValue().getTableId() + "," - + " key=" + rqst.getKeyValue().getKey() + "," - + " value=" + rqst.getKeyValue().getValue() + "." - + " Only one row should have been affected but " - + affectedRows + " rows where affected."; - LOG.warn(errorMsg); - throw new IllegalStateException(errorMsg); - } + return jdbcResource.execute(new LatestTxnIdInConflictHandler(txnid)); } /** - * Replicate Table Write Ids state to mark aborted write ids and writeid high water mark. + * Replicate Table Write Ids state to mark aborted write ids and writeid high watermark. * @param rqst info on table/partitions and writeid snapshot to replicate. * @throws MetaException */ - @Override - @RetrySemantics.Idempotent("No-op if already replicated the writeid state") - public void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException { - String dbName = rqst.getDbName().toLowerCase(); - String tblName = rqst.getTableName().toLowerCase(); - ValidWriteIdList validWriteIdList = new ValidReaderWriteIdList(rqst.getValidWriteIdlist()); - - // Get the abortedWriteIds which are already sorted in ascending order. - List abortedWriteIds = getAbortedWriteIds(validWriteIdList); - int numAbortedWrites = abortedWriteIds.size(); - try { - Connection dbConn = null; - Statement stmt = null; - PreparedStatement pStmt = null; - List insertPreparedStmts = null; - ResultSet rs = null; - List params = Arrays.asList(dbName, tblName); - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - - // Check if this txn state is already replicated for this given table. If yes, then it is - // idempotent case and just return. - String sql = "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?"; - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, sql, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + sql.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - rs = pStmt.executeQuery(); - if (rs.next()) { - LOG.info("Idempotent flow: WriteId state <{}> is already applied for the table: {}.{}", validWriteIdList, - dbName, tblName); - rollbackDBConn(dbConn); - return; - } - - if (numAbortedWrites > 0) { - // Allocate/Map one txn per aborted writeId and abort the txn to mark writeid as aborted. - // We don't use the txnLock, all of these transactions will be aborted in this one rdbm transaction - // So they will not effect the commitTxn in any way - List txnIds = openTxns(dbConn, - new OpenTxnRequest(numAbortedWrites, rqst.getUser(), rqst.getHostName())); - assert(numAbortedWrites == txnIds.size()); - - // Map each aborted write id with each allocated txn. - List rows = new ArrayList<>(); - List> paramsList = new ArrayList<>(); - int i = 0; - for (long txn : txnIds) { - long writeId = abortedWriteIds.get(i++); - rows.add(txn + ", ?, ?, " + writeId); - paramsList.add(params); - LOG.info("Allocated writeID: {} for txnId: {}", writeId, txn); - } - - // Insert entries to TXN_TO_WRITE_ID for aborted write ids - insertPreparedStmts = sqlGenerator.createInsertValuesPreparedStmt(dbConn, - "\"TXN_TO_WRITE_ID\" (\"T2W_TXNID\", \"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\")", rows, - paramsList); - for (PreparedStatement pst : insertPreparedStmts) { - pst.execute(); - } - - // Abort all the allocated txns so that the mapped write ids are referred as aborted ones. - int numAborts = abortTxns(dbConn, txnIds, false, false, TxnErrorMsg.ABORT_REPL_WRITEID_TXN); - assert(numAborts == numAbortedWrites); - } - - // There are some txns in the list which has no write id allocated and hence go ahead and do it. - // Get the next write id for the given table and update it with new next write id. - // It is expected NEXT_WRITE_ID doesn't have entry for this table and hence directly insert it. - long nextWriteId = validWriteIdList.getHighWatermark() + 1; - - // First allocation of write id (hwm+1) should add the table to the next_write_id meta table. - sql = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (?, ?, " - + Long.toString(nextWriteId) + ")"; - closeStmt(pStmt); - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, sql, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute insert <" + sql.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - pStmt.execute(); - - LOG.info("WriteId state <{}> is applied for the table: {}.{}", validWriteIdList, dbName, tblName); - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "replTableWriteIdState(" + rqst + ")", true); - throw new MetaException("Unable to update transaction database " - + StringUtils.stringifyException(e)); - } finally { - if (insertPreparedStmts != null) { - for (PreparedStatement pst : insertPreparedStmts) { - closeStmt(pst); - } - } - closeStmt(pStmt); - close(rs, stmt, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - replTableWriteIdState(rqst); - } - - // Schedule Major compaction on all the partitions/table to clean aborted data - if (numAbortedWrites > 0) { - CompactionRequest compactRqst = new CompactionRequest(rqst.getDbName(), rqst.getTableName(), - CompactionType.MAJOR); - if (rqst.isSetPartNames()) { - for (String partName : rqst.getPartNames()) { - compactRqst.setPartitionname(partName); - compact(compactRqst); - } - } else { - compact(compactRqst); - } - } - } - - private List getAbortedWriteIds(ValidWriteIdList validWriteIdList) { - return Arrays.stream(validWriteIdList.getInvalidWriteIds()) - .filter(validWriteIdList::isWriteIdAborted) - .boxed() - .collect(Collectors.toList()); - } - - private ValidTxnList getValidTxnList(Connection dbConn, String fullTableName, Long writeId) throws MetaException, - SQLException { - PreparedStatement pst = null; - ResultSet rs = null; - try { - String[] names = TxnUtils.getDbTableName(fullTableName); - assert names.length == 2; - List params = Arrays.asList(names[0], names[1]); - String s = - "SELECT \"T2W_TXNID\" FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = ? AND " - + "\"T2W_TABLE\" = ? AND \"T2W_WRITEID\" = "+ writeId; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + s.replace("?", "{}") + ">", quoteString(names[0]), - quoteString(names[1])); - } - rs = pst.executeQuery(); - if (rs.next()) { - long txnId = rs.getLong(1); - return TxnCommonUtils.createValidReadTxnList(getOpenTxns(dbConn), txnId); - } - throw new MetaException("invalid write id " + writeId + " for table " + fullTableName); - } finally { - close(rs, pst, null); - } - } - - @Override - @RetrySemantics.ReadOnly - public GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst) throws MetaException { - try { - Connection dbConn = null; - ValidTxnList validTxnList; - - try { - /** - * This runs at READ_COMMITTED for exactly the same reason as {@link #getOpenTxnsInfo()} - */ - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - // We should prepare the valid write ids list based on validTxnList of current txn. - // If no txn exists in the caller, then they would pass null for validTxnList and so it is - // required to get the current state of txns to make validTxnList - if (rqst.isSetValidTxnList()) { - assert rqst.isSetWriteId() == false; - validTxnList = new ValidReadTxnList(rqst.getValidTxnList()); - } else if (rqst.isSetWriteId()) { - validTxnList = getValidTxnList(dbConn, rqst.getFullTableNames().get(0), rqst.getWriteId()); - } else { - // Passing 0 for currentTxn means, this validTxnList is not wrt to any txn - validTxnList = TxnCommonUtils.createValidReadTxnList(getOpenTxns(dbConn), 0); - } - - // Get the valid write id list for all the tables read by the current txn - List tblValidWriteIdsList = new ArrayList<>(); - for (String fullTableName : rqst.getFullTableNames()) { - tblValidWriteIdsList.add(getValidWriteIdsForTable(dbConn, fullTableName, validTxnList)); - } - - GetValidWriteIdsResponse owr = new GetValidWriteIdsResponse(tblValidWriteIdsList); - return owr; - } catch (SQLException e) { - checkRetryable(e, "getValidWriteIds"); - throw new MetaException("Unable to select from transaction database, " - + StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - } - } catch (RetryException e) { - return getValidWriteIds(rqst); - } - } - - // Method to get the Valid write ids list for the given table - // Input fullTableName is expected to be of format . - private TableValidWriteIds getValidWriteIdsForTable(Connection dbConn, String fullTableName, - ValidTxnList validTxnList) throws SQLException { - PreparedStatement pst = null; - ResultSet rs = null; - String[] names = TxnUtils.getDbTableName(fullTableName); - assert(names.length == 2); - List params = Arrays.asList(names[0], names[1]); - try { - // Need to initialize to 0 to make sure if nobody modified this table, then current txn - // shouldn't read any data. - // If there is a conversion from non-acid to acid table, then by default 0 would be assigned as - // writeId for data from non-acid table and so writeIdHwm=0 would ensure those data are readable by any txns. - long writeIdHwm = 0; - List invalidWriteIdList = new ArrayList<>(); - long minOpenWriteId = Long.MAX_VALUE; - BitSet abortedBits = new BitSet(); - long txnHwm = validTxnList.getHighWatermark(); - - // Find the writeId high water mark based upon txnId high water mark. If found, then, need to - // traverse through all write Ids less than writeId HWM to make exceptions list. - // The writeHWM = min(NEXT_WRITE_ID.nwi_next-1, max(TXN_TO_WRITE_ID.t2w_writeid under txnHwm)) - String s = "SELECT MAX(\"T2W_WRITEID\") FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_TXNID\" <= " + txnHwm - + " AND \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ?"; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query<" + s.replace("?", "{}") + ">", - quoteString(names[0]), quoteString(names[1])); - } - rs = pst.executeQuery(); - if (rs.next()) { - writeIdHwm = rs.getLong(1); - } - - // If no writeIds allocated by txns under txnHwm, then find writeHwm from NEXT_WRITE_ID. - if (writeIdHwm <= 0) { - // Need to subtract 1 as nwi_next would be the next write id to be allocated but we need highest - // allocated write id. - s = "SELECT \"NWI_NEXT\"-1 FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?"; - closeStmt(pst); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query<" + s.replace("?", "{}") + ">", - quoteString(names[0]), quoteString(names[1])); - } - rs = pst.executeQuery(); - if (rs.next()) { - writeIdHwm = rs.getLong(1); - } - } - boolean foundValidUncompactedWrite = false; - // As writeIdHwm is known, query all writeIds under the writeId HWM. - // If any writeId under HWM is allocated by txn > txnId HWM or belongs to open/aborted txns, - // then will be added to invalid list. The results should be sorted in ascending order based - // on write id. The sorting is needed as exceptions list in ValidWriteIdList would be looked-up - // using binary search. - s = "SELECT \"T2W_TXNID\", \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_WRITEID\" <= " + Long.toString(writeIdHwm) - + " AND \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? ORDER BY \"T2W_WRITEID\" ASC"; - closeStmt(pst); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query<" + s.replace("?", "{}") + ">", - quoteString(names[0]), quoteString(names[1])); - } - rs = pst.executeQuery(); - while (rs.next()) { - long txnId = rs.getLong(1); - long writeId = rs.getLong(2); - if (validTxnList.isTxnValid(txnId)) { - // Skip if the transaction under evaluation is already committed. - foundValidUncompactedWrite = true; - continue; - } - // The current txn is either in open or aborted state. - // Mark the write ids state as per the txn state. - invalidWriteIdList.add(writeId); - if (validTxnList.isTxnAborted(txnId)) { - abortedBits.set(invalidWriteIdList.size() - 1); - } else { - minOpenWriteId = Math.min(minOpenWriteId, writeId); - } - } - // If we have compacted writes and some invalid writes on the table, - // return the lowest invalid write as a writeIdHwm and set it as invalid. - if (!foundValidUncompactedWrite) { - long writeId = invalidWriteIdList.isEmpty() ? -1 : invalidWriteIdList.get(0); - invalidWriteIdList = new ArrayList<>(); - abortedBits = new BitSet(); - - if (writeId != -1) { - invalidWriteIdList.add(writeId); - writeIdHwm = writeId; - if (writeId != minOpenWriteId) { - abortedBits.set(0); - } - } - } - ByteBuffer byteBuffer = ByteBuffer.wrap(abortedBits.toByteArray()); - TableValidWriteIds owi = new TableValidWriteIds(fullTableName, writeIdHwm, invalidWriteIdList, byteBuffer); - if (minOpenWriteId < Long.MAX_VALUE) { - owi.setMinOpenWriteId(minOpenWriteId); - } - return owi; - } finally { - closeStmt(pst); - close(rs); - } - } - - @Override - @RetrySemantics.Idempotent - public AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst) - throws MetaException { - List txnIds; - String dbName = rqst.getDbName().toLowerCase(); - String tblName = rqst.getTableName().toLowerCase(); - boolean shouldReallocate = rqst.isReallocate(); - try { - Connection dbConn = null; - PreparedStatement pStmt = null; - ResultSet rs = null; - List txnToWriteIds = new ArrayList<>(); - List srcTxnToWriteIds = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - if (rqst.isSetReplPolicy()) { - srcTxnToWriteIds = rqst.getSrcTxnToWriteIdList(); - List srcTxnIds = new ArrayList<>(); - assert (rqst.isSetSrcTxnToWriteIdList()); - assert (!rqst.isSetTxnIds()); - assert (!srcTxnToWriteIds.isEmpty()); - - for (TxnToWriteId txnToWriteId : srcTxnToWriteIds) { - srcTxnIds.add(txnToWriteId.getTxnId()); - } - txnIds = getTargetTxnIdList(rqst.getReplPolicy(), srcTxnIds, dbConn); - if (srcTxnIds.size() != txnIds.size()) { - // Idempotent case where txn was already closed but gets allocate write id event. - // So, just ignore it and return empty list. - LOG.info("Idempotent case: Target txn id is missing for source txn id : {} and repl policy {}", srcTxnIds, - rqst.getReplPolicy()); - return new AllocateTableWriteIdsResponse(txnToWriteIds); - } - } else { - assert (!rqst.isSetSrcTxnToWriteIdList()); - assert (rqst.isSetTxnIds()); - txnIds = rqst.getTxnIds(); - } - - //Easiest check since we can't differentiate do we handle singleton list or list with multiple txn ids. - if (txnIds.size() > 1) { - Collections.sort(txnIds); //easier to read logs and for assumption done in replication flow - } - - // Check if all the input txns are in valid state. - // Write IDs should be allocated only for open and not read-only transactions. - try (Statement stmt = dbConn.createStatement()) { - if (!isTxnsOpenAndNotReadOnly(txnIds, stmt)) { - String errorMsg = "Write ID allocation on " + TableName.getDbTable(dbName, tblName) - + " failed for input txns: " - + getAbortedAndReadOnlyTxns(txnIds, stmt) - + getCommittedTxns(txnIds, stmt); - LOG.error(errorMsg); - - throw new IllegalStateException("Write ID allocation failed on " + TableName.getDbTable(dbName, tblName) - + " as not all input txns in open state or read-only"); - } - } - - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - long writeId; - int allocatedTxnsCount = 0; - List params = Arrays.asList(dbName, tblName); - if (shouldReallocate) { - // during query recompilation after lock acquistion, it is important to realloc new writeIds - // to ensure writeIds are committed in increasing order. - prefix.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE") - .append(" \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, - txnIds, "\"T2W_TXNID\"", false, false); - for (String query : queries) { - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute delete <" + query.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - int numRowsDeleted = pStmt.executeUpdate(); - LOG.info("Removed {} prior writeIds during reallocation", numRowsDeleted); - closeStmt(pStmt); - } - } else { - // Traverse the TXN_TO_WRITE_ID to see if any of the input txns already have allocated a - // write id for the same db.table. If yes, then need to reuse it else have to allocate new one - // The write id would have been already allocated in case of multi-statement txns where - // first write on a table will allocate write id and rest of the writes should re-use it. - prefix.append("SELECT \"T2W_TXNID\", \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE") - .append(" \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, - txnIds, "\"T2W_TXNID\"", false, false); - for (String query : queries) { - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - rs = pStmt.executeQuery(); - while (rs.next()) { - // If table write ID is already allocated for the given transaction, then just use it - long txnId = rs.getLong(1); - writeId = rs.getLong(2); - txnToWriteIds.add(new TxnToWriteId(txnId, writeId)); - allocatedTxnsCount++; - LOG.info("Reused already allocated writeID: {} for txnId: {}", writeId, txnId); - } - closeStmt(pStmt); - } - } - - // Batch allocation should always happen atomically. Either write ids for all txns is allocated or none. - long numOfWriteIds = txnIds.size(); - assert ((allocatedTxnsCount == 0) || (numOfWriteIds == allocatedTxnsCount)); - if (allocatedTxnsCount == numOfWriteIds) { - // If all the txns in the list have pre-allocated write ids for the given table, then just return. - // This is for idempotent case. - return new AllocateTableWriteIdsResponse(txnToWriteIds); - } - - long srcWriteId = 0; - if (rqst.isSetReplPolicy()) { - // In replication flow, we always need to allocate write ID equal to that of source. - assert (srcTxnToWriteIds != null); - srcWriteId = srcTxnToWriteIds.get(0).getWriteId(); - } - - - // There are some txns in the list which does not have write id allocated and hence go ahead and do it. - // Get the next write id for the given table and update it with new next write id. - // This is select for update query which takes a lock if the table entry is already there in NEXT_WRITE_ID - String s = sqlGenerator.addForUpdateClause( - "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?"); - closeStmt(pStmt); - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + s.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - rs = pStmt.executeQuery(); - if (!rs.next()) { - // First allocation of write id should add the table to the next_write_id meta table - // The initial value for write id should be 1 and hence we add 1 with number of write ids allocated here - // For repl flow, we need to force set the incoming write id. - writeId = (srcWriteId > 0) ? srcWriteId : 1; - s = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (?, ?, " - + (writeId + numOfWriteIds) + ")"; - closeStmt(pStmt); - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute insert <" + s.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - pStmt.execute(); - } else { - long nextWriteId = rs.getLong(1); - writeId = (srcWriteId > 0) ? srcWriteId : nextWriteId; - - // Update the NEXT_WRITE_ID for the given table after incrementing by number of write ids allocated - s = "UPDATE \"NEXT_WRITE_ID\" SET \"NWI_NEXT\" = " + (writeId + numOfWriteIds) - + " WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?"; - closeStmt(pStmt); - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute update <" + s.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - pStmt.executeUpdate(); - - // For repl flow, if the source write id is mismatching with target next write id, then current - // metadata in TXN_TO_WRITE_ID is stale for this table and hence need to clean-up TXN_TO_WRITE_ID. - // This is possible in case of first incremental repl after bootstrap where concurrent write - // and drop table was performed at source during bootstrap dump. - if ((srcWriteId > 0) && (srcWriteId != nextWriteId)) { - s = "DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ?"; - closeStmt(pStmt); - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute delete <" + s.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - pStmt.executeUpdate(); - } - } - - // Map the newly allocated write ids against the list of txns which doesn't have pre-allocated write ids - try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_TO_WRITE_ID_INSERT_QUERY)) { - for (long txnId : txnIds) { - pstmt.setLong(1, txnId); - pstmt.setString(2, dbName); - pstmt.setString(3, tblName); - pstmt.setLong(4, writeId); - pstmt.addBatch(); - - txnToWriteIds.add(new TxnToWriteId(txnId, writeId)); - LOG.info("Allocated writeId: {} for txnId: {}", writeId, txnId); - writeId++; - if (txnToWriteIds.size() % maxBatchSize == 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_TO_WRITE_ID_INSERT_QUERY, - maxBatchSize); - pstmt.executeBatch(); - } - } - if (txnToWriteIds.size() % maxBatchSize != 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_TO_WRITE_ID_INSERT_QUERY, - txnToWriteIds.size() % maxBatchSize); - pstmt.executeBatch(); - } - } - - if (transactionalListeners != null) { - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.ALLOC_WRITE_ID, - new AllocWriteIdEvent(txnToWriteIds, dbName, tblName), - dbConn, sqlGenerator); - } - - LOG.info("Allocated write ids for dbName={}, tblName={} (txnIds: {})", dbName, tblName, rqst.getTxnIds()); - dbConn.commit(); - return new AllocateTableWriteIdsResponse(txnToWriteIds); - } catch (SQLException e) { - LOG.error("Exception during write ids allocation for request={}. Will retry if possible.", rqst, e); - rollbackDBConn(dbConn); - checkRetryable(e, "allocateTableWriteIds(" + rqst + ")", true); - throw new MetaException("Unable to update transaction database " - + StringUtils.stringifyException(e)); - } finally { - close(rs, pStmt, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - return allocateTableWriteIds(rqst); - } - } - - @Override - public MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWriteIdRequest rqst) throws MetaException { - String dbName = rqst.getDbName(); - String tableName = rqst.getTableName(); - try { - Connection dbConn = null; - PreparedStatement pStmt = null; - ResultSet rs = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID, - Arrays.asList(dbName, tableName)); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tableName)); - } - rs = pStmt.executeQuery(); - // If there is no record, we never allocated anything - long maxWriteId = 0l; - if (rs.next()) { - // The row contains the nextId not the previously allocated - maxWriteId = rs.getLong(1) - 1; - } - return new MaxAllocatedTableWriteIdResponse(maxWriteId); - } catch (SQLException e) { - LOG.error( - "Exception during reading the max allocated writeId for dbName={}, tableName={}. Will retry if possible.", - dbName, tableName, e); - checkRetryable(e, "getMaxAllocatedTableWrited(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " + StringUtils.stringifyException(e)); - } finally { - close(rs, pStmt, dbConn); - } - } catch (RetryException e) { - return getMaxAllocatedTableWrited(rqst); - } - } - - @Override - public void seedWriteId(SeedTableWriteIdsRequest rqst) - throws MetaException { - try { - Connection dbConn = null; - PreparedStatement pst = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - //since this is on conversion from non-acid to acid, NEXT_WRITE_ID should not have an entry - //for this table. It also has a unique index in case 'should not' is violated - - // First allocation of write id should add the table to the next_write_id meta table - // The initial value for write id should be 1 and hence we add 1 with number of write ids - // allocated here - String s = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (?, ?, " - + Long.toString(rqst.getSeedWriteId() + 1) + ")"; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, Arrays.asList(rqst.getDbName(), rqst.getTableName())); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute insert <" + s.replace("?", "{}") + ">", - quoteString(rqst.getDbName()), quoteString(rqst.getTableName())); - } - pst.execute(); - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - rollbackDBConn(dbConn); - checkRetryable(e, "seedWriteId(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " + StringUtils.stringifyException(e)); - } finally { - close(null, pst, dbConn); - } - } catch (RetryException e) { - seedWriteId(rqst); - } - } - - @Override - public void seedTxnId(SeedTxnIdRequest rqst) throws MetaException { - try { - Connection dbConn = null; - Statement stmt = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - /* - * Locking the txnLock an exclusive way, we do not want to set the txnId backward accidentally - * if there are concurrent open transactions - */ - acquireTxnLock(stmt, false); - long highWaterMark = getHighWaterMark(stmt); - if (highWaterMark >= rqst.getSeedTxnId()) { - throw new MetaException(MessageFormat - .format("Invalid txnId seed {}, the highWaterMark is {}", rqst.getSeedTxnId(), highWaterMark)); - } - TxnUtils.seedTxnSequence(dbConn, conf, stmt, rqst.getSeedTxnId()); - dbConn.commit(); - - } catch (SQLException e) { - rollbackDBConn(dbConn); - checkRetryable(e, "seedTxnId(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " + StringUtils.stringifyException(e)); - } finally { - close(null, stmt, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - seedTxnId(rqst); - } - } - - @Override - @RetrySemantics.Idempotent - public void addWriteNotificationLog(ListenerEvent acidWriteEvent) throws MetaException { - Connection dbConn = jdbcResource.getConnection(); - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - acidWriteEvent instanceof AcidWriteEvent ? EventMessage.EventType.ACID_WRITE - : EventMessage.EventType.BATCH_ACID_WRITE, - acidWriteEvent, dbConn, sqlGenerator); - } - - @Override - @RetrySemantics.SafeToRetry - public void performWriteSetGC() throws MetaException { - Connection dbConn = null; - Statement stmt = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - long commitHighWaterMark = getMinOpenTxnIdWaterMark(dbConn); - int delCnt = stmt.executeUpdate("DELETE FROM \"WRITE_SET\" WHERE \"WS_COMMIT_ID\" < " + commitHighWaterMark); - LOG.info("Deleted {} obsolete rows from WRITE_SET", delCnt); - dbConn.commit(); - } catch (SQLException ex) { - LOG.warn("WriteSet GC failed due to " + getMessage(ex), ex); - } finally { - close(null, stmt, dbConn); - } - } - - protected long getMinOpenTxnIdWaterMark(Connection dbConn) throws SQLException, MetaException { - /** - * We try to find the highest transactionId below everything was committed or aborted. - * For that we look for the lowest open transaction in the TXNS and the TxnMinTimeout boundary, - * because it is guaranteed there won't be open transactions below that. - */ - long minOpenTxn; - try (Statement stmt = dbConn.createStatement()) { - try (ResultSet rs = stmt - .executeQuery("SELECT MIN(\"TXN_ID\") FROM \"TXNS\" WHERE \"TXN_STATE\"=" + TxnStatus.OPEN)) { - if (!rs.next()) { - throw new IllegalStateException("Scalar query returned no rows?!?!!"); - } - minOpenTxn = rs.getLong(1); - if (rs.wasNull()) { - minOpenTxn = Long.MAX_VALUE; - } - } - } catch (SQLException e) { - throw new UncategorizedSQLException(null, null, e); - } - long lowWaterMark = getOpenTxnTimeoutLowBoundaryTxnId(dbConn); - LOG.debug("MinOpenTxnIdWaterMark calculated with minOpenTxn {}, lowWaterMark {}", minOpenTxn, lowWaterMark); - return Long.min(minOpenTxn, lowWaterMark + 1); - } - - @Override - public void updateTransactionStatistics(UpdateTransactionalStatsRequest req) throws MetaException { - String queryText = "UPDATE \"MV_TABLES_USED\" " + - "SET \"INSERTED_COUNT\"=\"INSERTED_COUNT\"+?" + - ",\"UPDATED_COUNT\"=\"UPDATED_COUNT\"+?" + - ",\"DELETED_COUNT\"=\"DELETED_COUNT\"+?" + - " WHERE \"TBL_ID\"=?"; - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) { - try (PreparedStatement pstmt = dbConn.prepareStatement(queryText)) { - pstmt.setLong(1, req.getInsertCount()); - pstmt.setLong(2, req.getUpdatedCount()); - pstmt.setLong(3, req.getDeletedCount()); - pstmt.setLong(4, req.getTableId()); - LOG.debug("Going to execute query <{}>", queryText); - int res = pstmt.executeUpdate(); - dbConn.commit(); - LOG.debug("Updated {} records tblId={}", res, req.getTableId()); - } - } catch (SQLException ex) { - LOG.warn("Unable to update transactional statistics tblId=" + req.getTableId(), ex); - throw new MetaException("Unable to update transactional statistics" + " " + StringUtils.stringifyException(ex)); - } - } - - /** - * Get invalidation info for the materialization. Materialization information - * contains information about whether there was update/delete/compaction operations on the source - * tables used by the materialization since it was created. - */ - @Override - @RetrySemantics.ReadOnly - public Materialization getMaterializationInvalidationInfo( - CreationMetadata creationMetadata, String validTxnListStr) throws MetaException { - if (creationMetadata.getTablesUsed().isEmpty()) { - // Bail out - LOG.warn("Materialization creation metadata does not contain any table"); - return null; - } - - // We are composing a query that returns a single row if an update happened after - // the materialization was created. Otherwise, query returns 0 rows. - - // Parse validReaderWriteIdList from creation metadata - MaterializationSnapshot mvSnapshot = MaterializationSnapshot.fromJson(creationMetadata.getValidTxnList()); - if (mvSnapshot.getTableSnapshots() != null && !mvSnapshot.getTableSnapshots().isEmpty()) { - // Incremental rebuild of MVs on Iceberg sources is not supported. - return null; - } - final ValidTxnWriteIdList validReaderWriteIdList = new ValidTxnWriteIdList(mvSnapshot.getValidTxnList()); - - // Parse validTxnList - final ValidReadTxnList currentValidTxnList = new ValidReadTxnList(validTxnListStr); - // Get the valid write id list for the tables in current state - final List currentTblValidWriteIdsList = new ArrayList<>(); - Connection dbConn = null; - for (String fullTableName : creationMetadata.getTablesUsed()) { - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - currentTblValidWriteIdsList.add(getValidWriteIdsForTable(dbConn, fullTableName, currentValidTxnList)); - } catch (SQLException ex) { - String errorMsg = "Unable to query Valid writeIds of table " + fullTableName; - LOG.warn(errorMsg, ex); - throw new MetaException(errorMsg + " " + StringUtils.stringifyException(ex)); - } finally { - closeDbConn(dbConn); - } - } - final ValidTxnWriteIdList currentValidReaderWriteIdList = TxnCommonUtils.createValidTxnWriteIdList( - currentValidTxnList.getHighWatermark(), currentTblValidWriteIdsList); - - List params = new ArrayList<>(); - StringBuilder queryUpdateDelete = new StringBuilder(); - StringBuilder queryCompletedCompactions = new StringBuilder(); - StringBuilder queryCompactionQueue = new StringBuilder(); - // compose a query that select transactions containing an update... - queryUpdateDelete.append("SELECT \"CTC_UPDATE_DELETE\" FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_UPDATE_DELETE\" ='Y' AND ("); - queryCompletedCompactions.append("SELECT 1 FROM \"COMPLETED_COMPACTIONS\" WHERE ("); - queryCompactionQueue.append("SELECT 1 FROM \"COMPACTION_QUEUE\" WHERE ("); - int i = 0; - for (String fullyQualifiedName : creationMetadata.getTablesUsed()) { - ValidWriteIdList tblValidWriteIdList = - validReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName); - if (tblValidWriteIdList == null) { - LOG.warn("ValidWriteIdList for table {} not present in creation metadata, this should not happen", fullyQualifiedName); - return null; - } - - // First, we check whether the low watermark has moved for any of the tables. - // If it has, we return true, since it is not incrementally refreshable, e.g., - // one of the commits that are not available may be an update/delete. - ValidWriteIdList currentTblValidWriteIdList = - currentValidReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName); - if (currentTblValidWriteIdList == null) { - LOG.warn("Current ValidWriteIdList for table {} not present in creation metadata, this should not happen", fullyQualifiedName); - return null; - } - if (!Objects.equals(currentTblValidWriteIdList.getMinOpenWriteId(), tblValidWriteIdList.getMinOpenWriteId())) { - LOG.debug("Minimum open write id do not match for table {}", fullyQualifiedName); - return null; - } - - // ...for each of the tables that are part of the materialized view, - // where the transaction had to be committed after the materialization was created... - if (i != 0) { - queryUpdateDelete.append("OR"); - queryCompletedCompactions.append("OR"); - queryCompactionQueue.append("OR"); - } - String[] names = TxnUtils.getDbTableName(fullyQualifiedName); - assert (names.length == 2); - queryUpdateDelete.append(" (\"CTC_DATABASE\"=? AND \"CTC_TABLE\"=?"); - queryCompletedCompactions.append(" (\"CC_DATABASE\"=? AND \"CC_TABLE\"=?"); - queryCompactionQueue.append(" (\"CQ_DATABASE\"=? AND \"CQ_TABLE\"=?"); - params.add(names[0]); - params.add(names[1]); - queryUpdateDelete.append(" AND (\"CTC_WRITEID\" > " + tblValidWriteIdList.getHighWatermark()); - queryCompletedCompactions.append(" AND (\"CC_HIGHEST_WRITE_ID\" > " + tblValidWriteIdList.getHighWatermark()); - queryUpdateDelete.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " : - " OR \"CTC_WRITEID\" IN(" + StringUtils.join(",", - Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ) "); - queryCompletedCompactions.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " : - " OR \"CC_HIGHEST_WRITE_ID\" IN(" + StringUtils.join(",", - Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ) "); - queryUpdateDelete.append(") "); - queryCompletedCompactions.append(") "); - queryCompactionQueue.append(") "); - i++; - } - // ... and where the transaction has already been committed as per snapshot taken - // when we are running current query - queryUpdateDelete.append(") AND \"CTC_TXNID\" <= " + currentValidTxnList.getHighWatermark()); - queryUpdateDelete.append(currentValidTxnList.getInvalidTransactions().length == 0 ? " " : - " AND \"CTC_TXNID\" NOT IN(" + StringUtils.join(",", - Arrays.asList(ArrayUtils.toObject(currentValidTxnList.getInvalidTransactions()))) + ") "); - queryCompletedCompactions.append(")"); - queryCompactionQueue.append(") "); - - boolean hasUpdateDelete = executeBoolean(queryUpdateDelete.toString(), params, - "Unable to retrieve materialization invalidation information: completed transaction components."); - - // Execute query - queryCompletedCompactions.append(" UNION "); - queryCompletedCompactions.append(queryCompactionQueue.toString()); - List paramsTwice = new ArrayList<>(params); - paramsTwice.addAll(params); - boolean hasCompaction = executeBoolean(queryCompletedCompactions.toString(), paramsTwice, - "Unable to retrieve materialization invalidation information: compactions"); - - return new Materialization(hasUpdateDelete, hasCompaction); - } - - private boolean executeBoolean(String queryText, List params, String errorMessage) throws MetaException { - Connection dbConn = null; - PreparedStatement pst = null; - ResultSet rs = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - LOG.debug("Going to execute query <{}>", queryText); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, queryText, params); - pst.setMaxRows(1); - rs = pst.executeQuery(); - - return rs.next(); - } catch (SQLException ex) { - LOG.warn(errorMessage, ex); - throw new MetaException(errorMessage + " " + StringUtils.stringifyException(ex)); - } finally { - close(rs, pst, dbConn); - } - } - - @Override - public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) - throws MetaException { - - if (LOG.isDebugEnabled()) { - LOG.debug("Acquiring lock for materialization rebuild with {} for {}", - JavaUtils.txnIdToString(txnId), TableName.getDbTable(dbName, tableName)); - } - - TxnStore.MutexAPI.LockHandle handle = null; - Connection dbConn = null; - PreparedStatement pst = null; - ResultSet rs = null; - try { - lockInternal(); - /** - * MUTEX_KEY.MaterializationRebuild lock ensures that there is only 1 entry in - * Initiated/Working state for any resource. This ensures we do not run concurrent - * rebuild operations on any materialization. - */ - handle = getMutexAPI().acquireLock(MUTEX_KEY.MaterializationRebuild.name()); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - List params = Arrays.asList(dbName, tableName); - String selectQ = "SELECT \"MRL_TXN_ID\" FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE" + - " \"MRL_DB_NAME\" = ? AND \"MRL_TBL_NAME\" = ?"; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, selectQ, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + selectQ.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tableName)); - } - rs = pst.executeQuery(); - if(rs.next()) { - LOG.info("Ignoring request to rebuild {}/{} since it is already being rebuilt", dbName, tableName); - return new LockResponse(txnId, LockState.NOT_ACQUIRED); - } - String insertQ = "INSERT INTO \"MATERIALIZATION_REBUILD_LOCKS\" " + - "(\"MRL_TXN_ID\", \"MRL_DB_NAME\", \"MRL_TBL_NAME\", \"MRL_LAST_HEARTBEAT\") VALUES (" + txnId + - ", ?, ?, " + Instant.now().toEpochMilli() + ")"; - closeStmt(pst); - pst = sqlGenerator.prepareStmtWithParameters(dbConn, insertQ, params); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute update <" + insertQ.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tableName)); - } - pst.executeUpdate(); - LOG.debug("Going to commit"); - dbConn.commit(); - return new LockResponse(txnId, LockState.ACQUIRED); - } catch (SQLException ex) { - LOG.warn("lockMaterializationRebuild failed due to " + getMessage(ex), ex); - throw new MetaException("Unable to retrieve materialization invalidation information due to " + - StringUtils.stringifyException(ex)); - } finally { - close(rs, pst, dbConn); - if(handle != null) { - handle.releaseLocks(); - } - unlockInternal(); - } - } - - @Override - public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) - throws MetaException { - try { - Connection dbConn = null; - PreparedStatement pst = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - String s = "UPDATE \"MATERIALIZATION_REBUILD_LOCKS\"" + - " SET \"MRL_LAST_HEARTBEAT\" = " + Instant.now().toEpochMilli() + - " WHERE \"MRL_TXN_ID\" = " + txnId + - " AND \"MRL_DB_NAME\" = ?" + - " AND \"MRL_TBL_NAME\" = ?"; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, Arrays.asList(dbName, tableName)); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute update <" + s.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tableName)); - } - int rc = pst.executeUpdate(); - if (rc < 1) { - LOG.debug("Going to rollback"); - dbConn.rollback(); - LOG.info("No lock found for rebuild of {} when trying to heartbeat", TableName.getDbTable(dbName, tableName)); - // It could not be renewed, return that information - return false; - } - LOG.debug("Going to commit"); - dbConn.commit(); - // It could be renewed, return that information - return true; - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, - "heartbeatLockMaterializationRebuild(" + TableName.getDbTable(dbName, tableName) + ", " + txnId + ")"); - throw new MetaException("Unable to heartbeat rebuild lock due to " + - StringUtils.stringifyException(e)); - } finally { - close(null, pst, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - return heartbeatLockMaterializationRebuild(dbName, tableName ,txnId); - } - } - - @Override - public long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) throws MetaException { - try { - // Aux values - long cnt = 0L; - List txnIds = new ArrayList<>(); - long timeoutTime = Instant.now().toEpochMilli() - timeout; - - Connection dbConn = null; - Statement stmt = null; - ResultSet rs = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - - String selectQ = "SELECT \"MRL_TXN_ID\", \"MRL_LAST_HEARTBEAT\" FROM \"MATERIALIZATION_REBUILD_LOCKS\""; - LOG.debug("Going to execute query <{}>", selectQ); - rs = stmt.executeQuery(selectQ); - while(rs.next()) { - long lastHeartbeat = rs.getLong(2); - if (lastHeartbeat < timeoutTime) { - // The heartbeat has timeout, double check whether we can remove it - long txnId = rs.getLong(1); - if (validTxnList.isTxnValid(txnId) || validTxnList.isTxnAborted(txnId)) { - // Txn was committed (but notification was not received) or it was aborted. - // Either case, we can clean it up - txnIds.add(txnId); - } - } - } - if (!txnIds.isEmpty()) { - String deleteQ = "DELETE FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE" + - " \"MRL_TXN_ID\" IN(" + StringUtils.join(",", txnIds) + ") "; - LOG.debug("Going to execute update <{}>", deleteQ); - cnt = stmt.executeUpdate(deleteQ); - } - LOG.debug("Going to commit"); - dbConn.commit(); - return cnt; - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "cleanupMaterializationRebuildLocks"); - throw new MetaException("Unable to clean rebuild locks due to " + - StringUtils.stringifyException(e)); - } finally { - close(rs, stmt, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - return cleanupMaterializationRebuildLocks(validTxnList, timeout); - } - } - - /** - * As much as possible (i.e. in absence of retries) we want both operations to be done on the same - * connection (but separate transactions). - * - * Retry-by-caller note: If the call to lock is from a transaction, then in the worst case - * there will be a duplicate set of locks but both sets will belong to the same txn so they - * will not conflict with each other. For locks w/o txn context (i.e. read-only query), this - * may lead to deadlock (at least a long wait). (e.g. 1st call creates locks in {@code LOCK_WAITING} - * mode and response gets lost. Then {@link org.apache.hadoop.hive.metastore.RetryingMetaStoreClient} - * retries, and enqueues another set of locks in LOCK_WAITING. The 2nd LockResponse is delivered - * to the DbLockManager, which will keep dong {@link #checkLock(CheckLockRequest)} until the 1st - * set of locks times out. - */ - @RetrySemantics.CannotRetry - public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException { - ConnectionLockIdPair connAndLockId = enqueueLockWithRetry(rqst); - try { - return checkLockWithRetry(connAndLockId.dbConn, connAndLockId.extLockId, rqst.getTxnid(), - rqst.isZeroWaitReadEnabled(), rqst.isExclusiveCTAS()); - } - catch(NoSuchLockException e) { - // This should never happen, as we just added the lock id - throw new MetaException("Couldn't find a lock we just created! " + e.getMessage()); - } - } - private static final class ConnectionLockIdPair { - private final Connection dbConn; - private final long extLockId; - private ConnectionLockIdPair(Connection dbConn, long extLockId) { - this.dbConn = dbConn; - this.extLockId = extLockId; - } - } - - /** - * Note that by definition select for update is divorced from update, i.e. you executeQuery() to read - * and then executeUpdate(). One other alternative would be to actually update the row in TXNS but - * to the same value as before thus forcing db to acquire write lock for duration of the transaction. - * - * SELECT ... FOR UPDATE locks the row until the transaction commits or rolls back. - * Second connection using `SELECT ... FOR UPDATE` will suspend until the lock is released. - * @return the txnType wrapped in an {@link Optional} - * @throws SQLException - * @throws MetaException - */ - private TxnType getOpenTxnTypeAndLock(Statement stmt, long txnId) throws SQLException, MetaException { - String query = "SELECT \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_ID\" = " + txnId - + " AND \"TXN_STATE\" = " + TxnStatus.OPEN; - try (ResultSet rs = stmt.executeQuery(sqlGenerator.addForUpdateClause(query))) { - return rs.next() ? TxnType.findByValue(rs.getInt(1)) : null; - } - } - - /** - * This enters locks into the queue in {@link #LOCK_WAITING} mode. - * - * Isolation Level Notes: - * 1. We use S4U (withe read_committed) to generate the next (ext) lock id. This serializes - * any 2 {@code enqueueLockWithRetry()} calls. - * 2. We use S4U on the relevant TXNS row to block any concurrent abort/commit/etc operations - * @see #checkLockWithRetry(Connection, long, long, boolean, boolean) - */ - private ConnectionLockIdPair enqueueLockWithRetry(LockRequest rqst) - throws NoSuchTxnException, TxnAbortedException, MetaException { - boolean success = false; - Connection dbConn = null; - try { - Statement stmt = null; - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - long txnid = rqst.getTxnid(); - stmt = dbConn.createStatement(); - if (isValidTxn(txnid)) { - //this also ensures that txn is still there in expected state - TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid); - if (txnType == null) { - ensureValidTxn(dbConn, txnid, stmt); - shouldNeverHappen(txnid); - } - } - /* Insert txn components and hive locks (with a temp extLockId) first, before getting the next lock ID in a select-for-update. - This should minimize the scope of the S4U and decrease the table lock duration. */ - insertTxnComponents(txnid, rqst, dbConn); - long tempExtLockId = insertHiveLocksWithTemporaryExtLockId(txnid, dbConn, rqst); - - /** Get the next lock id. - * This has to be atomic with adding entries to HIVE_LOCK entries (1st add in W state) to prevent a race. - * Suppose ID gen is a separate txn and 2 concurrent lock() methods are running. 1st one generates nl_next=7, - * 2nd nl_next=8. Then 8 goes first to insert into HIVE_LOCKS and acquires the locks. Then 7 unblocks, - * and add it's W locks but it won't see locks from 8 since to be 'fair' {@link #checkLock(java.sql.Connection, long)} - * doesn't block on locks acquired later than one it's checking*/ - long extLockId = getNextLockIdForUpdate(dbConn, stmt); - incrementLockIdAndUpdateHiveLocks(stmt, extLockId, tempExtLockId); - - dbConn.commit(); - success = true; - return new ConnectionLockIdPair(dbConn, extLockId); - } catch (SQLException e) { - LOG.error("enqueueLock failed for request: {}. Exception msg: {}", rqst, getMessage(e)); - rollbackDBConn(dbConn); - checkRetryable(e, "enqueueLockWithRetry(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " + - StringUtils.stringifyException(e)); - } finally { - closeStmt(stmt); - if (!success) { - /* This needs to return a "live" connection to be used by operation that follows it. - Thus it only closes Connection on failure/retry. */ - closeDbConn(dbConn); - } - unlockInternal(); - } - } - catch(RetryException e) { - LOG.debug("Going to retry enqueueLock for request: {}, after catching RetryException with message: {}", - rqst, e.getMessage()); - return enqueueLockWithRetry(rqst); - } - } - - private long getNextLockIdForUpdate(Connection dbConn, Statement stmt) throws SQLException, MetaException { - String s = sqlGenerator.addForUpdateClause("SELECT \"NL_NEXT\" FROM \"NEXT_LOCK_ID\""); - LOG.debug("Going to execute query <{}>", s); - try (ResultSet rs = stmt.executeQuery(s)) { - if (!rs.next()) { - LOG.error("Failure to get next lock ID for update! SELECT query returned empty ResultSet."); - dbConn.rollback(); - throw new MetaException("Transaction tables not properly " + - "initialized, no record found in next_lock_id"); - } - return rs.getLong(1); - } - } - - private void incrementLockIdAndUpdateHiveLocks(Statement stmt, long extLockId, long tempId) throws SQLException { - String incrCmd = String.format(INCREMENT_NEXT_LOCK_ID_QUERY, (extLockId + 1)); - // update hive locks entries with the real EXT_LOCK_ID (replace temp ID) - String updateLocksCmd = String.format(UPDATE_HIVE_LOCKS_EXT_ID_QUERY, extLockId, tempId); - LOG.debug("Going to execute updates in batch: <{}>, and <{}>", incrCmd, updateLocksCmd); - stmt.addBatch(incrCmd); - stmt.addBatch(updateLocksCmd); - stmt.executeBatch(); - } - - private void insertTxnComponents(long txnid, LockRequest rqst, Connection dbConn) throws SQLException { - if (txnid > 0) { - Map, Optional> writeIdCache = new HashMap<>(); - try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_INSERT_QUERY)) { - // For each component in this lock request, - // add an entry to the txn_components table - int insertCounter = 0; - - Predicate isDynPart = lc -> lc.isSetIsDynamicPartitionWrite() && lc.isIsDynamicPartitionWrite(); - Function> groupKey = lc -> - Pair.of(normalizeCase(lc.getDbname()), normalizeCase(lc.getTablename())); - - Set> isDynPartUpdate = rqst.getComponent().stream().filter(isDynPart) - .filter(lc -> lc.getOperationType() == DataOperationType.UPDATE || lc.getOperationType() == DataOperationType.DELETE) - .map(groupKey) - .collect(Collectors.toSet()); - - for (LockComponent lc : rqst.getComponent()) { - if (lc.isSetIsTransactional() && !lc.isIsTransactional()) { - //we don't prevent using non-acid resources in a txn but we do lock them - continue; - } - if (!shouldUpdateTxnComponent(txnid, rqst, lc)) { - continue; - } - String dbName = normalizeCase(lc.getDbname()); - String tblName = normalizeCase(lc.getTablename()); - String partName = normalizePartitionCase(lc.getPartitionname()); - OperationType opType = OperationType.fromDataOperationType(lc.getOperationType()); - - if (isDynPart.test(lc)) { - partName = null; - if (writeIdCache.containsKey(groupKey.apply(lc))) { - continue; - } - opType = isDynPartUpdate.contains(groupKey.apply(lc)) ? OperationType.UPDATE : OperationType.INSERT; - } - Optional writeId = getWriteId(writeIdCache, dbName, tblName, txnid, dbConn); - - pstmt.setLong(1, txnid); - pstmt.setString(2, dbName); - pstmt.setString(3, tblName); - pstmt.setString(4, partName); - pstmt.setString(5, opType.getSqlConst()); - pstmt.setObject(6, writeId.orElse(null)); - - pstmt.addBatch(); - insertCounter++; - if (insertCounter % maxBatchSize == 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY, maxBatchSize); - pstmt.executeBatch(); - } - } - if (insertCounter % maxBatchSize != 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY, - insertCounter % maxBatchSize); - pstmt.executeBatch(); - } - } - } - } - - private Optional getWriteId(Map, Optional> writeIdCache, String dbName, String tblName, long txnid, Connection dbConn) throws SQLException { - /* we can cache writeIDs based on dbName and tblName because txnid is invariant and - partitionName is not part of the writeID select query */ - Pair dbAndTable = Pair.of(dbName, tblName); - if (writeIdCache.containsKey(dbAndTable)) { - return writeIdCache.get(dbAndTable); - } else { - Optional writeId = getWriteIdFromDb(txnid, dbConn, dbName, tblName); - writeIdCache.put(dbAndTable, writeId); - return writeId; - } - } - - private Optional getWriteIdFromDb(long txnid, Connection dbConn, String dbName, String tblName) throws SQLException { - if (tblName != null) { - // It is assumed the caller have already allocated write id for adding/updating data to - // the acid tables. However, DDL operatons won't allocate write id and hence this query - // may return empty result sets. - // Get the write id allocated by this txn for the given table writes - try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_WRITE_ID_QUERY)) { - pstmt.setString(1, dbName); - pstmt.setString(2, tblName); - pstmt.setLong(3, txnid); - LOG.debug("Going to execute query <{}>", SELECT_WRITE_ID_QUERY); - try (ResultSet rs = pstmt.executeQuery()) { - if (rs.next()) { - return Optional.of(rs.getLong(1)); - } - } - } - } - return Optional.empty(); - } - - private boolean shouldUpdateTxnComponent(long txnid, LockRequest rqst, LockComponent lc) { - if(!lc.isSetOperationType()) { - //request came from old version of the client - return true; //this matches old behavior - } - else { - switch (lc.getOperationType()) { - case INSERT: - case UPDATE: - case DELETE: - return true; - case SELECT: - return false; - case NO_TXN: - /*this constant is a bit of a misnomer since we now always have a txn context. It - just means the operation is such that we don't care what tables/partitions it - affected as it doesn't trigger a compaction or conflict detection. A better name - would be NON_TRANSACTIONAL.*/ - return false; - default: - //since we have an open transaction, only 4 values above are expected - throw new IllegalStateException("Unexpected DataOperationType: " + lc.getOperationType() - + " agentInfo=" + rqst.getAgentInfo() + " " + JavaUtils.txnIdToString(txnid)); - } - } - } - - private long insertHiveLocksWithTemporaryExtLockId(long txnid, Connection dbConn, LockRequest rqst) throws MetaException, SQLException { - - String lastHB = isValidTxn(txnid) ? "0" : getEpochFn(dbProduct); - String insertLocksQuery = String.format(HIVE_LOCKS_INSERT_QRY, lastHB); - long intLockId = 0; - long tempExtLockId = generateTemporaryId(); - - try (PreparedStatement pstmt = dbConn.prepareStatement(insertLocksQuery)) { - for (LockComponent lc : rqst.getComponent()) { - intLockId++; - String lockType = LockTypeUtil.getEncodingAsStr(lc.getType()); - - pstmt.setLong(1, tempExtLockId); - pstmt.setLong(2, intLockId); - pstmt.setLong(3, txnid); - pstmt.setString(4, normalizeCase(lc.getDbname())); - pstmt.setString(5, normalizeCase(lc.getTablename())); - pstmt.setString(6, normalizePartitionCase(lc.getPartitionname())); - pstmt.setString(7, Character.toString(LOCK_WAITING)); - pstmt.setString(8, lockType); - pstmt.setString(9, rqst.getUser()); - pstmt.setString(10, rqst.getHostname()); - pstmt.setString(11, rqst.getAgentInfo()); - - pstmt.addBatch(); - if (intLockId % maxBatchSize == 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", insertLocksQuery, maxBatchSize); - pstmt.executeBatch(); - } - } - if (intLockId % maxBatchSize != 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", insertLocksQuery, intLockId % maxBatchSize); - pstmt.executeBatch(); - } - } - return tempExtLockId; - } - - private long generateTemporaryId() { - return -1 * ThreadLocalRandom.current().nextLong(); - } - - private static String normalizeCase(String s) { - return s == null ? null : s.toLowerCase(); - } - - private static String normalizePartitionCase(String s) { - if (s == null) { - return null; - } - Map map = Splitter.on(Path.SEPARATOR).withKeyValueSeparator('=').split(s); - return FileUtils.makePartName(new ArrayList<>(map.keySet()), new ArrayList<>(map.values())); - } - - private LockResponse checkLockWithRetry(Connection dbConn, long extLockId, long txnId, boolean zeroWaitReadEnabled, - boolean isExclusiveCTAS) - throws NoSuchLockException, TxnAbortedException, MetaException { - try { - try { - lockInternal(); - if(dbConn.isClosed()) { - //should only get here if retrying this op - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - } - return checkLock(dbConn, extLockId, txnId, zeroWaitReadEnabled, isExclusiveCTAS); - } catch (SQLException e) { - LOG.error("checkLock failed for extLockId={}/txnId={}. Exception msg: {}", extLockId, txnId, getMessage(e)); - rollbackDBConn(dbConn); - checkRetryable(e, "checkLockWithRetry(" + extLockId + "," + txnId + ")"); - throw new MetaException("Unable to update transaction database " + - StringUtils.stringifyException(e)); - } finally { - unlockInternal(); - closeDbConn(dbConn); - } - } - catch(RetryException e) { - LOG.debug("Going to retry checkLock for extLockId={}/txnId={} after catching RetryException with message: {}", - extLockId, txnId, e.getMessage()); - return checkLockWithRetry(dbConn, extLockId, txnId, zeroWaitReadEnabled, isExclusiveCTAS); - } - } - /** - * Why doesn't this get a txnid as parameter? The caller should either know the txnid or know there isn't one. - * Either way getTxnIdFromLockId() will not be needed. This would be a Thrift change. - * - * Also, when lock acquisition returns WAITING, it's retried every 15 seconds (best case, see DbLockManager.backoff(), - * in practice more often) - * which means this is heartbeating way more often than hive.txn.timeout and creating extra load on DB. - * - * The clients that operate in blocking mode, can't heartbeat a lock until the lock is acquired. - * We should make CheckLockRequest include timestamp or last request to skip unnecessary heartbeats. Thrift change. - * - * {@link #checkLock(java.sql.Connection, long, long, boolean, boolean)} must run at SERIALIZABLE - * (make sure some lock we are checking against doesn't move from W to A in another txn) - * but this method can heartbeat in separate txn at READ_COMMITTED. - * - * Retry-by-caller note: - * Retryable because {@link #checkLock(Connection, long, long, boolean, boolean)} is - */ - @Override - @RetrySemantics.SafeToRetry - public LockResponse checkLock(CheckLockRequest rqst) - throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException { - try { - Connection dbConn = null; - long extLockId = rqst.getLockid(); - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - // Heartbeat on the lockid first, to assure that our lock is still valid. - // Then look up the lock info (hopefully in the cache). If these locks - // are associated with a transaction then heartbeat on that as well. - LockInfo lockInfo = getLockFromLockId(dbConn, extLockId) - .orElseThrow(() -> new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId))); - if (lockInfo.txnId > 0) { - heartbeatTxn(dbConn, lockInfo.txnId); - } - else { - heartbeatLock(dbConn, extLockId); - } - //todo: strictly speaking there is a bug here. heartbeat*() commits but both heartbeat and - //checkLock() are in the same retry block, so if checkLock() throws, heartbeat is also retired - //extra heartbeat is logically harmless, but ... - return checkLock(dbConn, extLockId, lockInfo.txnId, false, false); - } catch (SQLException e) { - LOG.error("checkLock failed for request={}. Exception msg: {}", rqst, getMessage(e)); - rollbackDBConn(dbConn); - checkRetryable(e, "checkLock(" + rqst + " )"); - throw new MetaException("Unable to update transaction database " + - JavaUtils.lockIdToString(extLockId) + " " + StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - unlockInternal(); - } - } catch (RetryException e) { - LOG.debug("Going to retry checkLock for request={} after catching RetryException with message: {}", - rqst, e.getMessage()); - return checkLock(rqst); - } - - } - - /** - * This would have been made simpler if all locks were associated with a txn. Then only txn needs to - * be heartbeated, committed, etc. no need for client to track individual locks. - * When removing locks not associated with txn this potentially conflicts with - * heartbeat/performTimeout which are update/delete of HIVE_LOCKS thus will be locked as needed by db. - * since this only removes from HIVE_LOCKS at worst some lock acquire is delayed - */ - @RetrySemantics.Idempotent - public void unlock(UnlockRequest rqst) throws TxnOpenException, MetaException { - try { - Connection dbConn = null; - Statement stmt = null; - long extLockId = rqst.getLockid(); - try { - /** - * This method is logically like commit for read-only auto commit queries. - * READ_COMMITTED since this only has 1 delete statement and no new entries with the - * same hl_lock_ext_id can be added, i.e. all rows with a given hl_lock_ext_id are - * created in a single atomic operation. - * Theoretically, this competes with {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)} - * but hl_lock_ext_id is not known until that method returns. - * Also competes with {@link #checkLock(org.apache.hadoop.hive.metastore.api.CheckLockRequest)} - * but using SERIALIZABLE doesn't materially change the interaction. - * If "delete" stmt misses, additional logic is best effort to produce meaningful error msg. - */ - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - //hl_txnid <> 0 means it's associated with a transaction - String s = "DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + " AND (\"HL_TXNID\" = 0 OR" + - " (\"HL_TXNID\" <> 0 AND \"HL_LOCK_STATE\" = '" + LOCK_WAITING + "'))"; - //(hl_txnid <> 0 AND hl_lock_state = '" + LOCK_WAITING + "') is for multi-statement txns where - //some query attempted to lock (thus LOCK_WAITING state) but is giving up due to timeout for example - LOG.debug("Going to execute update <{}>", s); - int rc = stmt.executeUpdate(s); - if (rc < 1) { - LOG.info("Failure to unlock any locks with extLockId={}.", extLockId); - dbConn.rollback(); - Optional optLockInfo = getLockFromLockId(dbConn, extLockId); - if (!optLockInfo.isPresent()) { - //didn't find any lock with extLockId but at ReadCommitted there is a possibility that - //it existed when above delete ran but it didn't have the expected state. - LOG.info("No lock in {} mode found for unlock({})", LOCK_WAITING, - JavaUtils.lockIdToString(rqst.getLockid())); - //bail here to make the operation idempotent - return; - } - LockInfo lockInfo = optLockInfo.get(); - if (isValidTxn(lockInfo.txnId)) { - String msg = "Unlocking locks associated with transaction not permitted. " + lockInfo; - //if a lock is associated with a txn we can only "unlock" if if it's in WAITING state - // which really means that the caller wants to give up waiting for the lock - LOG.error(msg); - throw new TxnOpenException(msg); - } else { - //we didn't see this lock when running DELETE stmt above but now it showed up - //so should "should never happen" happened... - String msg = "Found lock in unexpected state " + lockInfo; - LOG.error(msg); - throw new MetaException(msg); - } - } - LOG.debug("Successfully unlocked at least 1 lock with extLockId={}", extLockId); - dbConn.commit(); - } catch (SQLException e) { - LOG.error("Unlock failed for request={}. Exception msg: {}", rqst, getMessage(e)); - rollbackDBConn(dbConn); - checkRetryable(e, "unlock(" + rqst + ")"); - throw new MetaException("Unable to update transaction database " + - JavaUtils.lockIdToString(extLockId) + " " + StringUtils.stringifyException(e)); - } finally { - closeStmt(stmt); - closeDbConn(dbConn); - } - } catch (RetryException e) { - unlock(rqst); - } - } - - /** - * used to sort entries in {@link org.apache.hadoop.hive.metastore.api.ShowLocksResponse} - */ - private static class LockInfoExt extends LockInfo { - private final ShowLocksResponseElement e; - LockInfoExt(ShowLocksResponseElement e) { - super(e); - this.e = e; - } - } - @RetrySemantics.ReadOnly - public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException { - try { - Connection dbConn = null; - ShowLocksResponse rsp = new ShowLocksResponse(); - List elems = new ArrayList<>(); - List sortedList = new ArrayList<>(); - PreparedStatement pst = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - String s = "SELECT \"HL_LOCK_EXT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", " + - "\"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", \"HL_ACQUIRED_AT\", \"HL_USER\", \"HL_HOST\", \"HL_LOCK_INT_ID\"," + - "\"HL_BLOCKEDBY_EXT_ID\", \"HL_BLOCKEDBY_INT_ID\", \"HL_AGENT_INFO\" FROM \"HIVE_LOCKS\""; - - // Some filters may have been specified in the SHOW LOCKS statement. Add them to the query. - String dbName = rqst.getDbname(); - String tableName = rqst.getTablename(); - String partName = rqst.getPartname(); - List params = new ArrayList<>(); - - StringBuilder filter = new StringBuilder(); - if (dbName != null && !dbName.isEmpty()) { - filter.append("\"HL_DB\"=?"); - params.add(dbName); - } - if (tableName != null && !tableName.isEmpty()) { - if (filter.length() > 0) { - filter.append(" and "); - } - filter.append("\"HL_TABLE\"=?"); - params.add(tableName); - } - if (partName != null && !partName.isEmpty()) { - if (filter.length() > 0) { - filter.append(" and "); - } - filter.append("\"HL_PARTITION\"=?"); - params.add(partName); - } - if (rqst.isSetTxnid()) { - if (filter.length() > 0) { - filter.append(" and "); - } - filter.append("\"HL_TXNID\"=" + rqst.getTxnid()); - } - String whereClause = filter.toString(); - - if (!whereClause.isEmpty()) { - s = s + " where " + whereClause; - } - - pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params); - LOG.debug("Going to execute query <{}>", s); - ResultSet rs = pst.executeQuery(); - while (rs.next()) { - ShowLocksResponseElement e = new ShowLocksResponseElement(); - e.setLockid(rs.getLong(1)); - long txnid = rs.getLong(2); - if (!rs.wasNull()) e.setTxnid(txnid); - e.setDbname(rs.getString(3)); - e.setTablename(rs.getString(4)); - String partition = rs.getString(5); - if (partition != null) e.setPartname(partition); - switch (rs.getString(6).charAt(0)) { - case LOCK_ACQUIRED: e.setState(LockState.ACQUIRED); break; - case LOCK_WAITING: e.setState(LockState.WAITING); break; - default: throw new MetaException("Unknown lock state " + rs.getString(6).charAt(0)); - } - - char lockChar = rs.getString(7).charAt(0); - LockType lockType = LockTypeUtil.getLockTypeFromEncoding(lockChar) - .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar)); - e.setType(lockType); - - e.setLastheartbeat(rs.getLong(8)); - long acquiredAt = rs.getLong(9); - if (!rs.wasNull()) e.setAcquiredat(acquiredAt); - e.setUser(rs.getString(10)); - e.setHostname(rs.getString(11)); - e.setLockIdInternal(rs.getLong(12)); - long id = rs.getLong(13); - if(!rs.wasNull()) { - e.setBlockedByExtId(id); - } - id = rs.getLong(14); - if(!rs.wasNull()) { - e.setBlockedByIntId(id); - } - e.setAgentInfo(rs.getString(15)); - sortedList.add(new LockInfoExt(e)); - } - } catch (SQLException e) { - checkRetryable(e, "showLocks(" + rqst + ")"); - throw new MetaException("Unable to select from transaction database " + - StringUtils.stringifyException(e)); - } finally { - closeStmt(pst); - closeDbConn(dbConn); - } - //this ensures that "SHOW LOCKS" prints the locks in the same order as they are examined - //by checkLock() - makes diagnostics easier. - Collections.sort(sortedList, new LockInfoComparator()); - for(LockInfoExt lockInfoExt : sortedList) { - elems.add(lockInfoExt.e); - } - rsp.setLocks(elems); - return rsp; - } catch (RetryException e) { - return showLocks(rqst); - } - } - - /** - * {@code ids} should only have txnid or lockid but not both, ideally. - * Currently DBTxnManager.heartbeat() enforces this. - */ - @Override - @RetrySemantics.SafeToRetry - public void heartbeat(HeartbeatRequest ids) - throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException { - try { - Connection dbConn = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - heartbeatLock(dbConn, ids.getLockid()); - heartbeatTxn(dbConn, ids.getTxnid()); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "heartbeat(" + ids + ")"); - throw new MetaException("Unable to select from transaction database " + - StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - } - } catch (RetryException e) { - heartbeat(ids); - } - } - @Override - @RetrySemantics.SafeToRetry - public HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) - throws MetaException { - try { - Connection dbConn = null; - Statement stmt = null; - HeartbeatTxnRangeResponse rsp = new HeartbeatTxnRangeResponse(); - Set nosuch = new HashSet<>(); - Set aborted = new HashSet<>(); - rsp.setNosuch(nosuch); - rsp.setAborted(aborted); - try { - /** - * READ_COMMITTED is sufficient since {@link #heartbeatTxn(java.sql.Connection, long)} - * only has 1 update statement in it and - * we only update existing txns, i.e. nothing can add additional txns that this operation - * would care about (which would have required SERIALIZABLE) - */ - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - /*do fast path first (in 1 statement) if doesn't work, rollback and do the long version*/ - stmt = dbConn.createStatement(); - List queries = new ArrayList<>(); - int numTxnsToHeartbeat = (int) (rqst.getMax() - rqst.getMin() + 1); - List txnIds = new ArrayList<>(numTxnsToHeartbeat); - for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) { - txnIds.add(txn); - } - TxnUtils.buildQueryWithINClause(conf, queries, - new StringBuilder("UPDATE \"TXNS\" SET \"TXN_LAST_HEARTBEAT\" = " + getEpochFn(dbProduct) + - " WHERE \"TXN_STATE\" = " + TxnStatus.OPEN + " AND "), - new StringBuilder(""), txnIds, "\"TXN_ID\"", true, false); - int updateCnt = 0; - for (String query : queries) { - LOG.debug("Going to execute update <{}>", query); - updateCnt += stmt.executeUpdate(query); - } - if (updateCnt == numTxnsToHeartbeat) { - //fast pass worked, i.e. all txns we were asked to heartbeat were Open as expected - dbConn.commit(); - return rsp; - } - //if here, do the slow path so that we can return info txns which were not in expected state - dbConn.rollback(); - for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) { - try { - heartbeatTxn(dbConn, txn); - } catch (NoSuchTxnException e) { - nosuch.add(txn); - } catch (TxnAbortedException e) { - aborted.add(txn); - } - } - return rsp; - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "heartbeatTxnRange(" + rqst + ")"); - throw new MetaException("Unable to select from transaction database " + - StringUtils.stringifyException(e)); - } finally { - close(null, stmt, dbConn); - } - } catch (RetryException e) { - return heartbeatTxnRange(rqst); - } - } - - @Deprecated - long generateCompactionQueueId(Statement stmt) throws SQLException, MetaException { - // Get the id for the next entry in the queue - String s = sqlGenerator.addForUpdateClause("SELECT \"NCQ_NEXT\" FROM \"NEXT_COMPACTION_QUEUE_ID\""); - LOG.debug("going to execute query <{}>", s); - try (ResultSet rs = stmt.executeQuery(s)) { - if (!rs.next()) { - throw new IllegalStateException("Transaction tables not properly initiated, " - + "no record found in next_compaction_queue_id"); - } - long id = rs.getLong(1); - s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id; - LOG.debug("Going to execute update <{}>", s); - if (stmt.executeUpdate(s) != 1) { - //TODO: Eliminate this id generation by implementing: https://issues.apache.org/jira/browse/HIVE-27121 - LOG.info("The returned compaction ID ({}) already taken, obtaining new", id); - return generateCompactionQueueId(stmt); - } - return id; - } - } - - long generateCompactionQueueId() throws MetaException { - // Get the id for the next entry in the queue - String sql = sqlGenerator.addForUpdateClause("SELECT \"NCQ_NEXT\" FROM \"NEXT_COMPACTION_QUEUE_ID\""); - LOG.debug("going to execute SQL <{}>", sql); - - Long allocatedId = jdbcResource.getJdbcTemplate().query(sql, rs -> { - if (!rs.next()) { - throw new IllegalStateException("Transaction tables not properly initiated, " - + "no record found in next_compaction_queue_id"); - } - long id = rs.getLong(1); - - int count = jdbcResource.getJdbcTemplate().update("UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = :newId WHERE \"NCQ_NEXT\" = :id", - new MapSqlParameterSource() - .addValue("id", id) - .addValue("newId", id + 1)); - - if (count != 1) { - //TODO: Eliminate this id generation by implementing: https://issues.apache.org/jira/browse/HIVE-27121 - LOG.info("The returned compaction ID ({}) already taken, obtaining new", id); - return null; - } - return id; - }); - if (allocatedId == null) { - return generateCompactionQueueId(); - } else { - return allocatedId; - } - } - - - @Override - @RetrySemantics.ReadOnly - public long getTxnIdForWriteId( - String dbName, String tblName, long writeId) throws MetaException { - try { - Connection dbConn = null; - PreparedStatement pst = null; - try { - /** - * This runs at READ_COMMITTED for exactly the same reason as {@link #getOpenTxnsInfo()} - */ - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - String query = "SELECT \"T2W_TXNID\" FROM \"TXN_TO_WRITE_ID\" WHERE" - + " \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND \"T2W_WRITEID\" = " + writeId; - pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, Arrays.asList(dbName, tblName)); - if (LOG.isDebugEnabled()) { - LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">", - quoteString(dbName), quoteString(tblName)); - } - ResultSet rs = pst.executeQuery(); - long txnId = -1; - if (rs.next()) { - txnId = rs.getLong(1); - } - return txnId; - } catch (SQLException e) { - checkRetryable(e, "getTxnIdForWriteId"); - throw new MetaException("Unable to select from transaction database, " - + StringUtils.stringifyException(e)); - } finally { - close(null, pst, dbConn); - } - } catch (RetryException e) { - return getTxnIdForWriteId(dbName, tblName, writeId); - } - } - - @Override - @RetrySemantics.Idempotent - public CompactionResponse compact(CompactionRequest rqst) throws MetaException { - // Put a compaction request in the queue. - try { - TxnStore.MutexAPI.LockHandle handle = null; - try { - lockInternal(); - /** - * MUTEX_KEY.CompactionScheduler lock ensures that there is only 1 entry in - * Initiated/Working state for any resource. This ensures that we don't run concurrent - * compactions for any resource. - */ - handle = getMutexAPI().acquireLock(MUTEX_KEY.CompactionScheduler.name()); - - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) { - try (Statement stmt = dbConn.createStatement()) { - - long id = generateCompactionQueueId(stmt); - - GetValidWriteIdsRequest request = new GetValidWriteIdsRequest( - Collections.singletonList(getFullTableName(rqst.getDbname(), rqst.getTablename()))); - final ValidCompactorWriteIdList tblValidWriteIds = - TxnUtils.createValidCompactWriteIdList(getValidWriteIds(request).getTblValidWriteIds().get(0)); - if (LOG.isDebugEnabled()) { - LOG.debug("ValidCompactWriteIdList: {}", tblValidWriteIds.writeToString()); - } - - StringBuilder sb = new StringBuilder("SELECT \"CQ_ID\", \"CQ_STATE\" FROM \"COMPACTION_QUEUE\" WHERE"). - append(" (\"CQ_STATE\" IN("). - append(quoteChar(INITIATED_STATE)).append(",").append(quoteChar(WORKING_STATE)). - append(") OR (\"CQ_STATE\" = ").append(quoteChar(READY_FOR_CLEANING)). - append(" AND \"CQ_HIGHEST_WRITE_ID\" = ?))"). - append(" AND \"CQ_DATABASE\"=?"). - append(" AND \"CQ_TABLE\"=?").append(" AND "); - if(rqst.getPartitionname() == null) { - sb.append("\"CQ_PARTITION\" is null"); - } else { - sb.append("\"CQ_PARTITION\"=?"); - } - - try (PreparedStatement pst = dbConn.prepareStatement(sqlGenerator.addEscapeCharacters(sb.toString()))) { - pst.setLong(1, tblValidWriteIds.getHighWatermark()); - pst.setString(2, rqst.getDbname()); - pst.setString(3, rqst.getTablename()); - if (rqst.getPartitionname() != null) { - pst.setString(4, rqst.getPartitionname()); - } - LOG.debug("Going to execute query <{}>", sb); - try (ResultSet rs = pst.executeQuery()) { - if(rs.next()) { - long enqueuedId = rs.getLong(1); - String state = CompactionState.fromSqlConst(rs.getString(2)).toString(); - LOG.info("Ignoring request to compact {}/{}/{} since it is already {} with id={}", rqst.getDbname(), - rqst.getTablename(), rqst.getPartitionname(), quoteString(state), enqueuedId); - CompactionResponse resp = new CompactionResponse(-1, REFUSED_RESPONSE, false); - resp.setErrormessage("Compaction is already scheduled with state=" + quoteString(state) + - " and id=" + enqueuedId); - return resp; - } - } - } - List params = new ArrayList<>(); - StringBuilder buf = new StringBuilder("INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", " + - "\"CQ_TABLE\", "); - String partName = rqst.getPartitionname(); - if (partName != null) buf.append("\"CQ_PARTITION\", "); - buf.append("\"CQ_STATE\", \"CQ_TYPE\", \"CQ_ENQUEUE_TIME\", \"CQ_POOL_NAME\""); - if (rqst.isSetNumberOfBuckets()) { - buf.append(", \"CQ_NUMBER_OF_BUCKETS\""); - } - if (rqst.isSetOrderByClause()) { - buf.append(", \"CQ_ORDER_BY\""); - } - if (rqst.getProperties() != null) { - buf.append(", \"CQ_TBLPROPERTIES\""); - } - if (rqst.getRunas() != null) { - buf.append(", \"CQ_RUN_AS\""); - } - if (rqst.getInitiatorId() != null) { - buf.append(", \"CQ_INITIATOR_ID\""); - } - if (rqst.getInitiatorVersion() != null) { - buf.append(", \"CQ_INITIATOR_VERSION\""); - } - buf.append(") values ("); - buf.append(id); - buf.append(", ?"); - buf.append(", ?"); - buf.append(", "); - params.add(rqst.getDbname()); - params.add(rqst.getTablename()); - if (partName != null) { - buf.append("?, '"); - params.add(partName); - } else { - buf.append("'"); - } - buf.append(INITIATED_STATE); - buf.append("', '"); - buf.append(TxnUtils.thriftCompactionType2DbType(rqst.getType())); - buf.append("',"); - buf.append(getEpochFn(dbProduct)); - buf.append(", ?"); - params.add(rqst.getPoolName()); - if (rqst.isSetNumberOfBuckets()) { - buf.append(", ").append(rqst.getNumberOfBuckets()); - } - if (rqst.isSetOrderByClause()) { - buf.append(", ?"); - params.add(rqst.getOrderByClause()); - } - if (rqst.getProperties() != null) { - buf.append(", ?"); - params.add(new StringableMap(rqst.getProperties()).toString()); - } - if (rqst.getRunas() != null) { - buf.append(", ?"); - params.add(rqst.getRunas()); - } - if (rqst.getInitiatorId() != null) { - buf.append(", ?"); - params.add(rqst.getInitiatorId()); - } - if (rqst.getInitiatorVersion() != null) { - buf.append(", ?"); - params.add(rqst.getInitiatorVersion()); - } - buf.append(")"); - String s = buf.toString(); - - try (PreparedStatement pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params)) { - LOG.debug("Going to execute update <{}>", s); - pst.executeUpdate(); - } - LOG.debug("Going to commit"); - dbConn.commit(); - return new CompactionResponse(id, INITIATED_RESPONSE, true); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - dbConn.rollback(); - throw e; - } - } - } catch (SQLException e) { - checkRetryable(e, "COMPACT(" + rqst + ")"); - throw new MetaException("Unable to put the compaction request into the queue: " + - StringUtils.stringifyException(e)); - } finally { - if (handle != null) { - handle.releaseLocks(); - } - unlockInternal(); - } - } catch (RetryException e) { - return compact(rqst); - } - } - - @Override - @RetrySemantics.SafeToRetry - public boolean submitForCleanup(CompactionRequest rqst, long highestWriteId, long txnId) throws MetaException { - // Put a compaction request in the queue. - try { - Connection dbConn = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - lockInternal(); - - List params = new ArrayList() {{ - add(rqst.getDbname()); - add(rqst.getTablename()); - }}; - long cqId; - try (Statement stmt = dbConn.createStatement()) { - cqId = generateCompactionQueueId(stmt); - } - StringBuilder buf = new StringBuilder( - "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TXN_ID\", \"CQ_ENQUEUE_TIME\", \"CQ_DATABASE\", \"CQ_TABLE\", "); - String partName = rqst.getPartitionname(); - if (partName != null) { - buf.append("\"CQ_PARTITION\", "); - params.add(partName); - } - buf.append("\"CQ_STATE\", \"CQ_TYPE\""); - params.add(String.valueOf(READY_FOR_CLEANING)); - params.add(TxnUtils.thriftCompactionType2DbType(rqst.getType()).toString()); - - if (rqst.getProperties() != null) { - buf.append(", \"CQ_TBLPROPERTIES\""); - params.add(new StringableMap(rqst.getProperties()).toString()); - } - if (rqst.getRunas() != null) { - buf.append(", \"CQ_RUN_AS\""); - params.add(rqst.getRunas()); - } - buf.append(") values (") - .append( - Stream.of(cqId, highestWriteId, txnId, getEpochFn(dbProduct)) - .map(Object::toString) - .collect(Collectors.joining(", "))) - .append(repeat(", ?", params.size())) - .append(")"); - - String s = buf.toString(); - try (PreparedStatement pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params)) { - LOG.debug("Going to execute update <{}>", s); - pst.executeUpdate(); - } - LOG.debug("Going to commit"); - dbConn.commit(); - return true; - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "submitForCleanup(" + rqst + ")"); - throw new MetaException("Failed to submit cleanup request: " + - StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - unlockInternal(); - } - } catch (RetryException e) { - return submitForCleanup(rqst, highestWriteId, txnId); - } - } - - @RetrySemantics.ReadOnly - @SuppressWarnings("squid:S2095") - public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException { - try { - ShowCompactResponse response = new ShowCompactResponse(new ArrayList<>()); - String query = TxnQueries.SHOW_COMPACTION_QUERY + - getShowCompactFilterClause(rqst) + - getShowCompactSortingOrderClause(rqst); - List params = getShowCompactParamList(rqst); - - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - PreparedStatement stmt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params)) { - if (rqst.isSetId()) { - stmt.setLong(1, rqst.getId()); - } - int rowLimit = (int) rqst.getLimit(); - if (rowLimit > 0) { - stmt.setMaxRows(rowLimit); - } - LOG.debug("Going to execute query <" + query + ">"); - try (ResultSet rs = stmt.executeQuery()) { - while (rs.next()) { - ShowCompactResponseElement e = new ShowCompactResponseElement(); - e.setDbname(rs.getString(1)); - e.setTablename(rs.getString(2)); - e.setPartitionname(rs.getString(3)); - e.setState(CompactionState.fromSqlConst(rs.getString(4)).toString()); - try { - e.setType(TxnUtils.dbCompactionType2ThriftType(rs.getString(5).charAt(0))); - } catch (SQLException ex) { - //do nothing to handle RU/D if we add another status - } - e.setWorkerid(rs.getString(6)); - long start = rs.getLong(7); - if (!rs.wasNull()) { - e.setStart(start); - } - long endTime = rs.getLong(8); - if (endTime != -1) { - e.setEndTime(endTime); - } - e.setRunAs(rs.getString(9)); - e.setHadoopJobId(rs.getString(10)); - e.setId(rs.getLong(11)); - e.setErrorMessage(rs.getString(12)); - long enqueueTime = rs.getLong(13); - if (!rs.wasNull()) { - e.setEnqueueTime(enqueueTime); - } - e.setWorkerVersion(rs.getString(14)); - e.setInitiatorId(rs.getString(15)); - e.setInitiatorVersion(rs.getString(16)); - long cleanerStart = rs.getLong(17); - if (!rs.wasNull() && (cleanerStart != -1)) { - e.setCleanerStart(cleanerStart); - } - String poolName = rs.getString(18); - if (isBlank(poolName)) { - e.setPoolName(DEFAULT_POOL_NAME); - } else { - e.setPoolName(poolName); - } - e.setTxnId(rs.getLong(19)); - e.setNextTxnId(rs.getLong(20)); - e.setCommitTime(rs.getLong(21)); - e.setHightestTxnId(rs.getLong(22)); - response.addToCompacts(e); - } - } - } catch (SQLException e) { - checkRetryable(e, "showCompact(" + rqst + ")"); - throw new MetaException("Unable to select from transaction database " + - StringUtils.stringifyException(e)); - } - return response; - } catch (RetryException e) { - return showCompact(rqst); - } - } - - private String getShowCompactSortingOrderClause(ShowCompactRequest request) { - String sortingOrder = request.getOrder(); - return isNotBlank(sortingOrder) ? " ORDER BY " + sortingOrder : TxnQueries.SHOW_COMPACTION_ORDERBY_CLAUSE; - } - - private List getShowCompactParamList(ShowCompactRequest request) throws MetaException { - if (request.getId() > 0) { - return Collections.emptyList(); - } - String poolName = request.getPoolName(); - String dbName = request.getDbName(); - String tableName = request.getTbName(); - String partName = request.getPartName(); - CompactionType type = request.getType(); - String state = request.getState(); - - List params = new ArrayList<>(); - if (isNotBlank(dbName)) { - params.add(dbName); - } - if (isNotBlank(tableName)) { - params.add(tableName); - } - if (isNotBlank(partName)) { - params.add(partName); - } - if (isNotBlank(state)) { - params.add(state); - } - if (type != null) { - params.add(TxnUtils.thriftCompactionType2DbType(type).toString()); - } - if (isNotBlank(poolName)) { - params.add(poolName); - } - return params; - } - - private String getShowCompactFilterClause(ShowCompactRequest request) { - List params = new ArrayList<>(); - - if (request.getId() > 0) { - params.add("\"CC_ID\"=?"); - } else { - if (isNotBlank(request.getDbName())) { - params.add("\"CC_DATABASE\"=?"); - } - if (isNotBlank(request.getTbName())) { - params.add("\"CC_TABLE\"=?"); - } - if (isNotBlank(request.getPartName())) { - params.add("\"CC_PARTITION\"=?"); - } - if (isNotBlank(request.getState())) { - params.add("\"CC_STATE\"=?"); - } - if (request.getType() != null) { - params.add("\"CC_TYPE\"=?"); - } - if (isNotBlank(request.getPoolName())) { - params.add("\"CC_POOL_NAME\"=?"); - } - } - return !params.isEmpty() ? - " WHERE " + StringUtils.join(" AND ", params) : EMPTY; - } - - /** - * We assume this is only called by metadata cache server to know if there are new base/delta files should be read. - * The query filters compactions by state and only returns SUCCEEDED or READY_FOR_CLEANING compactions because - * only these two states means there are new files ready to be read. - */ - @RetrySemantics.ReadOnly - public GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo( - GetLatestCommittedCompactionInfoRequest rqst) throws MetaException { - GetLatestCommittedCompactionInfoResponse response = new GetLatestCommittedCompactionInfoResponse(new ArrayList<>()); - Connection dbConn = null; - PreparedStatement pst = null; - ResultSet rs = null; - try { - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - - List params = new ArrayList<>(); - // This query combines the result sets of SUCCEEDED compactions and READY_FOR_CLEANING compactions - // We also sort the result by CC_ID in descending order so that we can keep only the latest record - // according to the order in result set - StringBuilder sb = new StringBuilder() - .append("SELECT * FROM (") - .append(" SELECT") - .append(" \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_TYPE\"") - .append(" FROM \"COMPLETED_COMPACTIONS\"") - .append(" WHERE \"CC_STATE\" = " + quoteChar(SUCCEEDED_STATE)) - .append(" UNION ALL") - .append(" SELECT") - .append(" \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\"") - .append(" ,\"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\"") - .append(" ,\"CQ_TYPE\" AS \"CC_TYPE\"") - .append(" FROM \"COMPACTION_QUEUE\"") - .append(" WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING)) - .append(") AS compactions ") - .append(" WHERE \"CC_DATABASE\" = ? AND \"CC_TABLE\" = ?"); - params.add(rqst.getDbname()); - params.add(rqst.getTablename()); - if (rqst.getPartitionnamesSize() > 0) { - sb.append(" AND \"CC_PARTITION\" IN ("); - sb.append(String.join(",", - Collections.nCopies(rqst.getPartitionnamesSize(), "?"))); - sb.append(")"); - params.addAll(rqst.getPartitionnames()); - } - if (rqst.isSetLastCompactionId()) { - sb.append(" AND \"CC_ID\" > ?"); - } - sb.append(" ORDER BY \"CC_ID\" DESC"); - - pst = sqlGenerator.prepareStmtWithParameters(dbConn, sb.toString(), params); - if (rqst.isSetLastCompactionId()) { - pst.setLong(params.size() + 1, rqst.getLastCompactionId()); - } - LOG.debug("Going to execute query <{}>", sb); - rs = pst.executeQuery(); - Set partitionSet = new HashSet<>(); - while (rs.next()) { - CompactionInfoStruct lci = new CompactionInfoStruct(); - lci.setId(rs.getLong(1)); - lci.setDbname(rs.getString(2)); - lci.setTablename(rs.getString(3)); - String partition = rs.getString(4); - if (!rs.wasNull()) { - lci.setPartitionname(partition); - } - lci.setType(TxnUtils.dbCompactionType2ThriftType(rs.getString(5).charAt(0))); - // Only put the latest record of each partition into response - if (!partitionSet.contains(partition)) { - response.addToCompactions(lci); - partitionSet.add(partition); - } - } - } catch (SQLException e) { - LOG.error("Unable to execute query", e); - checkRetryable(e, "getLatestCommittedCompactionInfo"); - } finally { - close(rs, pst, dbConn); - } - return response; - } catch (RetryException e) { - return getLatestCommittedCompactionInfo(rqst); - } - } - - public MetricsInfo getMetricsInfo() throws MetaException { - Connection dbConn = null; - try { - MetricsInfo metrics = new MetricsInfo(); - String s = MessageFormat.format(SELECT_METRICS_INFO_QUERY, getEpochFn(dbProduct)); - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - try (Statement stmt = dbConn.createStatement()){ - ResultSet rs = stmt.executeQuery(s); - if (rs.next()) { - metrics.setTxnToWriteIdCount(rs.getInt(1)); - metrics.setCompletedTxnsCount(rs.getInt(2)); - metrics.setOpenReplTxnsCount(rs.getInt(3)); - metrics.setOldestOpenReplTxnId(rs.getInt(4)); - metrics.setOldestOpenReplTxnAge(rs.getInt(5)); - metrics.setOpenNonReplTxnsCount(rs.getInt(6)); - metrics.setOldestOpenNonReplTxnId(rs.getInt(7)); - metrics.setOldestOpenNonReplTxnAge(rs.getInt(8)); - metrics.setAbortedTxnsCount(rs.getInt(9)); - metrics.setOldestAbortedTxnId(rs.getInt(10)); - metrics.setOldestAbortedTxnAge(rs.getInt(11)); - metrics.setLocksCount(rs.getInt(12)); - metrics.setOldestLockAge(rs.getInt(13)); - metrics.setOldestReadyForCleaningAge(rs.getInt(14)); - } - } - try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_TABLES_WITH_X_ABORTED_TXNS)) { - Set resourceNames = new TreeSet<>(); - pstmt.setInt(1, MetastoreConf.getIntVar(conf, ConfVars.METASTORE_ACIDMETRICS_TABLES_WITH_ABORTED_TXNS_THRESHOLD)); - ResultSet rs = pstmt.executeQuery(); - while (rs.next()) { - String resourceName = rs.getString(1) + "." + rs.getString(2); - String partName = rs.getString(3); - resourceName = partName != null ? resourceName + "#" + partName : resourceName; - resourceNames.add(resourceName); - } - metrics.setTablesWithXAbortedTxnsCount(resourceNames.size()); - metrics.setTablesWithXAbortedTxns(resourceNames); - } - return metrics; - } catch (SQLException e) { - LOG.error("Unable to getMetricsInfo", e); - checkRetryable(e, "getMetricsInfo"); - throw new MetaException("Unable to execute getMetricsInfo() " + StringUtils.stringifyException(e)); - } finally { - closeDbConn(dbConn); - } - } catch (RetryException e) { - return getMetricsInfo(); - } - } - - - private static void shouldNeverHappen(long txnid) { - throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid)); - } - private static void shouldNeverHappen(long txnid, long extLockId, long intLockId) { - throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid) + " " - + JavaUtils.lockIdToString(extLockId) + " " + intLockId); - } - - /** - * Retry-by-caller note: - * This may be retried after dbConn.commit. At worst, it will create duplicate entries in - * TXN_COMPONENTS which won't affect anything. See more comments in {@link #commitTxn(CommitTxnRequest)} - */ - @Override - @RetrySemantics.SafeToRetry - public void addDynamicPartitions(AddDynamicPartitions rqst) - throws NoSuchTxnException, TxnAbortedException, MetaException { - Connection dbConn = null; - Statement stmt = null; - try { - try { - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - TxnType txnType = getOpenTxnTypeAndLock(stmt, rqst.getTxnid()); - if (txnType == null) { - //ensures txn is still there and in expected state - ensureValidTxn(dbConn, rqst.getTxnid(), stmt); - shouldNeverHappen(rqst.getTxnid()); - } - //for RU this may be null so we should default it to 'u' which is most restrictive - OperationType ot = OperationType.UPDATE; - if(rqst.isSetOperationType()) { - ot = OperationType.fromDataOperationType(rqst.getOperationType()); - } - - Long writeId = rqst.getWriteid(); - try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_INSERT_QUERY)) { - int insertCounter = 0; - for (String partName : rqst.getPartitionnames()) { - pstmt.setLong(1, rqst.getTxnid()); - pstmt.setString(2, normalizeCase(rqst.getDbname())); - pstmt.setString(3, normalizeCase(rqst.getTablename())); - pstmt.setString(4, partName); - pstmt.setString(5, ot.getSqlConst()); - pstmt.setObject(6, writeId); - - pstmt.addBatch(); - insertCounter++; - if (insertCounter % maxBatchSize == 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY, maxBatchSize); - pstmt.executeBatch(); - } - } - if (insertCounter % maxBatchSize != 0) { - LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY, - insertCounter % maxBatchSize); - pstmt.executeBatch(); - } - } - try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_DP_DELETE_QUERY)) { - pstmt.setLong(1, rqst.getTxnid()); - pstmt.setString(2, normalizeCase(rqst.getDbname())); - pstmt.setString(3, normalizeCase(rqst.getTablename())); - pstmt.execute(); - } - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "addDynamicPartitions(" + rqst + ")"); - throw new MetaException("Unable to insert into from transaction database " + - StringUtils.stringifyException(e)); - } finally { - close(null, stmt, dbConn); - unlockInternal(); - } - } catch (RetryException e) { - addDynamicPartitions(rqst); - } - } - - /** - * Clean up corresponding records in metastore tables when corresponding object is dropped, - * specifically: TXN_COMPONENTS, COMPLETED_TXN_COMPONENTS, COMPACTION_QUEUE, COMPLETED_COMPACTIONS - * Retry-by-caller note: this is only idempotent assuming it's only called by dropTable/Db/etc - * operations. - * - * HIVE_LOCKS and WS_SET are cleaned up by {@link AcidHouseKeeperService}, if turned on - */ - @Override - @RetrySemantics.Idempotent - public void cleanupRecords(HiveObjectType type, Database db, Table table, - Iterator partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException { - cleanupRecords(type, db, table, partitionIterator, keepTxnToWriteIdMetaData, 0); - } - - @Override - @RetrySemantics.Idempotent - public void cleanupRecords(HiveObjectType type, Database db, Table table, - Iterator partitionIterator, long txnId) throws MetaException { - cleanupRecords(type, db , table, partitionIterator, false, txnId); - } - - private void cleanupRecords(HiveObjectType type, Database db, Table table, - Iterator partitionIterator, boolean keepTxnToWriteIdMetaData, long txnId) throws MetaException { - - // cleanup should be done only for objects belonging to default catalog - final String defaultCatalog = getDefaultCatalog(conf); - - try { - Connection dbConn = null; - Statement stmt = null; - - try { - String dbName; - String tblName; - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - List queries = new ArrayList<>(); - StringBuilder buff = new StringBuilder(); - - switch (type) { - case DATABASE: { - dbName = db.getName(); - if(!defaultCatalog.equals(db.getCatalogName())) { - LOG.debug("Skipping cleanup because db: " + dbName + " belongs to catalog " - + "other than default catalog: " + db.getCatalogName()); - return; - } - - buff.append("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_DATABASE\"='"); - buff.append(dbName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\"='"); - buff.append(dbName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CQ_TXN_ID\"!=").append(txnId); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_DATABASE\"='"); - buff.append(dbName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\"='"); - buff.append(dbName.toLowerCase()); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\"='"); - buff.append(dbName.toLowerCase()); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE \"CMC_DATABASE\"='"); - buff.append(dbName); - buff.append("'"); - queries.add(buff.toString()); - - break; - } - case TABLE: { - dbName = table.getDbName(); - tblName = table.getTableName(); - if(!defaultCatalog.equals(table.getCatName())) { - LOG.debug("Skipping cleanup because table: {} belongs to catalog other than default catalog: {}", tblName, - table.getCatName()); - return; - } - - buff.append("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"TC_TABLE\"='"); - buff.append(tblName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CTC_TABLE\"='"); - buff.append(tblName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CQ_TABLE\"='"); - buff.append(tblName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CC_TABLE\"='"); - buff.append(tblName); - buff.append("'"); - queries.add(buff.toString()); - - if (!keepTxnToWriteIdMetaData) { - buff.setLength(0); - buff.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\"='"); - buff.append(dbName.toLowerCase()); - buff.append("' AND \"T2W_TABLE\"='"); - buff.append(tblName.toLowerCase()); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\"='"); - buff.append(dbName.toLowerCase()); - buff.append("' AND \"NWI_TABLE\"='"); - buff.append(tblName.toLowerCase()); - buff.append("'"); - queries.add(buff.toString()); - } - buff.setLength(0); - buff.append("DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE \"CMC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CMC_TABLE\"='"); - buff.append(tblName); - buff.append("'"); - queries.add(buff.toString()); - - break; - } - case PARTITION: { - dbName = table.getDbName(); - tblName = table.getTableName(); - if(!defaultCatalog.equals(table.getCatName())) { - LOG.debug("Skipping cleanup because partitions belong to catalog other than default catalog: {}", - table.getCatName()); - return; - } - - List partCols = table.getPartitionKeys(); // partition columns - List partVals; // partition values - String partName; - - while (partitionIterator.hasNext()) { - Partition p = partitionIterator.next(); - partVals = p.getValues(); - partName = Warehouse.makePartName(partCols, partVals); - - buff.setLength(0); - buff.append("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"TC_TABLE\"='"); - buff.append(tblName); - buff.append("' AND \"TC_PARTITION\"='"); - buff.append(partName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CTC_TABLE\"='"); - buff.append(tblName); - buff.append("' AND \"CTC_PARTITION\"='"); - buff.append(partName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CQ_TABLE\"='"); - buff.append(tblName); - buff.append("' AND \"CQ_PARTITION\"='"); - buff.append(partName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CC_TABLE\"='"); - buff.append(tblName); - buff.append("' AND \"CC_PARTITION\"='"); - buff.append(partName); - buff.append("'"); - queries.add(buff.toString()); - - buff.setLength(0); - buff.append("DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE \"CMC_DATABASE\"='"); - buff.append(dbName); - buff.append("' AND \"CMC_TABLE\"='"); - buff.append(tblName); - buff.append("' AND \"CMC_PARTITION\"='"); - buff.append(partName); - buff.append("'"); - queries.add(buff.toString()); - } - - break; - } - default: { - throw new MetaException("Invalid object type for cleanup: " + type); - } - } - - for (String query : queries) { - LOG.debug("Going to execute update <{}>", query); - stmt.executeUpdate(query); - } - - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.debug("Going to rollback: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "cleanupRecords"); - if (e.getMessage().contains("does not exist")) { - LOG.warn("Cannot perform cleanup since metastore table does not exist"); - } else { - throw new MetaException("Unable to clean up " + StringUtils.stringifyException(e)); - } - } finally { - closeStmt(stmt); - closeDbConn(dbConn); - } - } catch (RetryException e) { - cleanupRecords(type, db, table, partitionIterator, keepTxnToWriteIdMetaData); - } - } - /** - * Catalog hasn't been added to transactional tables yet, so it's passed in but not used. - */ - @Override - public void onRename(String oldCatName, String oldDbName, String oldTabName, String oldPartName, - String newCatName, String newDbName, String newTabName, String newPartName) - throws MetaException { - String callSig = "onRename(" + - oldCatName + "," + oldDbName + "," + oldTabName + "," + oldPartName + "," + - newCatName + "," + newDbName + "," + newTabName + "," + newPartName + ")"; - - if(newPartName != null) { - assert oldPartName != null && oldTabName != null && oldDbName != null && oldCatName != null : - callSig; - } - if(newTabName != null) { - assert oldTabName != null && oldDbName != null && oldCatName != null : callSig; - } - if(newDbName != null) { - assert oldDbName != null && oldCatName != null : callSig; - } - - try { - Connection dbConn = null; - Statement stmt = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - List queries = new ArrayList<>(); - - String update = "UPDATE \"TXN_COMPONENTS\" SET "; - String where = " WHERE "; - if(oldPartName != null) { - update += "\"TC_PARTITION\" = " + quoteString(newPartName) + ", "; - where += "\"TC_PARTITION\" = " + quoteString(oldPartName) + " AND "; - } - if(oldTabName != null) { - update += "\"TC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"TC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"TC_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"TC_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"COMPLETED_TXN_COMPONENTS\" SET "; - where = " WHERE "; - if(oldPartName != null) { - update += "\"CTC_PARTITION\" = " + quoteString(newPartName) + ", "; - where += "\"CTC_PARTITION\" = " + quoteString(oldPartName) + " AND "; - } - if(oldTabName != null) { - update += "\"CTC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"CTC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"CTC_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"CTC_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"HIVE_LOCKS\" SET "; - where = " WHERE "; - if(oldPartName != null) { - update += "\"HL_PARTITION\" = " + quoteString(newPartName) + ", "; - where += "\"HL_PARTITION\" = " + quoteString(oldPartName) + " AND "; - } - if(oldTabName != null) { - update += "\"HL_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"HL_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"HL_DB\" = " + quoteString(normalizeCase(newDbName)); - where += "\"HL_DB\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"COMPACTION_QUEUE\" SET "; - where = " WHERE "; - if(oldPartName != null) { - update += "\"CQ_PARTITION\" = " + quoteString(newPartName) + ", "; - where += "\"CQ_PARTITION\" = " + quoteString(oldPartName) + " AND "; - } - if(oldTabName != null) { - update += "\"CQ_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"CQ_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"CQ_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"CQ_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"COMPLETED_COMPACTIONS\" SET "; - where = " WHERE "; - if(oldPartName != null) { - update += "\"CC_PARTITION\" = " + quoteString(newPartName) + ", "; - where += "\"CC_PARTITION\" = " + quoteString(oldPartName) + " AND "; - } - if(oldTabName != null) { - update += "\"CC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"CC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"CC_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"CC_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"WRITE_SET\" SET "; - where = " WHERE "; - if(oldPartName != null) { - update += "\"WS_PARTITION\" = " + quoteString(newPartName) + ", "; - where += "\"WS_PARTITION\" = " + quoteString(oldPartName) + " AND "; - } - if(oldTabName != null) { - update += "\"WS_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"WS_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"WS_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"WS_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"TXN_TO_WRITE_ID\" SET "; - where = " WHERE "; - if(oldTabName != null) { - update += "\"T2W_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"T2W_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"T2W_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"T2W_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"NEXT_WRITE_ID\" SET "; - where = " WHERE "; - if(oldTabName != null) { - update += "\"NWI_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"NWI_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if(oldDbName != null) { - update += "\"NWI_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"NWI_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - update = "UPDATE \"COMPACTION_METRICS_CACHE\" SET"; - where = " WHERE "; - if (oldPartName != null) { - update += "\"CMC_PARTITION\" = " + quoteString(normalizeCase(newPartName)) + ", "; - where += "\"CMC_PARTITION\" = " + quoteString(normalizeCase(oldPartName)) + " AND "; - } - if (oldTabName != null) { - update += "\"CMC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", "; - where += "\"CMC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND "; - } - if (oldDbName != null) { - update += "\"CMC_DATABASE\" = " + quoteString(normalizeCase(newDbName)); - where += "\"CMC_DATABASE\" = " + quoteString(normalizeCase(oldDbName)); - } - queries.add(update + where); - - for (String query : queries) { - LOG.debug("Going to execute update <{}>", query); - stmt.executeUpdate(query); - } - - LOG.debug("Going to commit: {}", callSig); - dbConn.commit(); - } catch (SQLException e) { - LOG.debug("Going to rollback: {}", callSig); - rollbackDBConn(dbConn); - checkRetryable(e, callSig); - if (e.getMessage().contains("does not exist")) { - LOG.warn("Cannot perform {} since metastore table does not exist", callSig); - } else { - throw new MetaException("Unable to " + callSig + ":" + StringUtils.stringifyException(e)); - } - } finally { - closeStmt(stmt); - closeDbConn(dbConn); - } - } catch (RetryException e) { - onRename(oldCatName, oldDbName, oldTabName, oldPartName, - newCatName, newDbName, newTabName, newPartName); - } - } - /** - * For testing only, do not use. - */ - @VisibleForTesting - public int numLocksInLockTable() throws SQLException, MetaException { - Connection dbConn = null; - Statement stmt = null; - ResultSet rs = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - String s = "SELECT COUNT(*) FROM \"HIVE_LOCKS\""; - LOG.debug("Going to execute query <{}>", s); - rs = stmt.executeQuery(s); - rs.next(); - int rc = rs.getInt(1); - // Necessary to clean up the transaction in the db. - dbConn.rollback(); - return rc; - } finally { - close(rs, stmt, dbConn); - } - } - - /** - * For testing only, do not use. - */ - public long setTimeout(long milliseconds) { - long previous_timeout = timeout; - timeout = milliseconds; - return previous_timeout; - } - - protected class RetryException extends Exception { - - } - - Connection getDbConn(int isolationLevel) throws SQLException { - return getDbConn(isolationLevel, connPool); - } - - protected Connection getDbConn(int isolationLevel, DataSource connPool) throws SQLException { - Connection dbConn = null; - try { - dbConn = connPool.getConnection(); - dbConn.setAutoCommit(false); - dbConn.setTransactionIsolation(isolationLevel); - return dbConn; - } catch (SQLException e) { - closeDbConn(dbConn); - throw e; - } - } - - static void rollbackDBConn(Connection dbConn) { - try { - if (dbConn != null && !dbConn.isClosed()) dbConn.rollback(); - } catch (SQLException e) { - LOG.warn("Failed to rollback db connection " + getMessage(e)); - } - } - protected static void closeDbConn(Connection dbConn) { - try { - if (dbConn != null && !dbConn.isClosed()) { - dbConn.close(); - } - } catch (SQLException e) { - LOG.warn("Failed to close db connection " + getMessage(e)); - } - } - - /** - * Close statement instance. - * @param stmt statement instance. - */ - protected static void closeStmt(Statement stmt) { - try { - if (stmt != null && !stmt.isClosed()) stmt.close(); - } catch (SQLException e) { - LOG.warn("Failed to close statement " + getMessage(e)); - } - } - - /** - * Close the ResultSet. - * @param rs may be {@code null} - */ - static void close(ResultSet rs) { - try { - if (rs != null && !rs.isClosed()) { - rs.close(); - } - } - catch(SQLException ex) { - LOG.warn("Failed to close statement " + getMessage(ex)); - } - } - - /** - * Close all 3 JDBC artifacts in order: {@code rs stmt dbConn} - */ - static void close(ResultSet rs, Statement stmt, Connection dbConn) { - close(rs); - closeStmt(stmt); - closeDbConn(dbConn); - } - - private boolean waitForRetry(String caller, String errMsg) { - if (retryNum++ < retryLimit) { - LOG.warn("Retryable error detected in {}. Will wait {} ms and retry up to {} times. Error: {}", caller, - retryInterval, (retryLimit - retryNum + 1), errMsg); - try { - Thread.sleep(retryInterval); - } catch (InterruptedException ex) { - // - } - return true; - } else { - LOG.error("Fatal error in {}. Retry limit ({}) reached. Last error: {}", caller, retryLimit, errMsg); - } - return false; - } - - /** - * See {@link #checkRetryable(SQLException, String, boolean)}. - */ - void checkRetryable(SQLException e, String caller) throws RetryException { - checkRetryable(e, caller, false); - } - - /** - * Determine if an exception was such that it makes sense to retry. Unfortunately there is no standard way to do - * this, so we have to inspect the error messages and catch the telltale signs for each - * different database. This method will throw {@code RetryException} - * if the error is retry-able. - * @param e exception that was thrown. - * @param caller name of the method calling this (and other info useful to log) - * @param retryOnDuplicateKey whether to retry on unique key constraint violation - * @throws org.apache.hadoop.hive.metastore.txn.TxnHandler.RetryException when the operation should be retried - */ - void checkRetryable(SQLException e, String caller, boolean retryOnDuplicateKey) - throws RetryException { - - // If you change this function, remove the @Ignore from TestTxnHandler.deadlockIsDetected() - // to test these changes. - // MySQL and MSSQL use 40001 as the state code for rollback. Postgres uses 40001 and 40P01. - // Oracle seems to return different SQLStates and messages each time, - // so I've tried to capture the different error messages (there appear to be fewer different - // error messages than SQL states). - // Derby and newer MySQL driver use the new SQLTransactionRollbackException - boolean sendRetrySignal = false; - try { - if(dbProduct == null) { - throw new IllegalStateException("DB Type not determined yet."); - } - if (dbProduct.isDeadlock(e)) { - if (deadlockCnt++ < ALLOWED_REPEATED_DEADLOCKS) { - long waitInterval = deadlockRetryInterval * deadlockCnt; - LOG.warn("Deadlock detected in {}. Will wait {} ms try again up to {} times.", caller, waitInterval, - (ALLOWED_REPEATED_DEADLOCKS - deadlockCnt + 1)); - // Pause for a just a bit for retrying to avoid immediately jumping back into the deadlock. - try { - Thread.sleep(waitInterval); - } catch (InterruptedException ie) { - // NOP - } - sendRetrySignal = true; - } else { - LOG.error("Too many repeated deadlocks in {}, giving up.", caller); - } - } else if (isRetryable(conf, e)) { - //in MSSQL this means Communication Link Failure - sendRetrySignal = waitForRetry(caller, e.getMessage()); - } else if (retryOnDuplicateKey && isDuplicateKeyError(e)) { - sendRetrySignal = waitForRetry(caller, e.getMessage()); - } - else { - //make sure we know we saw an error that we don't recognize - LOG.info("Non-retryable error in {} : {}", caller, getMessage(e)); - } - } - finally { - /*if this method ends with anything except a retry signal, the caller should fail the operation - and propagate the error up to the its caller (Metastore client); thus must reset retry counters*/ - if(!sendRetrySignal) { - deadlockCnt = 0; - retryNum = 0; - } - } - if(sendRetrySignal) { - throw new RetryException(); - } - } - - /** - * Determine the current time, using the RDBMS as a source of truth - * @param conn database connection - * @return current time in milliseconds - * @throws org.apache.hadoop.hive.metastore.api.MetaException if the time cannot be determined - */ - protected long getDbTime(Connection conn) throws MetaException { - Statement stmt = null; - try { - stmt = conn.createStatement(); - String s = dbProduct.getDBTime(); - - LOG.debug("Going to execute query <{}>", s); - ResultSet rs = stmt.executeQuery(s); - if (!rs.next()) throw new MetaException("No results from date query"); - return rs.getTimestamp(1).getTime(); - } catch (SQLException e) { - String msg = "Unable to determine current time: " + e.getMessage(); - LOG.error(msg); - throw new MetaException(msg); - } finally { - closeStmt(stmt); - } - } - - /** - * Determine the current time, using the RDBMS as a source of truth - * @return current time in milliseconds - * @throws org.apache.hadoop.hive.metastore.api.MetaException if the time cannot be determined - */ - protected Timestamp getDbTime() throws MetaException { - return jdbcResource.getJdbcTemplate().queryForObject( - dbProduct.getDBTime(), - new MapSqlParameterSource(), - (ResultSet rs, int rowNum) -> rs.getTimestamp(1)); - } - - - protected String isWithinCheckInterval(String expr, long interval) throws MetaException { - return dbProduct.isWithinCheckInterval(expr, interval); - } - - private void determineDatabaseProduct(Connection conn) { - try { - String s = conn.getMetaData().getDatabaseProductName(); - dbProduct = DatabaseProduct.determineDatabaseProduct(s, conf); - if (dbProduct.isUNDEFINED()) { - String msg = "Unrecognized database product name <" + s + ">"; - LOG.error(msg); - throw new IllegalStateException(msg); - } - } catch (SQLException e) { - String msg = "Unable to get database product name"; - LOG.error(msg, e); - throw new IllegalStateException(msg, e); - } - } - - private static class LockInfo { - private final long extLockId; - private final long intLockId; - //0 means there is no transaction, i.e. it a select statement which is not part of - //explicit transaction or a IUD statement that is not writing to ACID table - private final long txnId; - private final String db; - private final String table; - private final String partition; - private final LockState state; - private final LockType type; - - // Assumes the result set is set to a valid row - LockInfo(ResultSet rs) throws SQLException, MetaException { - extLockId = rs.getLong("HL_LOCK_EXT_ID"); // can't be null - intLockId = rs.getLong("HL_LOCK_INT_ID"); // can't be null - db = rs.getString("HL_DB"); // can't be null - String t = rs.getString("HL_TABLE"); - table = (rs.wasNull() ? null : t); - String p = rs.getString("HL_PARTITION"); - partition = (rs.wasNull() ? null : p); - switch (rs.getString("HL_LOCK_STATE").charAt(0)) { - case LOCK_WAITING: state = LockState.WAITING; break; - case LOCK_ACQUIRED: state = LockState.ACQUIRED; break; - default: - throw new MetaException("Unknown lock state " + rs.getString("HL_LOCK_STATE").charAt(0)); - } - char lockChar = rs.getString("HL_LOCK_TYPE").charAt(0); - type = LockTypeUtil.getLockTypeFromEncoding(lockChar) - .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar)); - txnId = rs.getLong("HL_TXNID"); //returns 0 if value is NULL - } - - LockInfo(ShowLocksResponseElement e) { - extLockId = e.getLockid(); - intLockId = e.getLockIdInternal(); - txnId = e.getTxnid(); - db = e.getDbname(); - table = e.getTablename(); - partition = e.getPartname(); - state = e.getState(); - type = e.getType(); - } - - public boolean equals(Object other) { - if (!(other instanceof LockInfo)) return false; - LockInfo o = (LockInfo)other; - // Lock ids are unique across the system. - return extLockId == o.extLockId && intLockId == o.intLockId; - } + @Override + public void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException { + new ReplTableWriteIdStateFunction(rqst, mutexAPI, transactionalListeners).execute(jdbcResource); + } - @Override - public String toString() { - return JavaUtils.lockIdToString(extLockId) + " intLockId:" + - intLockId + " " + JavaUtils.txnIdToString(txnId) - + " db:" + db + " table:" + table + " partition:" + - partition + " state:" + (state == null ? "null" : state.toString()) - + " type:" + (type == null ? "null" : type.toString()); - } - private boolean isDbLock() { - return db != null && table == null && partition == null; - } - private boolean isTableLock() { - return db != null && table != null && partition == null; - } - private boolean isPartitionLock() { - return !(isDbLock() || isTableLock()); - } + @Override + public GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst) throws MetaException { + return new GetValidWriteIdsFunction(rqst, openTxnTimeOutMillis).execute(jdbcResource); + } + + @Override + public AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst) throws MetaException { + return new AllocateTableWriteIdsFunction(rqst, transactionalListeners).execute(jdbcResource); } - private static class LockInfoComparator implements Comparator, Serializable { - private LockTypeComparator lockTypeComparator = new LockTypeComparator(); + @Override + public MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWriteIdRequest rqst) throws MetaException { + return jdbcResource.execute(new GetMaxAllocatedTableWriteIdHandler(rqst)); + } - public boolean equals(Object other) { - return this == other; - } + @Override + public void seedWriteId(SeedTableWriteIdsRequest rqst) throws MetaException { + //since this is on conversion from non-acid to acid, NEXT_WRITE_ID should not have an entry + //for this table. It also has a unique index in case 'should not' is violated - public int compare(LockInfo info1, LockInfo info2) { - // We sort by state (acquired vs waiting) and then by LockType, then by id - if (info1.state == LockState.ACQUIRED && - info2.state != LockState .ACQUIRED) { - return -1; - } - if (info1.state != LockState.ACQUIRED && - info2.state == LockState .ACQUIRED) { - return 1; - } + // First allocation of write id should add the table to the next_write_id meta table + // The initial value for write id should be 1 and hence we add 1 with number of write ids + // allocated here + jdbcResource.getJdbcTemplate().update( + "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (:db, :table, :writeId)", + new MapSqlParameterSource() + .addValue("db", rqst.getDbName()) + .addValue("table", rqst.getTableName()) + .addValue("writeId", rqst.getSeedWriteId() + 1)); + } - int sortByType = lockTypeComparator.compare(info1.type, info2.type); - if(sortByType != 0) { - return sortByType; - } - if (info1.extLockId < info2.extLockId) { - return -1; - } else if (info1.extLockId > info2.extLockId) { - return 1; - } else { - if (info1.intLockId < info2.intLockId) { - return -1; - } else if (info1.intLockId > info2.intLockId) { - return 1; - } else { - return 0; - } - } + @Override + public void seedTxnId(SeedTxnIdRequest rqst) throws MetaException { + /* + * Locking the txnLock an exclusive way, we do not want to set the txnId backward accidentally + * if there are concurrent open transactions + */ + acquireTxnLock(false); + long highWaterMark = jdbcResource.execute(new GetHighWaterMarkHandler()); + if (highWaterMark >= rqst.getSeedTxnId()) { + throw new MetaException(MessageFormat + .format("Invalid txnId seed {}, the highWaterMark is {}", rqst.getSeedTxnId(), highWaterMark)); } + jdbcResource.getJdbcTemplate().getJdbcTemplate() + .execute((Statement stmt) -> stmt.execute(dbProduct.getTxnSeedFn(rqst.getSeedTxnId()))); } - private enum LockAction {ACQUIRE, WAIT, KEEP_LOOKING} + @Override + public void addWriteNotificationLog(ListenerEvent acidWriteEvent) throws MetaException { + MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, + acidWriteEvent instanceof AcidWriteEvent ? EventMessage.EventType.ACID_WRITE + : EventMessage.EventType.BATCH_ACID_WRITE, + acidWriteEvent, jdbcResource.getConnection(), sqlGenerator); + } - // A jump table to figure out whether to wait, acquire, - // or keep looking . Since - // java doesn't have function pointers (grumble grumble) we store a - // character that we'll use to determine which function to call. - // The table maps the lock type of the lock we are looking to acquire to - // the lock type of the lock we are checking to the lock state of the lock - // we are checking to the desired action. - private static Map>> jumpTable; + @Override + public void performWriteSetGC() throws MetaException { + long commitHighWaterMark = new MinOpenTxnIdWaterMarkFunction(openTxnTimeOutMillis).execute(jdbcResource); + jdbcResource.getJdbcTemplate().update( + "DELETE FROM \"WRITE_SET\" WHERE \"WS_COMMIT_ID\" < :hwm", + new MapSqlParameterSource() + .addValue("hwm", commitHighWaterMark)); + } - private int abortTxns(Connection dbConn, List txnids, - boolean skipCount, boolean isReplReplayed, TxnErrorMsg txnErrorMsg) throws SQLException, MetaException { - return abortTxns(dbConn, txnids, false, skipCount, isReplReplayed, txnErrorMsg); + @Override + public void updateTransactionStatistics(UpdateTransactionalStatsRequest req) throws MetaException { + jdbcResource.execute( + "UPDATE \"MV_TABLES_USED\" " + + "SET \"INSERTED_COUNT\"=\"INSERTED_COUNT\"+ :insertCount" + + ",\"UPDATED_COUNT\"=\"UPDATED_COUNT\"+ :updateCount" + + ",\"DELETED_COUNT\"=\"DELETED_COUNT\"+ :deleteCount" + + " WHERE \"TBL_ID\"= :tableId", + new MapSqlParameterSource() + .addValue("insertCount", req.getInsertCount()) + .addValue("updateCount", req.getUpdatedCount()) + .addValue("deleteCount", req.getDeletedCount()) + .addValue("tableId", req.getTableId()), null); } + /** - * TODO: expose this as an operation to client. Useful for streaming API to abort all remaining - * transactions in a batch on IOExceptions. - * Caller must rollback the transaction if not all transactions were aborted since this will not - * attempt to delete associated locks in this case. - * - * @param dbConn An active connection - * @param txnids list of transactions to abort - * @param checkHeartbeat value used by {@link #performTimeOuts()} to ensure this doesn't Abort txn which were - * heartbeated after #performTimeOuts() select and this operation. - * @param skipCount If true, the method always returns 0, otherwise returns the number of actually aborted txns - * @return 0 if skipCount is true, the number of aborted transactions otherwise - * @throws SQLException + * Get invalidation info for the materialization. Materialization information + * contains information about whether there was update/delete/compaction operations on the source + * tables used by the materialization since it was created. */ - private int abortTxns(Connection dbConn, List txnids, boolean checkHeartbeat, - boolean skipCount, boolean isReplReplayed, TxnErrorMsg txnErrorMsg) - throws SQLException, MetaException { - Statement stmt = null; - if (txnids.isEmpty()) { - return 0; - } - Collections.sort(txnids); - LOG.debug("Aborting {} transaction(s) {} due to {}", txnids.size(), txnids, txnErrorMsg); - removeTxnsFromMinHistoryLevel(dbConn, txnids); - removeWriteIdsFromMinHistory(dbConn, txnids); - try { - stmt = dbConn.createStatement(); - //This is an update statement, thus at any Isolation level will take Write locks so will block - //all other ops using S4U on TXNS row. - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - - // add update txns queries to query list - prefix.append("UPDATE \"TXNS\" SET \"TXN_STATE\" = ").append(TxnStatus.ABORTED) - .append(" , \"TXN_META_INFO\" = ").append(txnErrorMsg.toSqlString()) - .append(" WHERE \"TXN_STATE\" = ").append(TxnStatus.OPEN).append(" AND "); - if (checkHeartbeat) { - suffix.append(" AND \"TXN_LAST_HEARTBEAT\" < ") - .append(getEpochFn(dbProduct)).append("-").append(timeout); - } - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"TXN_ID\"", true, false); - int numUpdateQueries = queries.size(); - - // add delete hive locks queries to query list - prefix.setLength(0); - suffix.setLength(0); - prefix.append("DELETE FROM \"HIVE_LOCKS\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"HL_TXNID\"", false, false); - - //If this abort is for REPL_CREATED TXN initiated outside the replication flow, then clean the corresponding entry - //from REPL_TXN_MAP and mark that database as replication incompatible. - if (!isReplReplayed) { - for (String database : getDbNamesForReplayedTxns(dbConn, txnids)) { - markDbAsReplIncompatible(dbConn, database); - } - // Delete mapping from REPL_TXN_MAP if it exists. - prefix.setLength(0); - suffix.setLength(0); - prefix.append("DELETE FROM \"REPL_TXN_MAP\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"RTM_TARGET_TXN_ID\"", false, false); - } - - // execute all queries in the list in one batch - int numAborted = 0; - if (skipCount) { - executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize); - } else { - List affectedRowsByQuery = executeQueriesInBatch(stmt, queries, maxBatchSize); - numAborted = getUpdateCount(numUpdateQueries, affectedRowsByQuery); - } + @Override + public Materialization getMaterializationInvalidationInfo( + CreationMetadata creationMetadata, String validTxnListStr) throws MetaException { + return new GetMaterializationInvalidationInfoFunction(creationMetadata, validTxnListStr).execute(jdbcResource); + } - if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) { - Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_ABORTED_TXNS).inc(txnids.size()); - } - LOG.warn("Aborted {} transaction(s) {} due to {}", txnids.size(), txnids, txnErrorMsg); - return numAborted; - } finally { - closeStmt(stmt); - } + @Override + public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) throws MetaException { + return new LockMaterializationRebuildFunction(dbName, tableName, txnId, mutexAPI).execute(jdbcResource); } - private int getUpdateCount(int numUpdateQueries, List affectedRowsByQuery) { - return affectedRowsByQuery.stream() - .limit(numUpdateQueries) - .mapToInt(Integer::intValue) - .sum(); + @Override + public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws MetaException { + int result = jdbcResource.execute( + "UPDATE \"MATERIALIZATION_REBUILD_LOCKS\"" + + " SET \"MRL_LAST_HEARTBEAT\" = " + Instant.now().toEpochMilli() + + " WHERE \"MRL_TXN_ID\" = " + txnId + + " AND \"MRL_DB_NAME\" = ?" + + " AND \"MRL_TBL_NAME\" = ?", + new MapSqlParameterSource() + .addValue("now", Instant.now().toEpochMilli()) + .addValue("txnId", txnId) + .addValue("dbName", dbName) + .addValue("tableNane", tableName), + ParameterizedCommand.AT_LEAST_ONE_ROW); + return result >= 1; } - private static boolean isValidTxn(long txnId) { - return txnId != 0; + @Override + public long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) throws MetaException { + return new ReleaseMaterializationRebuildLocks(validTxnList, timeout).execute(jdbcResource); } + /** - * Lock acquisition is meant to be fair, so every lock can only block on some lock with smaller - * hl_lock_ext_id by only checking earlier locks. - * - * For any given SQL statement all locks required by it are grouped under single extLockId and are - * granted all at once or all locks wait. - * - * This is expected to run at READ_COMMITTED. - * - * If there is a concurrent commitTxn/rollbackTxn, those can only remove rows from HIVE_LOCKS. - * If they happen to be for the same txnid, there will be a WW conflict (in MS DB), if different txnid, - * checkLock() will in the worst case keep locks in Waiting state a little longer. + * As much as possible (i.e. in absence of retries) we want both operations to be done on the same + * connection (but separate transactions). + * Retry-by-caller note: If the call to lock is from a transaction, then in the worst case + * there will be a duplicate set of locks but both sets will belong to the same txn so they + * will not conflict with each other. For locks w/o txn context (i.e. read-only query), this + * may lead to deadlock (at least a long wait). (e.g. 1st call creates locks in {@code LOCK_WAITING} + * mode and response gets lost. Then {@link org.apache.hadoop.hive.metastore.RetryingMetaStoreClient} + * retries, and enqueues another set of locks in LOCK_WAITING. The 2nd LockResponse is delivered + * to the DbLockManager, which will keep dong {@link #checkLock(CheckLockRequest)} until the 1st + * set of locks times out. */ - @RetrySemantics.SafeToRetry("See @SafeToRetry") - private LockResponse checkLock(Connection dbConn, long extLockId, long txnId, boolean zeroWaitReadEnabled, - boolean isExclusiveCTAS) - throws NoSuchLockException, TxnAbortedException, MetaException, SQLException { - Statement stmt = null; - ResultSet rs = null; - LockResponse response = new LockResponse(); - /** - * todo: Longer term we should pass this from client somehow - this would be an optimization; once - * that is in place make sure to build and test "writeSet" below using OperationType not LockType - * With Static Partitions we assume that the query modifies exactly the partitions it locked. (not entirely - * realistic since Update/Delete may have some predicate that filters out all records out of - * some partition(s), but plausible). For DP, we acquire locks very wide (all known partitions), - * but for most queries only a fraction will actually be updated. #addDynamicPartitions() tells - * us exactly which ones were written to. Thus using this trick to kill a query early for - * DP queries may be too restrictive. - */ - boolean isPartOfDynamicPartitionInsert = true; + @Override + public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException { + long lockId = txnLockManager.enqueueLock(rqst); try { - List locksBeingChecked = getLocksFromLockId(dbConn, extLockId); //being acquired now - response.setLockid(extLockId); - - //This is the set of entities that the statement represented by extLockId wants to update - List writeSet = new ArrayList<>(); - - for (LockInfo info : locksBeingChecked) { - if(!isPartOfDynamicPartitionInsert && info.type == LockType.SHARED_WRITE) { - writeSet.add(info); - } - } - if(!writeSet.isEmpty()) { - if(writeSet.get(0).txnId == 0) { - //Write operation always start a txn - throw new IllegalStateException("Found Write lock for " + JavaUtils.lockIdToString(extLockId) + " but no txnid"); - } - stmt = dbConn.createStatement(); - StringBuilder sb = new StringBuilder(" \"WS_DATABASE\", \"WS_TABLE\", \"WS_PARTITION\", " + - "\"WS_TXNID\", \"WS_COMMIT_ID\" " + - "FROM \"WRITE_SET\" WHERE WS_COMMIT_ID >= " + writeSet.get(0).txnId + " AND (");//see commitTxn() for more info on this inequality - for(LockInfo info : writeSet) { - sb.append("(\"WS_DATABASE\" = ").append(quoteString(info.db)).append(" AND \"WS_TABLE\" = ") - .append(quoteString(info.table)).append(" AND \"WS_PARTITION\" ") - .append(info.partition == null ? "IS NULL" : "= " + quoteString(info.partition)).append(") OR "); - } - sb.setLength(sb.length() - 4);//nuke trailing " or " - sb.append(")"); - //1 row is sufficient to know we have to kill the query - rs = stmt.executeQuery(sqlGenerator.addLimitClause(1, sb.toString())); - if(rs.next()) { - /** - * if here, it means we found an already committed txn which overlaps with the current one and - * it updated the same resource the current txn wants to update. By First-committer-wins - * rule, current txn will not be allowed to commit so may as well kill it now; This is just an - * optimization to prevent wasting cluster resources to run a query which is known to be DOA. - * {@link #commitTxn(CommitTxnRequest)} has the primary responsibility to ensure this. - * checkLock() runs at READ_COMMITTED so you could have another (Hive) txn running commitTxn() - * in parallel and thus writing to WRITE_SET. commitTxn() logic is properly mutexed to ensure - * that we don't "miss" any WW conflicts. We could've mutexed the checkLock() and commitTxn() - * as well but this reduces concurrency for very little gain. - * Note that update/delete (which runs as dynamic partition insert) acquires a lock on the table, - * but WRITE_SET has entries for actual partitions updated. Thus this optimization will "miss" - * the WW conflict but it will be caught in commitTxn() where actual partitions written are known. - * This is OK since we want 2 concurrent updates that update different sets of partitions to both commit. - */ - String resourceName = rs.getString(1) + '/' + rs.getString(2); - String partName = rs.getString(3); - if(partName != null) { - resourceName += '/' + partName; - } - - String msg = "Aborting " + JavaUtils.txnIdToString(writeSet.get(0).txnId) + - " since a concurrent committed transaction [" + JavaUtils.txnIdToString(rs.getLong(4)) + "," + rs.getLong(5) + - "] has already updated resource '" + resourceName + "'"; - LOG.info(msg); - if (abortTxns(dbConn, Collections.singletonList(writeSet.get(0).txnId), false, false, - TxnErrorMsg.ABORT_CONCURRENT) != 1) { - throw new IllegalStateException(msg + " FAILED!"); - } - dbConn.commit(); - throw new TxnAbortedException(msg); - } - close(rs, stmt, null); - } - - String queryStr = - " \"EX\".*, \"REQ\".\"HL_LOCK_INT_ID\" \"LOCK_INT_ID\", \"REQ\".\"HL_LOCK_TYPE\" \"LOCK_TYPE\" FROM (" + - " SELECT \"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\"," + - " \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\" FROM \"HIVE_LOCKS\"" + - " WHERE \"HL_LOCK_EXT_ID\" < " + extLockId + ") \"EX\"" + - " INNER JOIN (" + - " SELECT \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\"," + - " \"HL_LOCK_TYPE\" FROM \"HIVE_LOCKS\"" + - " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + ") \"REQ\"" + - " ON \"EX\".\"HL_DB\" = \"REQ\".\"HL_DB\"" + - " AND (\"EX\".\"HL_TABLE\" IS NULL OR \"REQ\".\"HL_TABLE\" IS NULL" + - " OR \"EX\".\"HL_TABLE\" = \"REQ\".\"HL_TABLE\"" + - " AND (\"EX\".\"HL_PARTITION\" IS NULL OR \"REQ\".\"HL_PARTITION\" IS NULL" + - " OR \"EX\".\"HL_PARTITION\" = \"REQ\".\"HL_PARTITION\"))" + - /*different locks from same txn should not conflict with each other, - txnId=0 means it's a select or IUD which does not write to ACID table*/ - " WHERE (\"REQ\".\"HL_TXNID\" = 0 OR \"EX\".\"HL_TXNID\" != \"REQ\".\"HL_TXNID\")" + - " AND "; - - /*EXCLUSIVE lock on partition should prevent SHARED_READ on the table, however there is no reason - for an EXCLUSIVE on a table to prevent SHARED_READ on a database. Similarly, EXCLUSIVE on a partition - should not conflict with SHARED_READ on a database. - SHARED_READ is usually acquired on a database to make sure it's not dropped, while some operation - is performed on that db (e.g. show tables, created table, etc). - EXCLUSIVE on an object may mean it's being dropped or overwritten.*/ - String[] whereStr = { - // shared-read - " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.sharedRead() + " AND \"EX\".\"HL_LOCK_TYPE\"=" + - LockTypeUtil.exclusive() + " AND NOT (\"EX\".\"HL_TABLE\" IS NOT NULL AND \"REQ\".\"HL_TABLE\" IS NULL)", - // exclusive - " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.exclusive() + - " AND NOT (\"EX\".\"HL_TABLE\" IS NULL AND \"EX\".\"HL_LOCK_TYPE\"=" + - LockTypeUtil.sharedRead() + " AND \"REQ\".\"HL_TABLE\" IS NOT NULL)", - // shared-write - " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.sharedWrite() + " AND \"EX\".\"HL_LOCK_TYPE\" IN (" + - LockTypeUtil.exclWrite() + "," + LockTypeUtil.exclusive() + ")", - // excl-write - " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.exclWrite() + " AND \"EX\".\"HL_LOCK_TYPE\"!=" + - LockTypeUtil.sharedRead() - }; - - List subQuery = new ArrayList<>(); - for (String subCond : whereStr) { - subQuery.add("(" + sqlGenerator.addLimitClause(1, queryStr + subCond) + ")"); - } - String query = String.join(" UNION ALL ", subQuery); - - stmt = dbConn.createStatement(); - LOG.debug("Going to execute query <{}>", query); - rs = stmt.executeQuery(query); - - if (rs.next()) { - // We acquire all locks for a given query atomically; if 1 blocks, all remain in Waiting state. - LockInfo blockedBy = new LockInfo(rs); - long intLockId = rs.getLong("LOCK_INT_ID"); - char lockChar = rs.getString("LOCK_TYPE").charAt(0); - if (LOG.isDebugEnabled()) { - LOG.debug("Failure to acquire lock({} intLockId:{} {}), blocked by ({})", JavaUtils.lockIdToString(extLockId), - intLockId, JavaUtils.txnIdToString(txnId), blockedBy); - } - - LockType lockType = LockTypeUtil.getLockTypeFromEncoding(lockChar) - .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar)); - - if ((zeroWaitReadEnabled && LockType.SHARED_READ == lockType || isExclusiveCTAS) && isValidTxn(txnId)) { - String cleanupQuery = "DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = " + extLockId; - LOG.debug("Going to execute query: <{}>", cleanupQuery); - stmt.executeUpdate(cleanupQuery); - dbConn.commit(); - - response.setErrorMessage(String.format( - isExclusiveCTAS ? EXCL_CTAS_ERR_MSG : ZERO_WAIT_READ_ERR_MSG, blockedBy)); - response.setState(LockState.NOT_ACQUIRED); - return response; - } - String updateBlockedByQuery = "UPDATE \"HIVE_LOCKS\"" + - " SET \"HL_BLOCKEDBY_EXT_ID\" = " + blockedBy.extLockId + - ", \"HL_BLOCKEDBY_INT_ID\" = " + blockedBy.intLockId + - " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + " AND \"HL_LOCK_INT_ID\" = " + intLockId; - - LOG.debug("Going to execute query: <{}>", updateBlockedByQuery); - int updCnt = stmt.executeUpdate(updateBlockedByQuery); - - if (updCnt != 1) { - LOG.error("Failure to update lock (extLockId={}, intLockId={}) with the blocking lock's IDs " + - "(extLockId={}, intLockId={})", extLockId, intLockId, blockedBy.extLockId, blockedBy.intLockId); - shouldNeverHappen(txnId, extLockId, intLockId); - } - dbConn.commit(); - - response.setState(LockState.WAITING); - return response; - } - // If here, there were no locks that would block any item from 'locksBeingChecked' - acquire them all - acquire(dbConn, stmt, locksBeingChecked); - - // We acquired all the locks, so commit and return acquired. - LOG.debug("Successfully acquired locks: {}", locksBeingChecked); - dbConn.commit(); - response.setState(LockState.ACQUIRED); - } finally { - close(rs, stmt, null); + return txnLockManager.checkLock(lockId, rqst.getTxnid(), rqst.isZeroWaitReadEnabled(), rqst.isExclusiveCTAS()); + } catch (NoSuchLockException e) { + // This should never happen, as we just added the lock id + throw new MetaException("Couldn't find a lock we just created! " + e.getMessage()); } - return response; } - - private void acquire(Connection dbConn, Statement stmt, List locksBeingChecked) - throws SQLException, NoSuchLockException, MetaException { - if (locksBeingChecked == null || locksBeingChecked.isEmpty()) { - return; - } - long txnId = locksBeingChecked.get(0).txnId; - long extLockId = locksBeingChecked.get(0).extLockId; - String s = "UPDATE \"HIVE_LOCKS\" SET \"HL_LOCK_STATE\" = '" + LOCK_ACQUIRED + "', " + - //if lock is part of txn, heartbeat info is in txn record - "\"HL_LAST_HEARTBEAT\" = " + (isValidTxn(txnId) ? 0 : getEpochFn(dbProduct)) + - ",\"HL_ACQUIRED_AT\" = " + getEpochFn(dbProduct) + - ",\"HL_BLOCKEDBY_EXT_ID\"=NULL,\"HL_BLOCKEDBY_INT_ID\"=NULL" + - " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId; - LOG.debug("Going to execute update <{}>", s); - int rc = stmt.executeUpdate(s); - if (rc < locksBeingChecked.size()) { - LOG.error("Failure to acquire all locks (acquired: {}, total needed: {}).", rc, locksBeingChecked.size()); - dbConn.rollback(); - /*select all locks for this ext ID and see which ones are missing*/ - String errorMsgTemplate = "No such lock(s): (%s: %s) %s"; - Set notFoundIds = locksBeingChecked.stream() - .map(lockInfo -> Long.toString(lockInfo.intLockId)) - .collect(Collectors.toSet()); - String getIntIdsQuery = "SELECT \"HL_LOCK_INT_ID\" FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = " + extLockId; - LOG.debug("Going to execute query: <{}>", getIntIdsQuery); - try (ResultSet rs = stmt.executeQuery(getIntIdsQuery)) { - while (rs.next()) { - notFoundIds.remove(rs.getString(1)); - } - } - String errorMsg = String.format(errorMsgTemplate, - JavaUtils.lockIdToString(extLockId), String.join(", ", notFoundIds), JavaUtils.txnIdToString(txnId)); - throw new NoSuchLockException(errorMsg); + + /** + * Why doesn't this get a txnid as parameter? The caller should either know the txnid or know there isn't one. + * Either way getTxnIdFromLockId() will not be needed. This would be a Thrift change. + *

+ * Also, when lock acquisition returns WAITING, it's retried every 15 seconds (best case, see DbLockManager.backoff(), + * in practice more often) + * which means this is heartbeating way more often than hive.txn.timeout and creating extra load on DB. + *

+ * The clients that operate in blocking mode, can't heartbeat a lock until the lock is acquired. + * We should make CheckLockRequest include timestamp or last request to skip unnecessary heartbeats. Thrift change. + *

+ * {@link #checkLock(CheckLockRequest)} must run at SERIALIZABLE + * (make sure some lock we are checking against doesn't move from W to A in another txn) + * but this method can heartbeat in separate txn at READ_COMMITTED. + *

+ * Retry-by-caller note: + * Retryable because {@link #checkLock(CheckLockRequest)} is + */ + @Override + public LockResponse checkLock(CheckLockRequest rqst) + throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException { + long extLockId = rqst.getLockid(); + // Heartbeat on the lockid first, to assure that our lock is still valid. + // Then look up the lock info (hopefully in the cache). If these locks + // are associated with a transaction then heartbeat on that as well. + List lockInfos = jdbcResource.execute(new GetLocksByLockId(extLockId, 1, sqlGenerator)); + if (CollectionUtils.isEmpty(lockInfos)) { + throw new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId)); + } + LockInfo lockInfo = lockInfos.get(0); + if (lockInfo.getTxnId() > 0) { + new HeartbeatTxnFunction(lockInfo.getTxnId()).execute(jdbcResource); + } else { + new HeartbeatLockFunction(rqst.getLockid()).execute(jdbcResource); } + return txnLockManager.checkLock(extLockId, lockInfo.getTxnId(), false, false); } /** - * Heartbeats on the lock table. This commits, so do not enter it with any state. - * Should not be called on a lock that belongs to transaction. + * This would have been made simpler if all locks were associated with a txn. Then only txn needs to + * be heartbeated, committed, etc. no need for client to track individual locks. + * When removing locks not associated with txn this potentially conflicts with + * heartbeat/performTimeout which are update/delete of HIVE_LOCKS thus will be locked as needed by db. + * since this only removes from HIVE_LOCKS at worst some lock acquire is delayed */ - private void heartbeatLock(Connection dbConn, long extLockId) - throws NoSuchLockException, SQLException, MetaException { - // If the lock id is 0, then there are no locks in this heartbeat - if (extLockId == 0) { - return; - } - try (Statement stmt = dbConn.createStatement()) { - String updateHeartbeatQuery = "UPDATE \"HIVE_LOCKS\" SET \"HL_LAST_HEARTBEAT\" = " + - getEpochFn(dbProduct) + " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId; - LOG.debug("Going to execute update <{}>", updateHeartbeatQuery); - int rc = stmt.executeUpdate(updateHeartbeatQuery); - if (rc < 1) { - LOG.error("Failure to update last heartbeat for extLockId={}.", extLockId); - dbConn.rollback(); - throw new NoSuchLockException("No such lock: " + JavaUtils.lockIdToString(extLockId)); - } - LOG.debug("Successfully heartbeated for extLockId={}", extLockId); - dbConn.commit(); - } + @Override + public void unlock(UnlockRequest rqst) throws TxnOpenException, MetaException { + txnLockManager.unlock(rqst); } - // Heartbeats on the txn table. This commits, so do not enter it with any state - private void heartbeatTxn(Connection dbConn, long txnid) - throws NoSuchTxnException, TxnAbortedException, SQLException, MetaException { - // If the txnid is 0, then there are no transactions in this heartbeat - if (txnid == 0) { - return; - } - try (Statement stmt = dbConn.createStatement()) { - String s = "UPDATE \"TXNS\" SET \"TXN_LAST_HEARTBEAT\" = " + getEpochFn(dbProduct) + - " WHERE \"TXN_ID\" = " + txnid + " AND \"TXN_STATE\" = " + TxnStatus.OPEN; - LOG.debug("Going to execute update <{}>", s); - int rc = stmt.executeUpdate(s); - if (rc < 1) { - ensureValidTxn(dbConn, txnid, stmt); // This should now throw some useful exception. - LOG.error("Can neither heartbeat txn (txnId={}) nor confirm it as invalid.", txnid); - dbConn.rollback(); - throw new NoSuchTxnException("No such txn: " + txnid); - } - LOG.debug("Successfully heartbeated for txnId={}", txnid); - dbConn.commit(); - } + @Override + public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException { + return txnLockManager.showLocks(rqst); } /** - * Returns the state of the transaction if it's able to determine it. Some cases where it cannot: - * 1. txnid was Aborted/Committed and then GC'd (compacted) - * 2. txnid was committed but it didn't modify anything (nothing in COMPLETED_TXN_COMPONENTS) + * {@code ids} should only have txnid or lockid but not both, ideally. + * Currently DBTxnManager.heartbeat() enforces this. */ - private TxnStatus findTxnState(long txnid, Statement stmt) throws SQLException, MetaException { - String s = "SELECT \"TXN_STATE\" FROM \"TXNS\" WHERE \"TXN_ID\" = " + txnid; - LOG.debug("Going to execute query <{}>", s); - try (ResultSet rs = stmt.executeQuery(s)) { - if (!rs.next()) { - s = - sqlGenerator.addLimitClause(1, "1 FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_TXNID\" = " - + txnid); - LOG.debug("Going to execute query <{}>", s); - try (ResultSet rs2 = stmt.executeQuery(s)) { - if (rs2.next()) { - return TxnStatus.COMMITTED; - } - } - // could also check WRITE_SET but that seems overkill - return TxnStatus.UNKNOWN; - } - return TxnStatus.fromString(rs.getString(1)); - } + @Override + public void heartbeat(HeartbeatRequest ids) + throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException { + new HeartbeatTxnFunction(ids.getTxnid()).execute(jdbcResource); + new HeartbeatLockFunction(ids.getLockid()).execute(jdbcResource); + } + + @Override + public HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) throws MetaException { + return new HeartbeatTxnRangeFunction(rqst).execute(jdbcResource); } - /** - * Checks if all the txns in the list are in open state and not read-only. - * @param txnIds list of txns to be evaluated for open state/read-only status - * @param stmt db statement - * @return If all the txns in open state and not read-only, then return true else false - */ - private boolean isTxnsOpenAndNotReadOnly(List txnIds, Statement stmt) throws SQLException { - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); + @Override + public long getTxnIdForWriteId(String dbName, String tblName, long writeId) throws MetaException { + return jdbcResource.execute(new TxnIdForWriteIdHandler(writeId, dbName, tblName)); + } - // Get the count of txns from the given list that are in open state and not read-only. - // If the returned count is same as the input number of txns, then all txns are in open state and not read-only. - prefix.append("SELECT COUNT(*) FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN - + " AND \"TXN_TYPE\" != " + TxnType.READ_ONLY.getValue() + " AND "); + @Override + public CompactionResponse compact(CompactionRequest rqst) throws MetaException { + return new CompactFunction(rqst, openTxnTimeOutMillis, getMutexAPI()).execute(jdbcResource); + } - TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), - txnIds, "\"TXN_ID\"", false, false); + @Override + public boolean submitForCleanup(CompactionRequest rqst, long highestWriteId, long txnId) throws MetaException { + // Put a compaction request in the queue. + long id = new GenerateCompactionQueueIdFunction().execute(jdbcResource); + jdbcResource.execute(new InsertCompactionRequestCommand(id, CompactionState.READY_FOR_CLEANING, rqst).withTxnDetails(highestWriteId, txnId)); + return true; + } - long count = 0; - for (String query : queries) { - LOG.debug("Going to execute query <{}>", query); - try (ResultSet rs = stmt.executeQuery(query)) { - if (rs.next()) { - count += rs.getLong(1); - } - } - } - return count == txnIds.size(); + @Override + public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException { + return jdbcResource.execute(new ShowCompactHandler(rqst, sqlGenerator)); } /** - * Get txns from the list that are either aborted or read-only. - * @param txnIds list of txns to be evaluated for aborted state/read-only status - * @param stmt db statement + * We assume this is only called by metadata cache server to know if there are new base/delta files should be read. + * The query filters compactions by state and only returns SUCCEEDED or READY_FOR_CLEANING compactions because + * only these two states means there are new files ready to be read. */ - private String getAbortedAndReadOnlyTxns(List txnIds, Statement stmt) throws SQLException { - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - - // Check if any of the txns in the list are either aborted or read-only. - prefix.append("SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\" FROM \"TXNS\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), - txnIds, "\"TXN_ID\"", false, false); - StringBuilder txnInfo = new StringBuilder(); - - for (String query : queries) { - LOG.debug("Going to execute query <{}>", query); - try (ResultSet rs = stmt.executeQuery(query)) { - while (rs.next()) { - long txnId = rs.getLong(1); - TxnStatus txnState = TxnStatus.fromString(rs.getString(2)); - TxnType txnType = TxnType.findByValue(rs.getInt(3)); + @Override + public GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo( + GetLatestCommittedCompactionInfoRequest rqst) throws MetaException { + return jdbcResource.execute(new GetLatestCommittedCompactionInfoHandler(rqst)); + } - if (txnState != TxnStatus.OPEN) { - txnInfo.append("{").append(txnId).append(",").append(txnState).append("}"); - } else if (txnType == TxnType.READ_ONLY) { - txnInfo.append("{").append(txnId).append(",read-only}"); - } - } - } - } - return txnInfo.toString(); + @Override + public MetricsInfo getMetricsInfo() throws MetaException { + int threshold = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_TABLES_WITH_ABORTED_TXNS_THRESHOLD); + MetricsInfo metrics = jdbcResource.execute(MetricsInfoHandler.INSTANCE); + Set resourceNames = jdbcResource.execute(new TablesWithAbortedTxnsHandler(threshold)); + metrics.setTablesWithXAbortedTxnsCount(resourceNames.size()); + metrics.setTablesWithXAbortedTxns(resourceNames); + return metrics; } /** - * Get txns from the list that are committed. - * @param txnIds list of txns to be evaluated for committed state - * @param stmt db statement + * Retry-by-caller note: + * This may be retried after dbConn.commit. At worst, it will create duplicate entries in + * TXN_COMPONENTS which won't affect anything. See more comments in {@link #commitTxn(CommitTxnRequest)} */ - private String getCommittedTxns(List txnIds, Statement stmt) throws SQLException { - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - - // Check if any of the txns in the list are committed. - prefix.append("SELECT \"CTC_TXNID\" FROM \"COMPLETED_TXN_COMPONENTS\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), - txnIds, "\"CTC_TXNID\"", false, false); - StringBuilder txnInfo = new StringBuilder(); - - for (String query : queries) { - LOG.debug("Going to execute query <{}>", query); - try (ResultSet rs = stmt.executeQuery(query)) { - while (rs.next()) { - long txnId = rs.getLong(1); - txnInfo.append("{").append(txnId).append(",c}"); - } - } - } - return txnInfo.toString(); + @Override + public void addDynamicPartitions(AddDynamicPartitions rqst) throws NoSuchTxnException, TxnAbortedException, MetaException { + TxnType txnType = jdbcResource.execute(new GetOpenTxnTypeAndLockHandler(sqlGenerator, rqst.getTxnid())); + if (txnType == null) { + //ensures txn is still there and in expected state + new EnsureValidTxnFunction(rqst.getTxnid()).execute(jdbcResource); + shouldNeverHappen(rqst.getTxnid()); + } + jdbcResource.execute(new InsertTxnComponentsCommand(rqst)); + jdbcResource.getJdbcTemplate().update("DELETE FROM \"TXN_COMPONENTS\" " + + "WHERE \"TC_TXNID\" = :txnId AND \"TC_DATABASE\" = :dbName AND \"TC_TABLE\" = :tableName AND \"TC_PARTITION\" IS NULL", + new MapSqlParameterSource() + .addValue("txnId", rqst.getTxnid()) + .addValue("dbName", org.apache.commons.lang3.StringUtils.lowerCase(rqst.getDbname())) + .addValue("tableName", org.apache.commons.lang3.StringUtils.lowerCase(rqst.getTablename()))); } /** - * Used to raise an informative error when the caller expected a txn in a particular TxnStatus - * but found it in some other status + * Clean up corresponding records in metastore tables when corresponding object is dropped, + * specifically: TXN_COMPONENTS, COMPLETED_TXN_COMPONENTS, COMPACTION_QUEUE, COMPLETED_COMPACTIONS + * Retry-by-caller note: this is only idempotent assuming it's only called by dropTable/Db/etc + * operations. + *

+ * HIVE_LOCKS and WS_SET are cleaned up by {@link AcidHouseKeeperService}, if turned on */ - private static void raiseTxnUnexpectedState(TxnStatus actualStatus, long txnid) - throws NoSuchTxnException, TxnAbortedException { - switch (actualStatus) { - case ABORTED: - throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) + " already aborted"); - case COMMITTED: - throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid) + " is already committed."); - case UNKNOWN: - throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid)); - case OPEN: - throw new NoSuchTxnException(JavaUtils.txnIdToString(txnid) + " is " + TxnStatus.OPEN); - default: - throw new IllegalArgumentException("Unknown TxnStatus " + actualStatus); - } + @Override + public void cleanupRecords(HiveObjectType type, Database db, Table table, + Iterator partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException { + new CleanupRecordsFunction(type, db, table, partitionIterator, getDefaultCatalog(conf), keepTxnToWriteIdMetaData, null) + .execute(jdbcResource); + } + + @Override + public void cleanupRecords(HiveObjectType type, Database db, Table table, + Iterator partitionIterator, long txnId) throws MetaException { + new CleanupRecordsFunction(type, db, table, partitionIterator, getDefaultCatalog(conf), false, txnId) + .execute(jdbcResource); } + /** - * Returns the state of the transaction with {@code txnid} or throws if {@code raiseError} is true. + * Catalog hasn't been added to transactional tables yet, so it's passed in but not used. */ - private static void ensureValidTxn(Connection dbConn, long txnid, Statement stmt) - throws SQLException, NoSuchTxnException, TxnAbortedException { - // We need to check whether this transaction is valid and open - String s = "SELECT \"TXN_STATE\" FROM \"TXNS\" WHERE \"TXN_ID\" = " + txnid; - LOG.debug("Going to execute query <{}>", s); - try (ResultSet rs = stmt.executeQuery(s)) { - if (!rs.next()) { - // todo: add LIMIT 1 instead of count - should be more efficient - s = "SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_TXNID\" = " + txnid; - try (ResultSet rs2 = stmt.executeQuery(s)) { - // todo: strictly speaking you can commit an empty txn, thus 2nd conjunct is wrong but - // only - // possible for for multi-stmt txns - boolean alreadyCommitted = rs2.next() && rs2.getInt(1) > 0; - LOG.debug("Going to rollback"); - rollbackDBConn(dbConn); - if (alreadyCommitted) { - // makes the message more informative - helps to find bugs in client code - throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid) - + " is already committed."); - } - throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid)); - } - } - if (TxnStatus.fromString(rs.getString(1)) == TxnStatus.ABORTED) { - LOG.debug("Going to rollback"); - rollbackDBConn(dbConn); - throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) - + " already aborted");// todo: add time of abort, which is not currently tracked. - // Requires schema change - } - } + @Override + public void onRename(String oldCatName, String oldDbName, String oldTabName, String oldPartName, + String newCatName, String newDbName, String newTabName, String newPartName) + throws MetaException { + new OnRenameFunction(oldCatName, oldDbName, oldTabName, oldPartName, newCatName, newDbName, newTabName, newPartName).execute(jdbcResource); } - - private Optional getLockFromLockId(Connection dbConn, long extLockId) throws MetaException, SQLException { - try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_LOCKS_FOR_LOCK_ID_QUERY)) { - pstmt.setLong(1, extLockId); - LOG.debug("Going to execute query <{}> for extLockId={}", SELECT_LOCKS_FOR_LOCK_ID_QUERY, extLockId); - try (ResultSet rs = pstmt.executeQuery()) { - if (!rs.next()) { - return Optional.empty(); - } - LockInfo info = new LockInfo(rs); - LOG.debug("getTxnIdFromLockId({}) Return {}", extLockId, JavaUtils.txnIdToString(info.txnId)); - return Optional.of(info); - } - } + + /** + * TODO: remove in future, for testing only, do not use. + */ + @VisibleForTesting + @Override + public int getNumLocks() { + return Objects.requireNonNull( + jdbcResource.getJdbcTemplate().queryForObject("SELECT COUNT(*) FROM \"HIVE_LOCKS\"", new MapSqlParameterSource(), Integer.TYPE), + "This never should be null, it's just to suppress warnings"); } - // NEVER call this function without first calling heartbeat(long, long) - private List getLocksFromLockId(Connection dbConn, long extLockId) throws MetaException, SQLException { - try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_LOCKS_FOR_LOCK_ID_QUERY)) { - List locks = new ArrayList<>(); - pstmt.setLong(1, extLockId); - LOG.debug("Going to execute query <{}> for extLockId={}", SELECT_LOCKS_FOR_LOCK_ID_QUERY, extLockId); - try (ResultSet rs = pstmt.executeQuery()) { - while (rs.next()) { - locks.add(new LockInfo(rs)); - } - } - if (locks.isEmpty()) { - throw new MetaException("This should never happen! We already " + - "checked the lock(" + JavaUtils.lockIdToString(extLockId) + ") existed but now we can't find it!"); - } - LOG.debug("Found {} locks for extLockId={}. Locks: {}", locks.size(), extLockId, locks); - return locks; - } + /** + * TODO: remove in future, for testing only, do not use. + */ + @VisibleForTesting + @Override + public long setTimeout(long milliseconds) { + long previous_timeout = timeout; + timeout = milliseconds; + return previous_timeout; } - // Clean time out locks from the database not associated with a transactions, i.e. locks - // for read-only autoCommit=true statements. This does a commit, - // and thus should be done before any calls to heartbeat that will leave - // open transactions. - private void timeOutLocks() { - //doing a SELECT first is less efficient but makes it easier to debug things - //when txnid is <> 0, the lock is associated with a txn and is handled by performTimeOuts() - //want to avoid expiring locks for a txn w/o expiring the txn itself + protected Connection getDbConn(int isolationLevel, DataSource connPool) throws SQLException { + Connection dbConn = null; try { - Set timedOutLockIds = new TreeSet<>( - jdbcResource.getJdbcTemplate().query(String.format(SELECT_TIMED_OUT_LOCKS_QUERY, getEpochFn(dbProduct)), - new MapSqlParameterSource().addValue("timeout", timeout), - (rs, rowNum) -> rs.getLong(1))); - if (timedOutLockIds.isEmpty()) { - LOG.debug("Did not find any timed-out locks, therefore retuning."); - return; + dbConn = connPool.getConnection(); + dbConn.setAutoCommit(false); + dbConn.setTransactionIsolation(isolationLevel); + return dbConn; + } catch (SQLException e) { + if (dbConn != null) { + dbConn.close(); } - - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - - //include same hl_last_heartbeat condition in case someone heartbeated since the select - prefix.append("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LAST_HEARTBEAT\" < "); - prefix.append(getEpochFn(dbProduct)).append("-").append(timeout); - prefix.append(" AND \"HL_TXNID\" = 0 AND "); - - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, timedOutLockIds, - "\"HL_LOCK_EXT_ID\"", true, false); - - int deletedLocks = 0; - for (String query : queries) { - LOG.debug("Going to execute update: <{}>", query); - deletedLocks += jdbcResource.getJdbcTemplate().update(query, new MapSqlParameterSource()); - } - if (deletedLocks > 0) { - LOG.info("Deleted {} locks due to timed-out. Lock ids: {}", deletedLocks, timedOutLockIds); - } - } catch (Exception ex) { - LOG.error("Failed to purge timed-out locks: " + ex.getMessage(), ex); + throw e; } } @@ -5783,274 +958,22 @@ private void timeOutLocks() { * Will also delete locks which are not associated with a transaction and have timed out * Tries to keep transactions (against metastore db) small to reduce lock contention. */ - @RetrySemantics.Idempotent + @Override public void performTimeOuts() { - jdbcResource.bindDataSource(POOL_TX); - try (TransactionContext context = jdbcResource.getTransactionManager().getTransaction(PROPAGATION_REQUIRED)) { - //We currently commit after selecting the TXNS to abort. So whether SERIALIZABLE - //READ_COMMITTED, the effect is the same. We could use FOR UPDATE on Select from TXNS - //and do the whole performTimeOuts() in a single huge transaction, but the only benefit - //would be to make sure someone cannot heartbeat one of these txns at the same time. - //The attempt to heartbeat would block and fail immediately after it's unblocked. - //With current (RC + multiple txns) implementation it is possible for someone to send - //heartbeat at the very end of the expiry interval, and just after the Select from TXNS - //is made, in which case heartbeat will succeed but txn will still be Aborted. - //Solving this corner case is not worth the perf penalty. The client should heartbeat in a - //timely way. - timeOutLocks(); - while (true) { - String s = " \"TXN_ID\", \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN + - " AND (" + - "\"TXN_TYPE\" != " + TxnType.REPL_CREATED.getValue() + - " AND \"TXN_LAST_HEARTBEAT\" < " + getEpochFn(dbProduct) + "-" + timeout + - " OR " + - " \"TXN_TYPE\" = " + TxnType.REPL_CREATED.getValue() + - " AND \"TXN_LAST_HEARTBEAT\" < " + getEpochFn(dbProduct) + "-" + replicationTxnTimeout + - ")"; - //safety valve for extreme cases - s = sqlGenerator.addLimitClause(10 * TIMED_OUT_TXN_ABORT_BATCH_SIZE, s); - - LOG.debug("Going to execute query <{}>", s); - List> timedOutTxns = jdbcResource.getJdbcTemplate().query(s, rs -> { - List> txnbatch = new ArrayList<>(); - Map currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE); - while (rs.next()) { - currentBatch.put(rs.getLong(1),TxnType.findByValue(rs.getInt(2))); - if (currentBatch.size() == TIMED_OUT_TXN_ABORT_BATCH_SIZE) { - txnbatch.add(currentBatch); - currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE); - } - } - if (currentBatch.size() > 0) { - txnbatch.add(currentBatch); - } - return txnbatch; - }); - //noinspection DataFlowIssue - if (timedOutTxns.size() == 0) { - jdbcResource.getTransactionManager().commit(context); - return; - } - - Object savePoint = context.getTransactionStatus().createSavepoint(); - - int numTxnsAborted = 0; - for (Map batchToAbort : timedOutTxns) { - context.getTransactionStatus().releaseSavepoint(savePoint); - savePoint = context.getTransactionStatus().createSavepoint(); - if (abortTxns(jdbcResource.getConnection(), new ArrayList<>(batchToAbort.keySet()), true, false, false, TxnErrorMsg.ABORT_TIMEOUT) == batchToAbort.size()) { - numTxnsAborted += batchToAbort.size(); - //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout' - LOG.info("Aborted the following transactions due to timeout: {}", batchToAbort); - if (transactionalListeners != null) { - for (Map.Entry txnEntry : batchToAbort.entrySet()) { - List dbsUpdated = getTxnDbsUpdated(txnEntry.getKey(), jdbcResource.getConnection()); - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.ABORT_TXN, - new AbortTxnEvent(txnEntry.getKey(), txnEntry.getValue(), null, dbsUpdated), - jdbcResource.getConnection(), sqlGenerator); - } - LOG.debug("Added Notifications for the transactions that are aborted due to timeout: {}", batchToAbort); - } - } else { - //could not abort all txns in this batch - this may happen because in parallel with this - //operation there was activity on one of the txns in this batch (commit/abort/heartbeat) - //This is not likely but may happen if client experiences long pause between heartbeats or - //unusually long/extreme pauses between heartbeat() calls and other logic in checkLock(), - //lock(), etc. - context.getTransactionStatus().rollbackToSavepoint(savePoint); - } - } - LOG.info("Aborted {} transaction(s) due to timeout", numTxnsAborted); - if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) { - Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_TIMED_OUT_TXNS).inc(numTxnsAborted); - } - } - } catch (MetaException | SQLException e) { - LOG.warn("Aborting timed out transactions failed due to " + e.getMessage(), e); - } finally { - jdbcResource.unbindDataSource(); - } + new PerformTimeoutsFunction(timeout, replicationTxnTimeout, transactionalListeners).execute(jdbcResource); } @Override - @RetrySemantics.ReadOnly public void countOpenTxns() throws MetaException { - Connection dbConn = null; - Statement stmt = null; - ResultSet rs = null; - try { - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - stmt = dbConn.createStatement(); - String s = "SELECT COUNT(*) FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN; - LOG.debug("Going to execute query <{}>", s); - rs = stmt.executeQuery(s); - if (!rs.next()) { - LOG.error("Transaction database not properly configured, can't find txn_state from TXNS."); - } else { - Long numOpen = rs.getLong(1); - if (numOpen > Integer.MAX_VALUE) { - LOG.error("Open transaction count above {}, can't count that high!", Integer.MAX_VALUE); - } else { - numOpenTxns.set(numOpen.intValue()); - } - } - } catch (SQLException e) { - LOG.info("Failed to update number of open transactions"); - checkRetryable(e, "countOpenTxns()"); - } finally { - close(rs, stmt, dbConn); - } - } catch (RetryException e) { - countOpenTxns(); - } - } - - /** - * Add min history level entry for each generated txn record - * @param dbConn Connection - * @param txnIds new transaction ids - * @deprecated Remove this method when min_history_level table is dropped - * @throws SQLException ex - */ - @Deprecated - private void addTxnToMinHistoryLevel(Connection dbConn, List txnIds, long minOpenTxnId) throws SQLException { - if (!useMinHistoryLevel) { - return; - } - // Need to register minimum open txnid for current transactions into MIN_HISTORY table. - try (Statement stmt = dbConn.createStatement()) { - List rows = txnIds.stream().map(txnId -> txnId + ", " + minOpenTxnId).collect(Collectors.toList()); - - // Insert transaction entries into MIN_HISTORY_LEVEL. - List inserts = - sqlGenerator.createInsertValuesStmt("\"MIN_HISTORY_LEVEL\" (\"MHL_TXNID\", \"MHL_MIN_OPEN_TXNID\")", rows); - for (String insert : inserts) { - LOG.debug("Going to execute insert <{}>", insert); - stmt.execute(insert); - } - LOG.info("Added entries to MIN_HISTORY_LEVEL for current txns: ({}) with min_open_txn: {}", txnIds, minOpenTxnId); - } catch (SQLException e) { - if (dbProduct.isTableNotExistsError(e)) { - // If the table does not exists anymore, we disable the flag and start to work the new way - // This enables to switch to the new functionality without a restart - useMinHistoryLevel = false; - } else { - throw e; - } + int openTxns = jdbcResource.execute(new CountOpenTxnsHandler()); + if (openTxns > -1) { + numOpenTxns.set(openTxns); } } @Override - @RetrySemantics.SafeToRetry public void addWriteIdsToMinHistory(long txnid, Map minOpenWriteIds) throws MetaException { - if (!useMinHistoryWriteId) { - return; - } - // Need to register minimum open writeId for current transactions into MIN_HISTORY_WRITE_ID table. - try { - Connection dbConn = null; - try { - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - try (PreparedStatement pstmt = dbConn.prepareStatement(MIN_HISTORY_WRITE_ID_INSERT_QUERY)) { - int writeId = 0; - - for (Map.Entry validWriteId : minOpenWriteIds.entrySet()) { - String[] names = TxnUtils.getDbTableName(validWriteId.getKey()); - - pstmt.setLong(1, txnid); - pstmt.setString(2, names[0]); - pstmt.setString(3, names[1]); - pstmt.setLong(4, validWriteId.getValue()); - - pstmt.addBatch(); - writeId++; - if (writeId % maxBatchSize == 0) { - LOG.debug("Executing a batch of <" + TXN_TO_WRITE_ID_INSERT_QUERY + "> queries. " + - "Batch size: " + maxBatchSize); - pstmt.executeBatch(); - } - } - if (writeId % maxBatchSize != 0) { - LOG.debug("Executing a batch of <" + TXN_TO_WRITE_ID_INSERT_QUERY + "> queries. " + - "Batch size: " + writeId % maxBatchSize); - pstmt.executeBatch(); - } - } - dbConn.commit(); - LOG.info("Added entries to MIN_HISTORY_WRITE_ID for current txn: {} with min_open_write_ids: ({})", txnid, minOpenWriteIds); - } catch (SQLException e) { - if (dbProduct.isTableNotExistsError(e)) { - // If the table does not exists anymore, we disable the flag and start to work the new way - // This enables to switch to the new functionality without a restart - useMinHistoryWriteId = false; - } else { - LOG.error("Caught exception while storing minOpenWriteIds: ", e); - rollbackDBConn(dbConn); - checkRetryable(e, "addWriteIdsToMinHistory"); - throw new MetaException(e.getMessage()); - } - } finally { - closeDbConn(dbConn); - } - } catch (RetryException e) { - addWriteIdsToMinHistory(txnid, minOpenWriteIds); - } - } - - /** - * Remove txns from min_history_level table - * @param dbConn connection - * @param txnids transactions - * @deprecated Remove this method when min_history_level table is dropped - */ - @Deprecated - private void removeTxnsFromMinHistoryLevel(Connection dbConn, List txnids) throws SQLException { - if (!useMinHistoryLevel) { - return; - } - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder("DELETE FROM \"MIN_HISTORY_LEVEL\" WHERE "); - try (Statement stmt = dbConn.createStatement()) { - TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), txnids, "\"MHL_TXNID\"", false, false); - executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize); - LOG.info("Removed transactions: ({}) from MIN_HISTORY_LEVEL", txnids); - } catch (SQLException e) { - if (dbProduct.isTableNotExistsError(e)) { - // If the table does not exists anymore, we disable the flag and start to work the new way - // This enables to switch to the new functionality without a restart - useMinHistoryLevel = false; - } else { - throw e; - } - } - } - - /** - * Remove minOpenWriteIds from min_history_write_id tables - * @param dbConn connection - * @param txnids transactions - */ - private void removeWriteIdsFromMinHistory(Connection dbConn, List txnids) throws SQLException { - if (!useMinHistoryWriteId) { - return; - } - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder("DELETE FROM \"MIN_HISTORY_WRITE_ID\" WHERE "); - try (Statement stmt = dbConn.createStatement()) { - TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), txnids, "\"MH_TXNID\"", false, false); - executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize); - LOG.info("Removed transactions: ({}) from MIN_HISTORY_WRITE_ID", txnids); - } catch (SQLException e) { - if (dbProduct.isTableNotExistsError(e)) { - // If the table does not exists anymore, we disable the flag and start to work the new way - // This enables to switch to the new functionality without a restart - useMinHistoryWriteId = false; - } else { - throw e; - } - } + jdbcResource.execute(new AddWriteIdsToMinHistoryCommand(txnid, minOpenWriteIds)); } protected synchronized static DataSource setupJdbcConnectionPool(Configuration conf, int maxPoolSize) { @@ -6063,450 +986,154 @@ protected synchronized static DataSource setupJdbcConnectionPool(Configuration c throw new RuntimeException(e); } } else { - String connectionPooler = MetastoreConf.getVar(conf, ConfVars.CONNECTION_POOLING_TYPE).toLowerCase(); + String connectionPooler = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE).toLowerCase(); if ("none".equals(connectionPooler)) { LOG.info("Choosing not to pool JDBC connections"); - return new NoPoolConnectionPool(conf); + return new NoPoolConnectionPool(conf, dbProduct); } else { throw new RuntimeException("Unknown JDBC connection pooling " + connectionPooler); } } } - /** - * Returns true if {@code ex} should be retried - */ - static boolean isRetryable(Configuration conf, Exception ex) { - if(ex instanceof SQLException) { - SQLException sqlException = (SQLException)ex; - if (MANUAL_RETRY.equalsIgnoreCase(sqlException.getSQLState())) { - // Manual retry exception was thrown - return true; - } - if ("08S01".equalsIgnoreCase(sqlException.getSQLState())) { - //in MSSQL this means Communication Link Failure - return true; - } - if ("ORA-08176".equalsIgnoreCase(sqlException.getSQLState()) || - sqlException.getMessage().contains("consistent read failure; rollback data not available")) { - return true; - } - - String regex = MetastoreConf.getVar(conf, ConfVars.TXN_RETRYABLE_SQLEX_REGEX); - if (regex != null && !regex.isEmpty()) { - String[] patterns = regex.split(",(?=\\S)"); - String message = getMessage((SQLException)ex); - for (String p : patterns) { - if (Pattern.matches(p, message)) { - return true; - } - } - } - //see also https://issues.apache.org/jira/browse/HIVE-9938 - } - return false; - } - - private boolean isDuplicateKeyError(SQLException ex) { - return dbProduct.isDuplicateKeyError(ex); + @Override + public MutexAPI getMutexAPI() { + return mutexAPI; } - private static String getMessage(SQLException ex) { - return ex.getMessage() + " (SQLState=" + ex.getSQLState() + ", ErrorCode=" + ex.getErrorCode() + ")"; - } - static String quoteString(String input) { - return "'" + input + "'"; - } - static String quoteChar(char c) { - return "'" + c + "'"; + @Override + public LockHandle acquireLock(String key) throws MetaException { + return mutexAPI.acquireLock(key); } - /** - * {@link #lockInternal()} and {@link #unlockInternal()} are used to serialize those operations that require - * Select ... For Update to sequence operations properly. In practice that means when running - * with Derby database. See more notes at class level. - */ - protected void lockInternal() { - if(dbProduct.isDERBY()) { - derbyLock.lock(); - } - } - protected void unlockInternal() { - if(dbProduct.isDERBY()) { - derbyLock.unlock(); - } - } @Override - @RetrySemantics.Idempotent - public MutexAPI getMutexAPI() { - return this; + public void acquireLock(String key, LockHandle handle) throws MetaException { + mutexAPI.acquireLock(key, handle); } @Override - public LockHandle acquireLock(String key) throws MetaException { - /** - * The implementation here is a bit kludgey but done so that code exercised by unit tests - * (which run against Derby which has no support for select for update) is as similar to - * production code as possible. - * In particular, with Derby we always run in a single process with a single metastore and - * the absence of For Update is handled via a Semaphore. The later would strictly speaking - * make the SQL statements below unnecessary (for Derby), but then they would not be tested. - */ - Connection dbConn = null; - Statement stmt = null; - ResultSet rs = null; - boolean needToCloseConn = true; - try { - try { - String sqlStmt = sqlGenerator.addForUpdateClause("SELECT \"MT_COMMENT\", \"MT_KEY2\" FROM \"AUX_TABLE\" WHERE \"MT_KEY1\"=" + quoteString(key)); - lockInternal(); - dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolMutex); - stmt = dbConn.createStatement(); - LOG.debug("About to execute SQL: {}", sqlStmt); - rs = stmt.executeQuery(sqlStmt); - if (!rs.next()) { - close(rs); - try { - stmt.executeUpdate("INSERT INTO \"AUX_TABLE\" (\"MT_KEY1\", \"MT_KEY2\") VALUES(" + quoteString(key) + ", 0)"); - dbConn.commit(); - } catch (SQLException ex) { - if (!isDuplicateKeyError(ex)) { - throw new RuntimeException("Unable to lock " + quoteString(key) + " due to: " + getMessage(ex), ex); - } - //if here, it means a concrurrent acquireLock() inserted the 'key' - - //rollback is done for the benefit of Postgres which throws (SQLState=25P02, ErrorCode=0) if - //you attempt any stmt in a txn which had an error. - dbConn.rollback(); - } - rs = stmt.executeQuery(sqlStmt); - if (!rs.next()) { - throw new IllegalStateException("Unable to lock " + quoteString(key) + ". Expected row in AUX_TABLE is missing."); - } - } - Semaphore derbySemaphore = null; - if(dbProduct.isDERBY()) { - derbyKey2Lock.putIfAbsent(key, new Semaphore(1)); - derbySemaphore = derbyKey2Lock.get(key); - derbySemaphore.acquire(); - } - if (LOG.isDebugEnabled()) { - LOG.debug("{} locked by {}", quoteString(key), quoteString(TxnHandler.hostname)); - } - needToCloseConn = false; //The connection is good, we need not close it - //OK, so now we have a lock - return new LockHandleImpl(dbConn, stmt, rs, key, derbySemaphore); - } catch (SQLException ex) { - checkRetryable(ex, "acquireLock(" + key + ")"); - throw new MetaException("Unable to lock " + quoteString(key) + " due to: " + getMessage(ex) + "; " + StringUtils.stringifyException(ex)); - } - catch(InterruptedException ex) { - throw new MetaException("Unable to lock " + quoteString(key) + " due to: " + ex.getMessage() + StringUtils.stringifyException(ex)); - } - finally { - if (needToCloseConn) { - rollbackDBConn(dbConn); - close(rs, stmt, dbConn); - } - unlockInternal(); - } - } - catch(RetryException ex) { - return acquireLock(key); + public AbortCompactResponse abortCompactions(AbortCompactionRequest reqst) throws MetaException, NoSuchCompactionException { + if (reqst.getCompactionIds().isEmpty()) { + LOG.info("Compaction ids are missing in request. No compactions to abort"); + throw new NoSuchCompactionException("Compaction ids missing in request. No compactions to abort"); } + return new AbortCompactionFunction(reqst, sqlRetryHandler).execute(jdbcResource); } - @Override - public void acquireLock(String key, LockHandle handle) { - //the idea is that this will use LockHandle.dbConn - throw new NotImplementedException("acquireLock(String, LockHandle) is not implemented"); + private static void shouldNeverHappen(long txnid) { + throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid)); + } + + private void deleteInvalidOpenTransactions(List txnIds) throws MetaException { + try { + sqlRetryHandler.executeWithRetry(new SqlRetryCallProperties().withCallerId("deleteInvalidOpenTransactions"), + () -> { + jdbcResource.execute(new DeleteInvalidOpenTxnsCommand(txnIds)); + LOG.info("Removed transactions: ({}) from TXNS", txnIds); + jdbcResource.execute(new RemoveTxnsFromMinHistoryLevelCommand(txnIds)); + return null; + }); + } catch (TException e) { + throw new MetaException(e.getMessage()); + } } /** * Acquire the global txn lock, used to mutex the openTxn and commitTxn. * @param shared either SHARED_READ or EXCLUSIVE - * @throws SQLException */ - private void acquireTxnLock(Statement stmt, boolean shared) throws SQLException, MetaException { + private void acquireTxnLock(boolean shared) throws MetaException { String sqlStmt = sqlGenerator.createTxnLockStatement(shared); - stmt.execute(sqlStmt); - LOG.debug("TXN lock locked by {} in mode {}", quoteString(TxnHandler.hostname), shared); + jdbcResource.getJdbcTemplate().getJdbcTemplate().execute((Statement stmt) -> { + stmt.execute(sqlStmt); + return null; + }); + LOG.debug("TXN lock locked by '{}' in mode {}", TxnHandler.hostname, shared); } - private static final class LockHandleImpl implements LockHandle { - private final Connection dbConn; - private final Statement stmt; - private final ResultSet rs; - private final Semaphore derbySemaphore; - private final String key; - private final Long lastUpdateTime; - - LockHandleImpl(Connection conn, Statement stmt, ResultSet rs, String key, Semaphore derbySemaphore) { - this.dbConn = conn; - this.stmt = stmt; - this.rs = rs; - this.derbySemaphore = derbySemaphore; - if(derbySemaphore != null) { - //oterwise it may later release permit acquired by someone else - assert derbySemaphore.availablePermits() == 0 : "Expected locked Semaphore"; - } - this.key = key; - Long lastUpdateTime; - try { - lastUpdateTime = rs.getLong("MT_KEY2"); - } catch (SQLException e) { - LOG.warn("Couldn't resolve MT_KEY2 for MT_KEY1=" + quoteString(this.key), e); - lastUpdateTime = -1L; - } - this.lastUpdateTime = lastUpdateTime; - } - - @Override - public void releaseLocks() { - rollbackDBConn(dbConn); - TxnHandler.close(rs, stmt, dbConn); - if(derbySemaphore != null) { - derbySemaphore.release(); - } - if (LOG.isDebugEnabled()) { - LOG.debug("{} unlocked by {}", quoteString(key), quoteString(TxnHandler.hostname)); - } - } - - @Override - public Long getLastUpdateTime() { - return lastUpdateTime; - } + /** + * Determine the current time, using the RDBMS as a source of truth + * @return current time in milliseconds + * @throws org.apache.hadoop.hive.metastore.api.MetaException if the time cannot be determined + */ + protected Timestamp getDbTime() throws MetaException { + return jdbcResource.getJdbcTemplate().queryForObject( + dbProduct.getDBTime(), + new MapSqlParameterSource(), + (ResultSet rs, int rowNum) -> rs.getTimestamp(1)); + } - @Override - public void releaseLocks(Long timestamp) { - try { - stmt.executeUpdate("UPDATE \"AUX_TABLE\" SET \"MT_KEY2\" = "+ timestamp + " WHERE \"MT_KEY1\"=" + quoteString(key)); - dbConn.commit(); - } catch (SQLException ex) { - LOG.warn("Unable to update MT_KEY2 value for MT_KEY1=" + key, ex); - rollbackDBConn(dbConn); - } - TxnHandler.close(rs, stmt, dbConn); - if(derbySemaphore != null) { - derbySemaphore.release(); - } - if (LOG.isDebugEnabled()) { - LOG.debug("{} unlocked by {}", quoteString(key), quoteString(TxnHandler.hostname)); + private void determineDatabaseProduct(Connection conn) { + try { + String s = conn.getMetaData().getDatabaseProductName(); + dbProduct = DatabaseProduct.determineDatabaseProduct(s, conf); + if (dbProduct.isUNDEFINED()) { + String msg = "Unrecognized database product name <" + s + ">"; + LOG.error(msg); + throw new IllegalStateException(msg); } + } catch (SQLException e) { + String msg = "Unable to get database product name"; + LOG.error(msg, e); + throw new IllegalStateException(msg, e); } - - @Override - public void close() { - releaseLocks(); - } - } - - - private static class NoPoolConnectionPool implements DataSource { - // Note that this depends on the fact that no-one in this class calls anything but - // getConnection. If you want to use any of the Logger or wrap calls you'll have to - // implement them. - private final Configuration conf; - private Driver driver; - private String connString; - private String user; - private String passwd; - - public NoPoolConnectionPool(Configuration conf) { - this.conf = conf; + + private void initJdbcResource() { + if (jdbcResource == null) { + jdbcResource = new MultiDataSourceJdbcResource(dbProduct, conf, sqlGenerator); + jdbcResource.registerDataSource(POOL_TX, connPool); + jdbcResource.registerDataSource(POOL_MUTEX, connPoolMutex); + jdbcResource.registerDataSource(POOL_COMPACTOR, connPoolCompactor); } + } - @Override - public Connection getConnection() throws SQLException { - if (user == null) { - user = DataSourceProvider.getMetastoreJdbcUser(conf); - passwd = DataSourceProvider.getMetastoreJdbcPasswd(conf); - } - return getConnection(user, passwd); + /** + * Check if provided table is usable + * @return + */ + private boolean checkIfTableIsUsable(String tableName, boolean configValue) { + if (!configValue) { + // don't check it if disabled + return false; } - - @Override - public Connection getConnection(String username, String password) throws SQLException { - // Find the JDBC driver - if (driver == null) { - String driverName = MetastoreConf.getVar(conf, ConfVars.CONNECTION_DRIVER); - if (driverName == null || driverName.equals("")) { - String msg = "JDBC driver for transaction db not set in configuration " + - "file, need to set " + ConfVars.CONNECTION_DRIVER.getVarname(); - LOG.error(msg); - throw new RuntimeException(msg); - } - try { - LOG.info("Going to load JDBC driver {}", driverName); - driver = (Driver) Class.forName(driverName).newInstance(); - } catch (InstantiationException e) { - throw new RuntimeException("Unable to instantiate driver " + driverName + ", " + - e.getMessage(), e); - } catch (IllegalAccessException e) { + jdbcResource.bindDataSource(POOL_TX); + try { + jdbcResource.getJdbcTemplate().query("SELECT 1 FROM \"" + tableName + "\"", + new MapSqlParameterSource(), ResultSet::next); + } catch (DataAccessException e) { + LOG.debug("Catching sql exception in " + tableName + " check", e); + if (e.getCause() instanceof SQLException) { + if (dbProduct.isTableNotExistsError(e)) { + return false; + } else { throw new RuntimeException( - "Unable to access driver " + driverName + ", " + e.getMessage(), - e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Unable to find driver " + driverName + ", " + e.getMessage(), - e); - } - connString = MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY); - } - - try { - LOG.info("Connecting to transaction db with connection string {}", connString); - Properties connectionProps = new Properties(); - connectionProps.setProperty("user", username); - connectionProps.setProperty("password", password); - Connection conn = driver.connect(connString, connectionProps); - String prepareStmt = dbProduct != null ? dbProduct.getPrepareTxnStmt() : null; - if (prepareStmt != null) { - try (Statement stmt = conn.createStatement()) { - stmt.execute(prepareStmt); - } + "Unable to select from transaction database: " + SqlRetryHandler.getMessage(e) + StringUtils.stringifyException(e)); } - conn.setAutoCommit(false); - return conn; - } catch (SQLException e) { - throw new RuntimeException("Unable to connect to transaction manager using " + connString - + ", " + e.getMessage(), e); } + } finally { + jdbcResource.unbindDataSource(); } - - @Override - public PrintWriter getLogWriter() throws SQLException { - throw new UnsupportedOperationException(); - } - - @Override - public void setLogWriter(PrintWriter out) throws SQLException { - throw new UnsupportedOperationException(); - } - - @Override - public void setLoginTimeout(int seconds) throws SQLException { - throw new UnsupportedOperationException(); - } - - @Override - public int getLoginTimeout() throws SQLException { - throw new UnsupportedOperationException(); - } - - @Override - public java.util.logging.Logger getParentLogger() throws SQLFeatureNotSupportedException { - throw new UnsupportedOperationException(); - } - - @Override - public T unwrap(Class iface) throws SQLException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isWrapperFor(Class iface) throws SQLException { - throw new UnsupportedOperationException(); - } - } - - @Override - @RetrySemantics.SafeToRetry - public AbortCompactResponse abortCompactions(AbortCompactionRequest reqst) throws MetaException, NoSuchCompactionException { - Map abortCompactionResponseElements = new HashMap<>(); - AbortCompactResponse response = new AbortCompactResponse(new HashMap<>()); - response.setAbortedcompacts(abortCompactionResponseElements); - - List compactionIdsToAbort = reqst.getCompactionIds(); - if (compactionIdsToAbort.isEmpty()) { - LOG.info("Compaction ids are missing in request. No compactions to abort"); - throw new NoSuchCompactionException("Compaction ids missing in request. No compactions to abort"); - } - reqst.getCompactionIds().forEach(x -> abortCompactionResponseElements.put(x, getAbortCompactionResponseElement(x,"Error","No Such Compaction Id Available"))); - - List eligibleCompactionsToAbort = findEligibleCompactionsToAbort(abortCompactionResponseElements, - compactionIdsToAbort); - for (CompactionInfo compactionInfo : eligibleCompactionsToAbort) { - abortCompactionResponseElements.put(compactionInfo.id, abortCompaction(compactionInfo)); - } - return response; - } - - private AbortCompactionResponseElement getAbortCompactionResponseElement(long compactionId, String status, String message) { - AbortCompactionResponseElement resEle = new AbortCompactionResponseElement(compactionId); - resEle.setMessage(message); - resEle.setStatus(status); - return resEle; - } - - @RetrySemantics.SafeToRetry - private AbortCompactionResponseElement abortCompaction(CompactionInfo compactionInfo) throws MetaException { - SqlRetryFunction function = () -> { - jdbcResource.bindDataSource(POOL_TX); - try (TransactionContext context = jdbcResource.getTransactionManager().getTransaction(PROPAGATION_REQUIRED)) { - compactionInfo.state = TxnStore.ABORTED_STATE; - compactionInfo.errorMessage = "Compaction Aborted by Abort Comapction request."; - int updCount; - try { - updCount = jdbcResource.execute(new InsertCompactionInfoCommand(compactionInfo, getDbTime().getTime())); - } catch (Exception e) { - LOG.error("Unable to update compaction record: {}.", compactionInfo); - return getAbortCompactionResponseElement(compactionInfo.id, "Error", - "Error while aborting compaction:Unable to update compaction record in COMPLETED_COMPACTIONS"); - } - LOG.debug("Inserted {} entries into COMPLETED_COMPACTIONS", updCount); - try { - updCount = jdbcResource.getJdbcTemplate().update("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = :id", - new MapSqlParameterSource().addValue("id", compactionInfo.id)); - if (updCount != 1) { - LOG.error("Unable to update compaction record: {}. updCnt={}", compactionInfo, updCount); - return getAbortCompactionResponseElement(compactionInfo.id, "Error", - "Error while aborting compaction: Unable to update compaction record in COMPACTION_QUEUE"); - } else { - jdbcResource.getTransactionManager().commit(context); - return getAbortCompactionResponseElement(compactionInfo.id, "Success", - "Successfully aborted compaction"); - } - } catch (DataAccessException e) { - return getAbortCompactionResponseElement(compactionInfo.id, "Error", - "Error while aborting compaction:" + e.getMessage()); - } - } finally { - jdbcResource.unbindDataSource(); - } - }; - return sqlRetryHandler.executeWithRetry( - new SqlRetryCallProperties().withCallerId("abortCompaction(" + compactionInfo + ")"), function); + return true; } - - private List findEligibleCompactionsToAbort(Map abortCompactionResponseElements, List requestedCompId) throws MetaException { - List compactionInfoList = new ArrayList<>(); - String queryText = TxnQueries.SELECT_COMPACTION_QUEUE_BY_COMPID + " WHERE \"CC_ID\" IN (?) " ; - String sqlIN = requestedCompId.stream() - .map(x -> String.valueOf(x)) - .collect(Collectors.joining(",", "(", ")")); - queryText = queryText.replace("(?)", sqlIN); - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED); - Statement pStmt = dbConn.createStatement()) { - try (ResultSet rs = pStmt.executeQuery(queryText)) { - while (rs.next()) { - char compState = rs.getString(5).charAt(0); - long compID = rs.getLong(1); - if (CompactionState.INITIATED.equals(CompactionState.fromSqlConst(compState))) { - compactionInfoList.add(CompactionInfo.loadFullFromCompactionQueue(rs)); - } else { - abortCompactionResponseElements.put(compID, getAbortCompactionResponseElement(compID,"Error", - "Error while aborting compaction as compaction is in state-" + CompactionState.fromSqlConst(compState))); - } - } - } - } catch (SQLException e) { - throw new MetaException("Unable to select from transaction database-" + StringUtils.stringifyException(e)); + /** + * Returns the databases updated by txnId. + * Queries TXN_TO_WRITE_ID using txnId. + * + * @param txnId + * @throws MetaException + */ + private List getTxnDbsUpdated(long txnId) throws MetaException { + try { + return sqlRetryHandler.executeWithRetry( + new SqlRetryCallProperties().withCallerId("GetTxnDbsUpdatedHandler"), + () -> jdbcResource.execute(new GetTxnDbsUpdatedHandler(txnId))); + } catch (MetaException e) { + throw e; + } catch (TException e) { + throw new MetaException(e.getMessage()); } - return compactionInfoList; } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java new file mode 100644 index 000000000000..e5f31f402124 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn; + +import org.apache.hadoop.hive.common.classification.RetrySemantics; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; +import org.apache.hadoop.hive.metastore.api.UnlockRequest; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetry; +import org.springframework.transaction.annotation.Transactional; + +import static org.apache.hadoop.hive.metastore.txn.TxnStore.POOL_TX; + +public interface TxnLockManager { + + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) + long enqueueLock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException; + + /** + * Check whether a lock has been obtained. This is used after {@link #enqueueLock(LockRequest)} returned a wait + * state. + * @param extLockId + * @param txnId Transaction id + * @param zeroWaitReadEnabled + * @param isExclusiveCTAS + * @return info on the state of the lock + * @throws NoSuchTxnException + * @throws NoSuchLockException + * @throws TxnAbortedException + * @throws MetaException + */ + @SqlRetry(lockInternally = true) + @Transactional(value = POOL_TX, noRollbackFor = {TxnAbortedException.class}) + LockResponse checkLock(long extLockId, long txnId, boolean zeroWaitReadEnabled, boolean isExclusiveCTAS) + throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException; + + /** + * Unlock a lock. It is not legal to call this if the caller is part of a txn. In that case + * the txn should be committed or aborted instead. (Note someday this will change since + * multi-statement transactions will allow unlocking in the transaction.) + * @param rqst lock to unlock + * @throws TxnOpenException + * @throws MetaException + */ + @SqlRetry + @Transactional(POOL_TX) + @RetrySemantics.Idempotent + void unlock(UnlockRequest rqst) + throws TxnOpenException, MetaException; + + /** + * Get information on current locks. + * @param rqst lock information to retrieve + * @return lock information. + * @throws MetaException + */ + @SqlRetry + @Transactional(POOL_TX) + @RetrySemantics.ReadOnly + ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException; + +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index bb1a5858191c..f31308ba3973 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -18,17 +18,15 @@ package org.apache.hadoop.hive.metastore.txn; import com.google.common.annotations.VisibleForTesting; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.classification.RetrySemantics; -import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; -import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException; -import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest; import org.apache.hadoop.hive.metastore.api.AbortCompactResponse; import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; +import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest; import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions; import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest; import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse; @@ -55,6 +53,7 @@ import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdRequest; import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdResponse; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException; import org.apache.hadoop.hive.metastore.api.NoSuchLockException; import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; @@ -74,17 +73,21 @@ import org.apache.hadoop.hive.metastore.api.UnlockRequest; import org.apache.hadoop.hive.metastore.api.UpdateTransactionalStatsRequest; import org.apache.hadoop.hive.metastore.events.ListenerEvent; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData; +import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo; import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetry; -import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetry; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryException; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler; import org.springframework.transaction.annotation.Transactional; import java.sql.SQLException; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.Map; /** * A handler to answer transaction related calls that come into the metastore @@ -157,23 +160,29 @@ enum MUTEX_KEY { * @return information about open transactions * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException; /** * Get list of valid transactions. This gives just the list of transactions that are open. - * @return list of open transactions, as well as a high water mark. + * @return list of open transactions, as well as a high watermark. * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly GetOpenTxnsResponse getOpenTxns() throws MetaException; /** * Get list of valid transactions. This gives just the list of transactions that are open. * @param excludeTxnTypes : excludes this type of txns while getting the open txns - * @return list of open transactions, as well as a high water mark. + * @return list of open transactions, as well as a high watermark. * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly GetOpenTxnsResponse getOpenTxns(List excludeTxnTypes) throws MetaException; @@ -181,6 +190,8 @@ enum MUTEX_KEY { * Get the count for open transactions. * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly void countOpenTxns() throws MetaException; @@ -190,9 +201,13 @@ enum MUTEX_KEY { * @return information on opened transactions * @throws MetaException */ + @SqlRetry + @Transactional(value = POOL_TX, noRollbackFor = SqlRetryException.class) @RetrySemantics.Idempotent OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException; + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) @RetrySemantics.Idempotent long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException; @@ -202,6 +217,8 @@ enum MUTEX_KEY { * @throws NoSuchTxnException * @throws MetaException */ + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) @RetrySemantics.Idempotent void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException; @@ -211,6 +228,8 @@ enum MUTEX_KEY { * @throws NoSuchTxnException * @throws MetaException */ + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) @RetrySemantics.Idempotent void abortTxns(AbortTxnsRequest rqst) throws NoSuchTxnException, MetaException; @@ -221,7 +240,9 @@ enum MUTEX_KEY { * @throws TxnAbortedException * @throws MetaException */ - @RetrySemantics.Idempotent + @SqlRetry(lockInternally = true) + @Transactional(value = POOL_TX, noRollbackFor = TxnAbortedException.class) + @RetrySemantics.Idempotent("No-op if already committed") void commitTxn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException; @@ -230,9 +251,12 @@ void commitTxn(CommitTxnRequest rqst) * @param rqst info on table/partitions and writeid snapshot to replicate. * @throws MetaException in case of failure */ - @RetrySemantics.Idempotent + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) + @RetrySemantics.Idempotent("No-op if already replicated the writeid state") void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException; + @Transactional(POOL_TX) void updateTransactionStatistics(UpdateTransactionalStatsRequest req) throws MetaException; /** @@ -243,6 +267,7 @@ void commitTxn(CommitTxnRequest rqst) * @param validTxnList valid transaction list for snapshot taken for current query * @throws MetaException */ + @Transactional(POOL_TX) @RetrySemantics.Idempotent Materialization getMaterializationInvalidationInfo( final CreationMetadata cm, final String validTxnList) @@ -252,27 +277,40 @@ Materialization getMaterializationInvalidationInfo( long getTxnIdForWriteId(String dbName, String tblName, long writeId) throws MetaException; + @SqlRetry + @Transactional(POOL_TX) + @RetrySemantics.ReadOnly long getLatestTxnIdInConflict(long txnid) throws MetaException; + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) throws MetaException; + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws MetaException; + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) throws MetaException; - /** - * Gets the list of valid write ids for the given table wrt to current txn - * @param rqst info on transaction and list of table names associated with given transaction - * @throws NoSuchTxnException - * @throws MetaException - */ + /** + * Gets the list of valid write ids for the given table wrt to current txn + * @param rqst info on transaction and list of table names associated with given transaction + * @throws NoSuchTxnException + * @throws MetaException + */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst) throws NoSuchTxnException, MetaException; + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry void addWriteIdsToMinHistory(long txnId, Map minOpenWriteIds) throws MetaException; @@ -283,6 +321,8 @@ GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst) * @throws TxnAbortedException * @throws MetaException */ + @SqlRetry(lockInternally = true, retryOnDuplicateKey = true) + @Transactional(POOL_TX) AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException; @@ -291,6 +331,8 @@ AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest * @param rqst table for which the maximum writeId is requested * @return the maximum allocated writeId */ + @SqlRetry + @Transactional(POOL_TX) MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWriteIdRequest rqst) throws MetaException; @@ -298,6 +340,8 @@ MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWri * Called on conversion of existing table to full acid. Sets initial write ID to a high * enough value so that we can assign unique ROW__IDs to data in existing files. */ + @SqlRetry + @Transactional(POOL_TX) void seedWriteId(SeedTableWriteIdsRequest rqst) throws MetaException; /** @@ -305,6 +349,8 @@ MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWri * If the actual txnId is greater it will throw an exception. * @param rqst */ + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) void seedTxnId(SeedTxnIdRequest rqst) throws MetaException; /** @@ -330,6 +376,7 @@ LockResponse lock(LockRequest rqst) * @throws TxnAbortedException * @throws MetaException */ + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry LockResponse checkLock(CheckLockRequest rqst) throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException; @@ -343,6 +390,7 @@ LockResponse checkLock(CheckLockRequest rqst) * @throws TxnOpenException * @throws MetaException */ + @Transactional(POOL_TX) @RetrySemantics.Idempotent void unlock(UnlockRequest rqst) throws NoSuchLockException, TxnOpenException, MetaException; @@ -353,6 +401,7 @@ void unlock(UnlockRequest rqst) * @return lock information. * @throws MetaException */ + @Transactional(POOL_TX) @RetrySemantics.ReadOnly ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException; @@ -364,6 +413,8 @@ void unlock(UnlockRequest rqst) * @throws TxnAbortedException * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry void heartbeat(HeartbeatRequest ids) throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException; @@ -374,6 +425,8 @@ void heartbeat(HeartbeatRequest ids) * @return info on txns that were heartbeated * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) throws MetaException; @@ -385,9 +438,13 @@ HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) * @return id of the compaction that has been started or existing id if this resource is already scheduled * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.Idempotent CompactionResponse compact(CompactionRequest rqst) throws MetaException; + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry boolean submitForCleanup(CompactionRequest rqst, long highestWriteId, long txnId) throws MetaException; @@ -397,6 +454,8 @@ HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) * @return compaction information * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException; @@ -408,6 +467,7 @@ HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) * @throws NoSuchCompactionException * @throws MetaException */ + @Transactional(POOL_TX) @RetrySemantics.Idempotent AbortCompactResponse abortCompactions(AbortCompactionRequest rqst) throws NoSuchCompactionException, MetaException; @@ -422,6 +482,8 @@ HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) * partition specified by the request. * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo( GetLatestCommittedCompactionInfoRequest rqst) throws MetaException; @@ -433,6 +495,8 @@ GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo( * @throws TxnAbortedException * @throws MetaException */ + @SqlRetry(lockInternally = true) + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry void addDynamicPartitions(AddDynamicPartitions rqst) throws NoSuchTxnException, TxnAbortedException, MetaException; @@ -445,20 +509,28 @@ void addDynamicPartitions(AddDynamicPartitions rqst) * @param partitionIterator partition iterator * @throws MetaException */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.Idempotent default void cleanupRecords(HiveObjectType type, Database db, Table table, Iterator partitionIterator) throws MetaException { cleanupRecords(type, db, table, partitionIterator, false); } + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.Idempotent void cleanupRecords(HiveObjectType type, Database db, Table table, Iterator partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException; + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.Idempotent void cleanupRecords(HiveObjectType type, Database db, Table table, Iterator partitionIterator, long txnId) throws MetaException; + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.Idempotent void onRename(String oldCatName, String oldDbName, String oldTabName, String oldPartName, String newCatName, String newDbName, String newTabName, String newPartName) @@ -467,6 +539,7 @@ void onRename(String oldCatName, String oldDbName, String oldTabName, String old /** * Timeout transactions and/or locks. This should only be called by the compactor. */ + @Transactional(POOL_TX) @RetrySemantics.Idempotent void performTimeOuts(); @@ -712,6 +785,7 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * WriteSet tracking is used to ensure proper transaction isolation. This method deletes the * transaction metadata once it becomes unnecessary. */ + @Transactional(POOL_TX) @RetrySemantics.SafeToRetry void performWriteSetGC() throws MetaException; @@ -729,7 +803,8 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT boolean checkFailedCompactions(CompactionInfo ci) throws MetaException; @VisibleForTesting - int numLocksInLockTable() throws SQLException, MetaException; + @Transactional(POOL_TX) + int getNumLocks() throws SQLException, MetaException; @VisibleForTesting long setTimeout(long milliseconds); @@ -755,6 +830,7 @@ interface MutexAPI { * The {@code key} is name of the lock. Will acquire an exclusive lock or block. It returns * a handle which must be used to release the lock. Each invocation returns a new handle. */ + @SqlRetry(lockInternally = true) LockHandle acquireLock(String key) throws MetaException; /** @@ -838,6 +914,8 @@ interface LockHandle extends AutoCloseable { * Returns ACID metadata related metrics info. * @return metrics info object */ + @SqlRetry + @Transactional(POOL_TX) @RetrySemantics.ReadOnly MetricsInfo getMetricsInfo() throws MetaException; @@ -855,7 +933,7 @@ interface LockHandle extends AutoCloseable { @SqlRetry @Transactional(POOL_COMPACTOR) CompactionMetricsData getCompactionMetricsData(String dbName, String tblName, String partitionName, - CompactionMetricsData.MetricType type) throws MetaException; + CompactionMetricsData.MetricType type) throws MetaException; /** * Remove records from the compaction metrics cache matching the filter criteria passed in as parameters diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java new file mode 100644 index 000000000000..1013493a7913 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn; + +import org.apache.commons.lang3.NotImplementedException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.tools.SQLGenerator; +import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource; +import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext; +import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler; +import org.apache.hadoop.hive.metastore.utils.JavaUtils; +import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.dao.DataAccessException; +import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; + +import java.sql.SQLException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; + +import static org.apache.hadoop.hive.metastore.txn.TxnStore.POOL_MUTEX; +import static org.springframework.transaction.TransactionDefinition.PROPAGATION_REQUIRED; + +public class TxnStoreMutex implements TxnStore.MutexAPI { + + private static final Logger LOG = LoggerFactory.getLogger(TxnStoreMutex.class); + /** + * must be static since even in UT there may be > 1 instance of TxnHandler + * (e.g. via Compactor services) + */ + private final static ConcurrentHashMap derbyKey2Lock = new ConcurrentHashMap<>(); + + + private final SQLGenerator sqlGenerator; + private final MultiDataSourceJdbcResource jdbcResource; + + public TxnStoreMutex(SQLGenerator sqlGenerator, MultiDataSourceJdbcResource jdbcResource) { + this.sqlGenerator = sqlGenerator; + this.jdbcResource = jdbcResource; + } + + @Override + public LockHandle acquireLock(String key) throws MetaException { + /** + * The implementation here is a bit kludgey but done so that code exercised by unit tests + * (which run against Derby which has no support for select for update) is as similar to + * production code as possible. + * In particular, with Derby we always run in a single process with a single metastore and + * the absence of For Update is handled via a Semaphore. The later would strictly speaking + * make the SQL statements below unnecessary (for Derby), but then they would not be tested. + */ + TransactionContext context = null; + try { + jdbcResource.bindDataSource(POOL_MUTEX); + context = jdbcResource.getTransactionManager().getNewTransaction(PROPAGATION_REQUIRED); + + MapSqlParameterSource paramSource = new MapSqlParameterSource().addValue("key", key); + String sqlStmt = sqlGenerator.addForUpdateClause("SELECT \"MT_COMMENT\", \"MT_KEY2\" FROM \"AUX_TABLE\" WHERE \"MT_KEY1\" = :key"); + + LOG.debug("About to execute SQL: {}", sqlStmt); + + Long lastUpdateTime = jdbcResource.getJdbcTemplate().query(sqlStmt, paramSource, rs -> rs.next() ? rs.getLong("MT_KEY2") : null); + if (lastUpdateTime == null) { + try { + jdbcResource.getJdbcTemplate().update("INSERT INTO \"AUX_TABLE\" (\"MT_KEY1\", \"MT_KEY2\") VALUES(:key, 0)", paramSource); + context.createSavepoint(); + } catch (DataAccessException e) { + if (!jdbcResource.getDatabaseProduct().isDuplicateKeyError(e)) { + throw new RuntimeException("Unable to lock " + key + " due to: " + SqlRetryHandler.getMessage(e), e); + } + //if here, it means a concrurrent acquireLock() inserted the 'key' + + //rollback is done for the benefit of Postgres which throws (SQLState=25P02, ErrorCode=0) if + //you attempt any stmt in a txn which had an error. + try { + jdbcResource.getConnection().rollback(); + } catch (SQLException ex) { + throw new MetaException("Unable to lock " + key + " due to: " + SqlRetryHandler.getMessage(ex) + "; " + StringUtils.stringifyException(ex)); + } + } + lastUpdateTime = jdbcResource.getJdbcTemplate().query(sqlStmt, paramSource, rs -> rs.next() ? rs.getLong("MT_KEY2") : null); + if (lastUpdateTime ==null) { + throw new IllegalStateException("Unable to lock " + key + ". Expected row in AUX_TABLE is missing."); + } + } + Semaphore derbySemaphore = null; + if (jdbcResource.getDatabaseProduct().isDERBY()) { + derbyKey2Lock.putIfAbsent(key, new Semaphore(1)); + derbySemaphore = derbyKey2Lock.get(key); + derbySemaphore.acquire(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("{} locked by {}", key, JavaUtils.hostname()); + } + //OK, so now we have a lock + return new LockHandleImpl(jdbcResource, context, key, lastUpdateTime, derbySemaphore); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + if (context != null) { + jdbcResource.getTransactionManager().rollback(context); + } + throw new MetaException("Unable to lock " + key + " due to: " + ex.getMessage() + StringUtils.stringifyException(ex)); + } catch (Throwable e) { + if (context != null) { + jdbcResource.getTransactionManager().rollback(context); + } + throw e; + } finally { + jdbcResource.unbindDataSource(); + } + } + + @Override + public void acquireLock(String key, LockHandle handle) throws MetaException { + //the idea is that this will use LockHandle.dbConn + throw new NotImplementedException("acquireLock(String, LockHandle) is not implemented"); + } + + public static final class LockHandleImpl implements LockHandle { + + private static final Logger LOG = LoggerFactory.getLogger(LockHandleImpl.class); + private static final String HOSTNAME = JavaUtils.hostname(); + + private final MultiDataSourceJdbcResource jdbcResource; + private final TransactionContext context; + private final Semaphore derbySemaphore; + private final String key; + private final Long lastUpdateTime; + + public LockHandleImpl(MultiDataSourceJdbcResource jdbcResource, TransactionContext context, String key, + Long lastUpdateTime, Semaphore derbySemaphore) { + assert derbySemaphore == null || derbySemaphore.availablePermits() == 0 : "Expected locked Semaphore"; + + this.jdbcResource = jdbcResource; + this.context = context; + this.derbySemaphore = derbySemaphore; + this.key = key; + this.lastUpdateTime = lastUpdateTime == null ? -1L : lastUpdateTime; + } + + @Override + public void releaseLocks() { + try { + jdbcResource.bindDataSource(POOL_MUTEX); + jdbcResource.getTransactionManager().rollback(context); + if (derbySemaphore != null) { + derbySemaphore.release(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("{} unlocked by {}", key, HOSTNAME); + } + } finally { + jdbcResource.unbindDataSource(); + } + } + + @Override + public Long getLastUpdateTime() { + return lastUpdateTime; + } + + @Override + public void releaseLocks(Long timestamp) { + try { + jdbcResource.bindDataSource(POOL_MUTEX); + try { + jdbcResource.getJdbcTemplate().update("UPDATE \"AUX_TABLE\" SET \"MT_KEY2\" = :time WHERE \"MT_KEY1\"= :key", + new MapSqlParameterSource() + .addValue("time", timestamp) + .addValue("key", key)); + jdbcResource.getTransactionManager().commit(context); + } catch (DataAccessException ex) { + LOG.warn("Unable to update MT_KEY2 value for MT_KEY1=" + key, ex); + } + + if (derbySemaphore != null) { + derbySemaphore.release(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("{} unlocked by {}", key, HOSTNAME); + } + } finally { + jdbcResource.unbindDataSource(); + } + } + + @Override + public void close() { + releaseLocks(); + } + + } +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java index 35a8ab71cc8e..e7e97b5f23de 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hive.metastore.txn; +import com.google.common.base.Splitter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -28,18 +29,20 @@ import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; +import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus; +import org.apache.hadoop.hive.metastore.utils.FileUtils; import org.apache.hadoop.hive.metastore.utils.JavaUtils; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; -import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate; import java.io.IOException; import java.security.PrivilegedExceptionAction; @@ -50,10 +53,9 @@ import java.util.Arrays; import java.util.BitSet; import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.regex.Pattern; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_TABLE; @@ -147,7 +149,7 @@ public static TxnStore getTxnStore(Configuration conf) { try { TxnStore handler = JavaUtils.getClass(className, TxnStore.class).newInstance(); handler.setConf(conf); - handler = ProxyTxnHandler.getProxy(handler, handler.getRetryHandler(), handler.getJdbcResourceHolder()); + handler = TransactionalRetryProxy.getProxy(handler.getRetryHandler(), handler.getJdbcResourceHolder(), handler); return handler; } catch (Exception e) { LOG.error("Unable to instantiate raw store directly in fastpath mode", e); @@ -410,50 +412,6 @@ public static List buildQueryWithINClauseStrings(Configuration conf, Li return ret; } - /** - * Executes the statement with an IN clause. If the number of elements or the length of the constructed statement would be - * too big, the IN clause will be split into multiple smaller ranges, and the statement will be executed multiple times. - * @param conf Hive configuration used to get the query and IN clause length limits. - * @param jdbcTemplate The {@link NamedParameterJdbcTemplate} instance to used for statement execution. - * @param query The query with the IN clause - * @param params A {@link MapSqlParameterSource} instance with the parameters of the query - * @param inClauseParamName The name of the parameter representing the content of the IN clause - * @param elements A {@link List} containing the elements to put in the IN clause - * @param comparator A {@link Comparator} instance used to find the longest element in the list. Used to - * estimate the length of the query. - * @return Returns the total number of affected rows. - * @param Type of the elements in the list. - */ - public static int executeStatementWithInClause(Configuration conf, NamedParameterJdbcTemplate jdbcTemplate, - String query, MapSqlParameterSource params, String inClauseParamName, - List elements, Comparator comparator) { - if (elements.size() == 0) { - throw new IllegalArgumentException("The elements list cannot be empty! An empty IN clause is invalid!"); - } - if (!Pattern.compile("IN\\s*\\(\\s*:" + inClauseParamName + "\\s*\\)", Pattern.CASE_INSENSITIVE).matcher(query).find()) { - throw new IllegalArgumentException("The query must contain the IN(:" + inClauseParamName + ") clause!"); - } - - int maxQueryLength = MetastoreConf.getIntVar(conf, ConfVars.DIRECT_SQL_MAX_QUERY_LENGTH) * 1024; - int batchSize = MetastoreConf.getIntVar(conf, ConfVars.DIRECT_SQL_MAX_ELEMENTS_IN_CLAUSE); - // The length of a single element is the string length of the longest element + 2 characters (comma, space) - int elementLength = elements.stream().max(comparator).get().toString().length() + 2; - // estimated base query size: query size + the length of all parameters. - int baseQuerySize = query.length() + params.getValues().values().stream().mapToInt(s -> s.toString().length()).sum(); - int maxElementsByLength = (maxQueryLength - baseQuerySize) / elementLength; - - int inClauseMaxSize = Math.min(batchSize, maxElementsByLength); - - int fromIndex = 0, totalCount = 0; - while (fromIndex < elements.size()) { - int endIndex = Math.min(elements.size(), fromIndex + inClauseMaxSize); - params.addValue(inClauseParamName, elements.subList(fromIndex, endIndex)); - totalCount += jdbcTemplate.update(query, params); - fromIndex = endIndex; - } - return totalCount; - } - /** * Compute and return the size of a query statement with the given parameters as input variables. * @@ -666,4 +624,60 @@ public static Character thriftCompactionType2DbType(CompactionType ct) throws Me public static String nvl(String input) { return input != null ? " = ? " : " IS NULL "; } + + public static String normalizePartitionCase(String s) { + if (s == null) { + return null; + } + Map map = Splitter.on(Path.SEPARATOR).withKeyValueSeparator('=').split(s); + return FileUtils.makePartName(new ArrayList<>(map.keySet()), new ArrayList<>(map.values())); + } + + @SuppressWarnings("squid:S2245") + public static long generateTemporaryId() { + return -1 * ThreadLocalRandom.current().nextLong(); + } + + public static boolean isValidTxn(long txnId) { + return txnId != 0; + } + + /** + * Used to raise an informative error when the caller expected a txn in a particular TxnStatus + * but found it in some other status + */ + public static void raiseTxnUnexpectedState(TxnStatus actualStatus, long txnid) + throws NoSuchTxnException, TxnAbortedException { + switch (actualStatus) { + case ABORTED: + throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) + " already aborted"); + case COMMITTED: + throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid) + " is already committed."); + case UNKNOWN: + throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid)); + case OPEN: + throw new NoSuchTxnException(JavaUtils.txnIdToString(txnid) + " is " + TxnStatus.OPEN); + default: + throw new IllegalArgumentException("Unknown TxnStatus " + actualStatus); + } + } + + /** + * Checks is the passed exception, or any of the root (cause) exceptions are an instance of {@link SQLException}. + * Returns with the found {@link SQLException} or throws an {@link IllegalArgumentException} if no {@link SQLException} + * found in the chain. + * @param ex The exception to check + * @return Returns with the {@link SQLException} found in the exception chain. + * @throws IllegalArgumentException Thrown if there is no {@link SQLException} in the exception chain + */ + public static SQLException getSqlException(Throwable ex) throws IllegalArgumentException { + while (ex != null && !(ex instanceof SQLException) ) { + ex = ex.getCause(); + } + if (ex == null) { + throw new IllegalArgumentException("No SQLException found in the exception chain!"); + } + return (SQLException)ex; + } + } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionInfo.java similarity index 98% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionInfo.java index 40b32482cf93..ca8dff5b8f22 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionInfo.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.hadoop.hive.common.ValidCompactorWriteIdList; @@ -24,6 +24,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.OptionalCompactionInfoStruct; import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.metastore.utils.StringableMap; import java.sql.PreparedStatement; @@ -208,7 +209,7 @@ public boolean equals(Object obj) { * @param rs ResultSet after call to rs.next() * @throws SQLException */ - static CompactionInfo loadFullFromCompactionQueue(ResultSet rs) throws SQLException, MetaException { + public static CompactionInfo loadFullFromCompactionQueue(ResultSet rs) throws SQLException { CompactionInfo fullCi = new CompactionInfo(); fullCi.id = rs.getLong(1); fullCi.dbname = rs.getString(2); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsData.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionMetricsData.java similarity index 98% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsData.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionMetricsData.java index b56009563b47..33dd362307d4 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsData.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionMetricsData.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; public class CompactionMetricsData { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionState.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionState.java similarity index 81% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionState.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionState.java index b0b2d0cd66ea..fdafd3fe259d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionState.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import java.util.Arrays; import java.util.Map; @@ -28,22 +28,24 @@ * These are the valid values for Compaction states. */ public enum CompactionState { - INITIATED('i'), - WORKING('w'), - READY_FOR_CLEANING('r'), - FAILED('f'), - SUCCEEDED('s'), - DID_NOT_INITIATE('a'), - REFUSED('c'), - ABORTED('x'); + INITIATED('i', "initiated"), + WORKING('w', "working"), + READY_FOR_CLEANING('r', "ready for cleaning"), + FAILED('f', "failed"), + SUCCEEDED('s', "succeeded"), + DID_NOT_INITIATE('a', "did not initiate"), + REFUSED('c', "refused"), + ABORTED('x', "aborted"); + private final String message; private final char sqlConst; private static final Map LOOKUP = Arrays.stream(CompactionState.values()).collect(toMap(CompactionState::getSqlConst, identity())); - CompactionState(char sqlConst) { + CompactionState(char sqlConst, String message) { this.sqlConst = sqlConst; + this.message = message; } @Override diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java new file mode 100644 index 000000000000..a7550b6fed78 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn.entities; + +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; +import org.apache.hadoop.hive.metastore.utils.JavaUtils; +import org.apache.hadoop.hive.metastore.utils.LockTypeUtil; + +import java.sql.ResultSet; +import java.sql.SQLException; + +public class LockInfo { + + public static final char LOCK_ACQUIRED = 'a'; + public static final char LOCK_WAITING = 'w'; + + + private final long extLockId; + private final long intLockId; + //0 means there is no transaction, i.e. it a select statement which is not part of + //explicit transaction or a IUD statement that is not writing to ACID table + private final long txnId; + private final String db; + private final String table; + private final String partition; + private final LockState state; + private final LockType type; + + // Assumes the result set is set to a valid row + public LockInfo(ResultSet rs) throws SQLException, MetaException { + extLockId = rs.getLong("HL_LOCK_EXT_ID"); // can't be null + intLockId = rs.getLong("HL_LOCK_INT_ID"); // can't be null + db = rs.getString("HL_DB"); // can't be null + String t = rs.getString("HL_TABLE"); + table = (rs.wasNull() ? null : t); + String p = rs.getString("HL_PARTITION"); + partition = (rs.wasNull() ? null : p); + switch (rs.getString("HL_LOCK_STATE").charAt(0)) { + case LOCK_WAITING: state = LockState.WAITING; break; + case LOCK_ACQUIRED: state = LockState.ACQUIRED; break; + default: + throw new MetaException("Unknown lock state " + rs.getString("HL_LOCK_STATE").charAt(0)); + } + char lockChar = rs.getString("HL_LOCK_TYPE").charAt(0); + type = LockTypeUtil.getLockTypeFromEncoding(lockChar) + .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar)); + txnId = rs.getLong("HL_TXNID"); //returns 0 if value is NULL + } + + public LockInfo(ShowLocksResponseElement e) { + extLockId = e.getLockid(); + intLockId = e.getLockIdInternal(); + txnId = e.getTxnid(); + db = e.getDbname(); + table = e.getTablename(); + partition = e.getPartname(); + state = e.getState(); + type = e.getType(); + } + + public long getExtLockId() { + return extLockId; + } + + public long getIntLockId() { + return intLockId; + } + + public long getTxnId() { + return txnId; + } + + public String getDb() { + return db; + } + + public String getTable() { + return table; + } + + public String getPartition() { + return partition; + } + + public LockState getState() { + return state; + } + + public LockType getType() { + return type; + } + + public boolean equals(Object other) { + if (!(other instanceof LockInfo)) return false; + LockInfo o = (LockInfo)other; + // Lock ids are unique across the system. + return extLockId == o.extLockId && intLockId == o.intLockId; + } + + @Override + public int hashCode() { + return new HashCodeBuilder() + .append(intLockId) + .append(extLockId) + .append(txnId) + .append(db) + .build(); + } + + @Override + public String toString() { + return JavaUtils.lockIdToString(extLockId) + " intLockId:" + + intLockId + " " + JavaUtils.txnIdToString(txnId) + + " db:" + db + " table:" + table + " partition:" + + partition + " state:" + (state == null ? "null" : state.toString()) + + " type:" + (type == null ? "null" : type.toString()); + } + private boolean isDbLock() { + return db != null && table == null && partition == null; + } + private boolean isTableLock() { + return db != null && table != null && partition == null; + } + private boolean isPartitionLock() { + return !(isDbLock() || isTableLock()); + } + +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/MetricsInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/MetricsInfo.java similarity index 98% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/MetricsInfo.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/MetricsInfo.java index 8fc7d8fed0cc..b76a5d1070ca 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/MetricsInfo.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/MetricsInfo.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import java.util.Set; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxn.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxn.java similarity index 84% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxn.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxn.java index 8ef5fa167f30..60044977b2fa 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxn.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import org.apache.hadoop.hive.metastore.api.TxnInfo; import org.apache.hadoop.hive.metastore.api.TxnType; @@ -25,12 +25,6 @@ */ public class OpenTxn { - public static final String OPEN_TXNS_QUERY = "SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\", " - + "(%s - \"TXN_STARTED\") FROM \"TXNS\" ORDER BY \"TXN_ID\""; - public static final String OPEN_TXNS_INFO_QUERY = "SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\", " - + "(%s - \"TXN_STARTED\"), \"TXN_USER\", \"TXN_HOST\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\" " - + "FROM \"TXNS\" ORDER BY \"TXN_ID\""; - private long txnId; private TxnStatus status; private TxnType type; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxnList.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxnList.java similarity index 89% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxnList.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxnList.java index 4d7244e4f24f..d102f7efeefe 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxnList.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxnList.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; @@ -27,15 +27,16 @@ import java.util.List; import static java.util.stream.Collectors.toList; -import static org.apache.hadoop.hive.metastore.txn.TxnStatus.ABORTED; -import static org.apache.hadoop.hive.metastore.txn.TxnStatus.OPEN; +import static org.apache.hadoop.hive.metastore.txn.entities.TxnStatus.ABORTED; +import static org.apache.hadoop.hive.metastore.txn.entities.TxnStatus.OPEN; /** * Class for the getOpenTxnList calculation. */ public class OpenTxnList { - private long hwm; - private List openTxnList; + + private final long hwm; + private final List openTxnList; public OpenTxnList(long hwm, List openTxnList) { this.hwm = hwm; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OperationType.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OperationType.java similarity index 97% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OperationType.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OperationType.java index 39cacd24e3eb..821ba9387522 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OperationType.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OperationType.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import org.apache.hadoop.hive.metastore.api.DataOperationType; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStatus.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/TxnStatus.java similarity index 97% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStatus.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/TxnStatus.java index a9ad560cc2cb..570c6ab71366 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStatus.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/TxnStatus.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.entities; import org.apache.hadoop.hive.metastore.api.TxnState; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ConditionalCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ConditionalCommand.java new file mode 100644 index 000000000000..75781f484623 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ConditionalCommand.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn.jdbc; + +import org.apache.hadoop.hive.metastore.DatabaseProduct; + +/** + * {@link ParameterizedCommand}. {@link ParameterizedBatchCommand}, and {@link InClauseBatchCommand} implementations can also + * implement this interface, marking that the execution is conditial, and the command wants to get notified about + * execution errors. Can be used to implement commands depending on some feature flag(s). + */ +public interface ConditionalCommand { + + /** + * Indicates if the command should be executed or not + * @param databaseProduct + * @return Returns true if the command can be executed, false otherwise. + */ + boolean shouldBeUsed(DatabaseProduct databaseProduct); + + /** + * Called in case of execution error in order to notify this command about the failure + * @param databaseProduct + * @param e The caught Exception + */ + void onError(DatabaseProduct databaseProduct, Exception e); + +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/InClauseBatchCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/InClauseBatchCommand.java new file mode 100644 index 000000000000..dc387e33fb53 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/InClauseBatchCommand.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn.jdbc; + +import org.apache.hadoop.hive.metastore.DatabaseProduct; +import org.springframework.jdbc.core.namedparam.SqlParameterSource; + +import java.util.Comparator; + +/** + * Represents a query with an IN() clause. The values inside the IN() clause are passed separately + * @param + */ +public class InClauseBatchCommand implements ParameterizedQuery { + + + private final String query; + private final SqlParameterSource queryParameters; + private final String inClauseParameterName; + private final Comparator parmeterLengthComparator; + + + public InClauseBatchCommand(String query, SqlParameterSource queryParameters, + String inClauseParameterName, Comparator parmeterLengthComparator) { + this.query = query; + this.queryParameters = queryParameters; + this.inClauseParameterName = inClauseParameterName; + this.parmeterLengthComparator = parmeterLengthComparator; + } + + /** + * The parameterized query string. The query must have exactly parameter inside the IN clause, and can have zero or + * more parameters everywhere else in the query string. + * @see ParameterizedQuery#getParameterizedQueryString(DatabaseProduct) + */ + public String getParameterizedQueryString(DatabaseProduct databaseProduct) { + return query; + } + + @Override + public SqlParameterSource getQueryParameters() { + return queryParameters; + } + + /** + * @return Returns with the name of the parameter which is inside the IN clause. + */ + public String getInClauseParameterName() { + return inClauseParameterName; + } + + /** + * @return Returns a {@link Comparator} instance which can be used to determine the longest element in the + * list IN clause parameters. This is required to be able to estimate the final legth of the command. + */ + public Comparator getParameterLengthComparator() { + return parmeterLengthComparator; + } + +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java index 16f44f2caa82..101172c74077 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java @@ -17,12 +17,19 @@ */ package org.apache.hadoop.hive.metastore.txn.jdbc; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.DatabaseProduct; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.txn.ContextNode; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.tools.SQLGenerator; +import org.apache.hadoop.hive.metastore.utils.StackThreadLocal; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter; import org.springframework.jdbc.core.ResultSetExtractor; +import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate; import org.springframework.jdbc.core.namedparam.SqlParameterSource; import org.springframework.jdbc.datasource.DataSourceTransactionManager; @@ -33,9 +40,13 @@ import javax.sql.DataSource; import java.sql.Connection; import java.sql.ResultSet; +import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.function.Function; +import java.util.regex.Pattern; /** * Holds multiple {@link DataSource}s as a single object and offers JDBC related resources. @@ -46,19 +57,23 @@ public class MultiDataSourceJdbcResource { private static final Logger LOG = LoggerFactory.getLogger(MultiDataSourceJdbcResource.class); - private final ThreadLocal> threadLocal = new ThreadLocal<>(); + private final StackThreadLocal threadLocal = new StackThreadLocal<>(); private final Map dataSources = new HashMap<>(); private final Map transactionManagers = new HashMap<>(); private final Map jdbcTemplates = new HashMap<>(); private final DatabaseProduct databaseProduct; + private final Configuration conf; + private final SQLGenerator sqlGenerator; /** * Creates a new instance of the {@link MultiDataSourceJdbcResource} class * @param databaseProduct A {@link DatabaseProduct} instance representing the type of the underlying HMS dabatabe. */ - public MultiDataSourceJdbcResource(DatabaseProduct databaseProduct) { + public MultiDataSourceJdbcResource(DatabaseProduct databaseProduct, Configuration conf, SQLGenerator sqlGenerator) { this.databaseProduct = databaseProduct; + this.conf = conf; + this.sqlGenerator = sqlGenerator; } /** @@ -81,11 +96,11 @@ public void registerDataSource(String dataSourceName, DataSource dataSource) { * @param dataSourceName The name of the {@link DataSource} bind to the current {@link Thread}. */ public void bindDataSource(String dataSourceName) { - threadLocal.set(new ContextNode<>(threadLocal.get(), dataSourceName)); + threadLocal.set(dataSourceName); } public void bindDataSource(Transactional transactional) { - threadLocal.set(new ContextNode<>(threadLocal.get(), transactional.value())); + threadLocal.set(transactional.value()); } /** @@ -96,12 +111,18 @@ public void bindDataSource(Transactional transactional) { * {@link DataSource} the JDBC resources should be returned. */ public void unbindDataSource() { - ContextNode node = threadLocal.get(); - if (node != null && node.getParent() != null) { - threadLocal.set(node.getParent()); - } else { - threadLocal.remove(); - } + threadLocal.unset(); + } + + /** + * @return Returns the {@link Configuration} object used to create this {@link MultiDataSourceJdbcResource} instance. + */ + public Configuration getConf() { + return conf; + } + + public SQLGenerator getSqlGenerator() { + return sqlGenerator; } /** @@ -141,15 +162,6 @@ public DatabaseProduct getDatabaseProduct() { return databaseProduct; } - private String getDataSourceName() { - ContextNode node = threadLocal.get(); - if (node == null) { - throw new IllegalStateException("In order to access the JDBC resources, first you need to obtain a transaction " + - "using getTransaction(int propagation, String dataSourceName)!"); - } - return node.getValue(); - } - /** * Executes a {@link NamedParameterJdbcTemplate#update(String, org.springframework.jdbc.core.namedparam.SqlParameterSource)} * calls using the query string and parameters obtained from {@link ParameterizedCommand#getParameterizedQueryString(DatabaseProduct)} and @@ -162,8 +174,118 @@ private String getDataSourceName() { * thrown if the update count was rejected by the {@link ParameterizedCommand#resultPolicy()} method */ public Integer execute(ParameterizedCommand command) throws MetaException { - return execute(command.getParameterizedQueryString(getDatabaseProduct()), - command.getQueryParameters(), command.resultPolicy()); + if (!shouldExecute(command)) { + return null; + } + try { + return execute(command.getParameterizedQueryString(getDatabaseProduct()), + command.getQueryParameters(), command.resultPolicy()); + } catch (Exception e) { + handleError(command, e); + throw e; + } + } + + /** + * Executes a {@link org.springframework.jdbc.core.JdbcTemplate#batchUpdate(String, Collection, int, ParameterizedPreparedStatementSetter)} + * call using the query string obtained from {@link ParameterizedBatchCommand#getParameterizedQueryString(DatabaseProduct)}, + * the parameters obtained from {@link ParameterizedBatchCommand#getQueryParameters()}, and the + * {@link org.springframework.jdbc.core.PreparedStatementSetter} obtained from + * {@link ParameterizedBatchCommand#getPreparedStatementSetter()} methods. The batchSize is coming fomr the + * {@link Configuration} object. After the execution, this method validates the resulted number of affected rows using the + * {@link ParameterizedBatchCommand#resultPolicy()} function for each element in the batch. + * + * @param command The {@link ParameterizedBatchCommand} to execute. + * @return Returns an integer array,containing the number of affected rows for each element in the batch. + */ + public int execute(ParameterizedBatchCommand command) throws MetaException { + if (!shouldExecute(command)) { + return 0; + } + try { + int maxBatchSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.JDBC_MAX_BATCH_SIZE); + int[][] result = getJdbcTemplate().getJdbcTemplate().batchUpdate( + command.getParameterizedQueryString(databaseProduct), + command.getQueryParameters(), + maxBatchSize, + command.getPreparedStatementSetter() + ); + + Function resultPolicy = command.resultPolicy(); + if (resultPolicy != null && !Arrays.stream(result).allMatch(inner -> Arrays.stream(inner).allMatch(resultPolicy::apply))) { + LOG.error("The update count was rejected in at least one of the result array. Rolling back."); + throw new MetaException("The update count was rejected in at least one of the result array. Rolling back."); + } + return Arrays.stream(result).reduce(0, (acc, i) -> acc + Arrays.stream(i).sum(), Integer::sum); + } catch (Exception e) { + handleError(command, e); + throw e; + } + } + + /** + * Executes the passed {@link InClauseBatchCommand}. It estimates the length of the query and if it exceeds the limit + * set in {@link org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars#DIRECT_SQL_MAX_QUERY_LENGTH}, or the + * number of elements in the IN() clause exceeds + * {@link org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars#DIRECT_SQL_MAX_ELEMENTS_IN_CLAUSE}, the query + * will be split to multiple queries. + * @param command The {@link InClauseBatchCommand} to execute + * @return Returns with the number of affected rows in total. + * @param The type of the elements in the IN() clause + * @throws MetaException If {@link InClauseBatchCommand#getInClauseParameterName()} is blank, or the value of the + * IN() clause parameter in {@link InClauseBatchCommand#getQueryParameters()} is not exist or not an instance of List<T> + */ + public int execute(InClauseBatchCommand command) throws MetaException { + if (!shouldExecute(command)) { + return -1; + } + + List elements; + try { + if (StringUtils.isBlank(command.getInClauseParameterName())) { + throw new MetaException("The IN() clause parameter name (InClauseBatchCommand.getInClauseParameterName() " + + "cannot be blank!"); + } + try { + //noinspection unchecked + elements = (List) command.getQueryParameters().getValue(command.getInClauseParameterName()); + } catch (ClassCastException e) { + throw new MetaException("The parameter " + command.getInClauseParameterName() + "must be of type List!"); + } + MapSqlParameterSource params = (MapSqlParameterSource) command.getQueryParameters(); + String query = command.getParameterizedQueryString(databaseProduct); + if (CollectionUtils.isEmpty(elements)) { + throw new IllegalArgumentException("The elements list cannot be null or empty! An empty IN clause is invalid!"); + } + if (!Pattern.compile("IN\\s*\\(\\s*:" + command.getInClauseParameterName() + "\\s*\\)", Pattern.CASE_INSENSITIVE).matcher(query).find()) { + throw new IllegalArgumentException("The query must contain the IN(:" + command.getInClauseParameterName() + ") clause!"); + } + + int maxQueryLength = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.DIRECT_SQL_MAX_QUERY_LENGTH) * 1024; + int batchSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.DIRECT_SQL_MAX_ELEMENTS_IN_CLAUSE); + // The length of a single element is the string length of the longest element + 2 characters (comma, space) + int elementLength = elements.isEmpty() ? 1 : elements + .stream() + .max(command.getParameterLengthComparator()) + .orElseThrow(IllegalStateException::new).toString().length() + 2; + // estimated base query size: query size + the length of all parameters. + int baseQueryLength = query.length(); + int maxElementsByLength = (maxQueryLength - baseQueryLength) / elementLength; + + int inClauseMaxSize = Math.min(batchSize, maxElementsByLength); + + int fromIndex = 0, totalCount = 0; + while (fromIndex < elements.size()) { + int endIndex = Math.min(elements.size(), fromIndex + inClauseMaxSize); + params.addValue(command.getInClauseParameterName(), elements.subList(fromIndex, endIndex)); + totalCount += getJdbcTemplate().update(query, params); + fromIndex = endIndex; + } + return totalCount; + } catch (Exception e) { + handleError(command, e); + throw e; + } } /** @@ -211,4 +333,18 @@ public Result execute(QueryHandler queryHandler) throws MetaExc } } + private String getDataSourceName() { + return threadLocal.get(); + } + + private boolean shouldExecute(Object command) { + return !(command instanceof ConditionalCommand) || ((ConditionalCommand)command).shouldBeUsed(databaseProduct); + } + + private void handleError(Object command, Exception e) { + if (command instanceof ConditionalCommand) { + ((ConditionalCommand)command).onError(databaseProduct, e); + } + } + } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/NoPoolConnectionPool.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/NoPoolConnectionPool.java new file mode 100644 index 000000000000..9b869be0f442 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/NoPoolConnectionPool.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn.jdbc; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.DatabaseProduct; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.sql.DataSource; +import java.io.PrintWriter; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Properties; + +public class NoPoolConnectionPool implements DataSource { + + private static final Logger LOG = LoggerFactory.getLogger(NoPoolConnectionPool.class); + + // Note that this depends on the fact that no-one in this class calls anything but + // getConnection. If you want to use any of the Logger or wrap calls you'll have to + // implement them. + private final Configuration conf; + private final DatabaseProduct dbProduct; + private Driver driver; + private String connString; + private String user; + private String passwd; + + public NoPoolConnectionPool(Configuration conf, DatabaseProduct dbProduct) { + this.conf = conf; + this.dbProduct = dbProduct; + } + + @Override + public Connection getConnection() throws SQLException { + if (user == null) { + user = DataSourceProvider.getMetastoreJdbcUser(conf); + passwd = DataSourceProvider.getMetastoreJdbcPasswd(conf); + } + return getConnection(user, passwd); + } + + @Override + public Connection getConnection(String username, String password) throws SQLException { + // Find the JDBC driver + if (driver == null) { + String driverName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_DRIVER); + if (driverName == null || driverName.equals("")) { + String msg = "JDBC driver for transaction db not set in configuration " + + "file, need to set " + MetastoreConf.ConfVars.CONNECTION_DRIVER.getVarname(); + LOG.error(msg); + throw new RuntimeException(msg); + } + try { + LOG.info("Going to load JDBC driver {}", driverName); + driver = (Driver) Class.forName(driverName).newInstance(); + } catch (InstantiationException e) { + throw new RuntimeException("Unable to instantiate driver " + driverName + ", " + + e.getMessage(), e); + } catch (IllegalAccessException e) { + throw new RuntimeException( + "Unable to access driver " + driverName + ", " + e.getMessage(), + e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to find driver " + driverName + ", " + e.getMessage(), + e); + } + connString = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY); + } + + try { + LOG.info("Connecting to transaction db with connection string {}", connString); + Properties connectionProps = new Properties(); + connectionProps.setProperty("user", username); + connectionProps.setProperty("password", password); + Connection conn = driver.connect(connString, connectionProps); + String prepareStmt = dbProduct != null ? dbProduct.getPrepareTxnStmt() : null; + if (prepareStmt != null) { + try (Statement stmt = conn.createStatement()) { + stmt.execute(prepareStmt); + } + } + conn.setAutoCommit(false); + return conn; + } catch (SQLException e) { + throw new RuntimeException("Unable to connect to transaction manager using " + connString + + ", " + e.getMessage(), e); + } + } + + @Override + public PrintWriter getLogWriter() { + throw new UnsupportedOperationException(); + } + + @Override + public void setLogWriter(PrintWriter out) { + throw new UnsupportedOperationException(); + } + + @Override + public void setLoginTimeout(int seconds) { + throw new UnsupportedOperationException(); + } + + @Override + public int getLoginTimeout() { + throw new UnsupportedOperationException(); + } + + @Override + public java.util.logging.Logger getParentLogger() { + throw new UnsupportedOperationException(); + } + + @Override + public T unwrap(Class iface) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isWrapperFor(Class iface) { + throw new UnsupportedOperationException(); + } +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ParameterizedBatchCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ParameterizedBatchCommand.java new file mode 100644 index 000000000000..9f2647cb9f79 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/ParameterizedBatchCommand.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn.jdbc; + +import org.apache.hadoop.hive.metastore.DatabaseProduct; +import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter; + +import java.util.List; +import java.util.function.Function; + +/** + * Represents a parameterized batch command (for exmaple an UPDATE statement) as a Spring + * {@link org.springframework.jdbc.core.JdbcTemplate} style parameterized query string + * (for example: UPDATE TBL SET COL1 = ? WHERE ID = ?), its parameters, and a result policy. The result policy + * is a Function<Integer, Boolean> function which must decide if the number of + * affected rows is acceptable or not. It is called for each result in the batch. + *

   
+ * Please note that for batch command, named parameters cannot be used! 
+ * (like UPDATE TBL SET COL1 = :value WHERE ID = :id)
+ */
+public interface ParameterizedBatchCommand {
+
+  /**
+   * The parameterized query string. It is allowed if the query has no parameters at all.
+   * @param databaseProduct A {@link DatabaseProduct} instance representing the type of the underlying HMS dabatabe.
+   * @return Returns the parameterized query string.
+   */
+  String getParameterizedQueryString(DatabaseProduct databaseProduct);
+
+  /**
+   * A {@link List} instance containing the required parameters for the query string.
+   */
+  List getQueryParameters();
+
+  /**
+   * Implementations must return a {@link ParameterizedPreparedStatementSetter} instance which will be 
+   * responsible for setting the parameter values for all the items in the batch 
+   */
+  ParameterizedPreparedStatementSetter getPreparedStatementSetter();
+
+  /**
+   * @return Returns the result policy to be used to validate the number of affected rows.
+   */
+  Function resultPolicy();  
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/RollbackException.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/RollbackException.java
new file mode 100644
index 000000000000..94474852878c
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/RollbackException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc;
+
+import java.lang.reflect.Method;
+
+/**
+ * This exception can be used to trigger rollback in 
+ * {@link org.apache.hadoop.hive.metastore.txn.TransactionalRetryProxy#invoke(Object, Method, Object[])}
+ * for the current transaction, without propagating the exception to the caller. The proxy will catch this exception,
+ * rollback the transaction (if not yet completed already) and return the value supplied in the constructor to te caller.
+ */
+public class RollbackException extends RuntimeException {  
+  
+  private final Object result;
+  
+  public RollbackException(Object result) {
+    this.result = result;
+  }
+
+  public Object getResult() {
+    return result;
+  }
+  
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContext.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContext.java
index ec5e32643aa2..c5350ffc1711 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContext.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContext.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hive.metastore.txn.jdbc;
 
+import org.springframework.lang.NonNull;
+import org.springframework.transaction.TransactionException;
 import org.springframework.transaction.TransactionStatus;
 
 /**
@@ -26,7 +28,7 @@
  * In other words: This wrapper automatically rolls back uncommitted transactions, but the commit
  * needs to be done manually using {@link TransactionContextManager#commit(TransactionContext)} method.
  */
-public class TransactionContext implements AutoCloseable {
+public class TransactionContext implements TransactionStatus, AutoCloseable {
 
   private final TransactionStatus transactionStatus;
   private final TransactionContextManager transactionManager;
@@ -36,15 +38,61 @@ public class TransactionContext implements AutoCloseable {
     this.transactionManager = transactionManager;
   }
 
+  @Override
+  public boolean hasSavepoint() {
+    return transactionStatus.hasSavepoint();
+  }
+
+  @Override
+  public void flush() {
+    transactionStatus.flush();
+  }
+
+  @NonNull
+  @Override
+  public Object createSavepoint() throws TransactionException {
+    return transactionStatus.createSavepoint();
+  }
+
+  @Override
+  public void rollbackToSavepoint(@NonNull Object savepoint) throws TransactionException {
+    transactionStatus.rollbackToSavepoint(savepoint);
+  }
+
+  @Override
+  public void releaseSavepoint(@NonNull Object savepoint) throws TransactionException {
+    transactionStatus.releaseSavepoint(savepoint);
+  }
+
+  @Override
+  public boolean isNewTransaction() {
+    return transactionStatus.isNewTransaction();
+  }
+
+  @Override
+  public void setRollbackOnly() {
+    transactionStatus.setRollbackOnly();
+  }
+
+  @Override
+  public boolean isRollbackOnly() {
+    return transactionStatus.isRollbackOnly();
+  }
+
+  @Override
+  public boolean isCompleted() {
+    return transactionStatus.isCompleted();
+  }
+
   /**
    * @return Returns the {@link TransactionStatus} instance wrapped by this object.
    */
-  public TransactionStatus getTransactionStatus() {
+  TransactionStatus getTransactionStatus() {
     return transactionStatus;
   }
 
   /**
-   * @see TransactionContext TransactionWrapper class level javadoc.
+   * @see TransactionContext class level javadoc.
    */
   @Override
   public void close() {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContextManager.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContextManager.java
index f91e57e0e604..5be42248fe73 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContextManager.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionContextManager.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hive.metastore.txn.jdbc;
 
+import org.apache.hadoop.hive.metastore.utils.StackThreadLocal;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.springframework.transaction.PlatformTransactionManager;
 import org.springframework.transaction.TransactionStatus;
 import org.springframework.transaction.support.DefaultTransactionDefinition;
@@ -29,8 +32,19 @@
  */
 public class TransactionContextManager {
 
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionContextManager.class);
+  
   private final PlatformTransactionManager realTransactionManager;
 
+  /**
+   * We must keep track of the requested transactions, to be able to return the current transaction in {@link #getActiveTransaction()}.
+   * In Spring JDBCTemplate users have to choose programmatic transaction management to access the {@link TransactionStatus}
+   * object which can be used for savepoint management. With this enhancement, it is possible to identify and return
+   * the active transaction, which allows combining the use of {@link org.springframework.transaction.annotation.Transactional} 
+   * annotation with programmatic savepoint management.
+   */
+  private final StackThreadLocal contexts = new StackThreadLocal<>();
+
   TransactionContextManager(PlatformTransactionManager realTransactionManager) {
     this.realTransactionManager = realTransactionManager;
   }
@@ -38,26 +52,50 @@ public class TransactionContextManager {
   /**
    * Begins a new transaction or returns an existing, depending on the passed Transaction Propagation.
    * The created transaction is wrapped into a {@link TransactionContext} which is {@link AutoCloseable} and allows using
-   * the wrapper inside a try-with-resources block. 
+   * the wrapper inside a try-with-resources block.
+   *
    * @param propagation The transaction propagation to use.
    */
-  public TransactionContext getTransaction(int propagation) {
-      return new TransactionContext(realTransactionManager.getTransaction(new DefaultTransactionDefinition(propagation)), this);
+  public TransactionContext getNewTransaction(int propagation) {
+    TransactionContext context = new TransactionContext(realTransactionManager.getTransaction(
+        new DefaultTransactionDefinition(propagation)), this);
+    contexts.set(context);
+    return context;
   }
-
+  
+  public TransactionContext getActiveTransaction() {
+    return contexts.get();
+  }
+  
   public void commit(TransactionContext context) {
-      realTransactionManager.commit(context.getTransactionStatus());
+    TransactionContext storedContext = contexts.get();
+    if (!storedContext.equals(context)) {
+      throw new IllegalStateException();
+    }
+    try {
+      realTransactionManager.commit(context.getTransactionStatus());      
+    } finally {
+      contexts.unset();
+    }
   }
 
   public void rollback(TransactionContext context) {
+    TransactionContext storedContext = contexts.get();
+    if (!storedContext.equals(context)) {
+      throw new IllegalStateException();
+    }
+    try {
       realTransactionManager.rollback(context.getTransactionStatus());
+    } finally {
+      contexts.unset();
+    }
   }
 
   void rollbackIfNotCommitted(TransactionContext context) {
-    TransactionStatus status = context.getTransactionStatus();
-      if (!status.isCompleted()) {
-        realTransactionManager.rollback(status);
-      }
+    if (!context.isCompleted()) {
+      LOG.debug("The transaction is not committed and we are leaving the try-with-resources block. Going to rollback: {}", context);
+      rollback(context);
+    }
   }
 
-}
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionalFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionalFunction.java
index 7917039245c1..5f38ee961ce5 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionalFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/TransactionalFunction.java
@@ -18,6 +18,9 @@
 package org.apache.hadoop.hive.metastore.txn.jdbc;
 
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 
 /**
  * A functional interface representing a function call (typically a query or statement) which has a result and done within
@@ -26,7 +29,7 @@
  */
 @FunctionalInterface
 public interface TransactionalFunction {
-
+  
   /**
    * Implementations typically should execute transsactional database calls inside.
    * @param jdbcResource A {@link MultiDataSourceJdbcResource} instance responsible for providing all the necessary resources 
@@ -35,6 +38,7 @@ public interface TransactionalFunction {
    * @throws org.springframework.dao.DataAccessException Thrown if any of the JDBC calls fail
    * @throws MetaException Thrown in case of application error within the function
    */
-  Result execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException;
+  Result execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, NoSuchTxnException, TxnAbortedException, 
+      NoSuchLockException;
 
 }
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToMinHistoryCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToMinHistoryCommand.java
new file mode 100644
index 000000000000..dfe87bd34085
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToMinHistoryCommand.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ConditionalCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedBatchCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+public class AddWriteIdsToMinHistoryCommand implements ParameterizedBatchCommand, ConditionalCommand {
+
+  private static final String MIN_HISTORY_WRITE_ID_INSERT_QUERY = "INSERT INTO \"MIN_HISTORY_WRITE_ID\" (\"MH_TXNID\", " +
+      "\"MH_DATABASE\", \"MH_TABLE\", \"MH_WRITEID\") VALUES (?, ?, ?, ?)";
+  
+  private final List params;
+
+  public AddWriteIdsToMinHistoryCommand(long txnId, Map minOpenWriteIds) {
+    this.params = new ArrayList<>();
+    for (Map.Entry validWriteId : minOpenWriteIds.entrySet()) {
+      String[] names = TxnUtils.getDbTableName(validWriteId.getKey());
+      params.add(new Object[]{ txnId, names[0], names[1], validWriteId.getValue() });
+    }
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) {
+    return MIN_HISTORY_WRITE_ID_INSERT_QUERY;
+  }
+
+  @Override
+  public List getQueryParameters() {
+    return params;
+  }
+
+  @Override
+  public ParameterizedPreparedStatementSetter getPreparedStatementSetter() {
+    return (ps, argument) -> {
+      ps.setLong(1, (Long)argument[0]);
+      ps.setString(2, argument[1].toString());
+      ps.setString(3, argument[2].toString());
+      ps.setLong(4, (Long)argument[3]);
+    };
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return ParameterizedCommand.EXACTLY_ONE_ROW;
+  }
+
+  @Override
+  public boolean shouldBeUsed(DatabaseProduct databaseProduct) {
+    return TxnHandler.ConfVars.useMinHistoryWriteId();
+  }
+
+  @Override
+  public void onError(DatabaseProduct databaseProduct, Exception e) {
+    if (databaseProduct.isTableNotExistsError(e)) {
+      // If the table does not exists anymore, we disable the flag and start to work the new way
+      // This enables to switch to the new functionality without a restart
+      TxnHandler.ConfVars.setUseMinHistoryWriteId(false);
+    }
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToTxnToWriteIdCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToTxnToWriteIdCommand.java
new file mode 100644
index 000000000000..eaa6d1e5d9be
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/AddWriteIdsToTxnToWriteIdCommand.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedBatchCommand;
+import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+public class AddWriteIdsToTxnToWriteIdCommand implements ParameterizedBatchCommand {
+
+  private final List params;
+
+  public AddWriteIdsToTxnToWriteIdCommand(String dbName, String tableName, long writeId, List txnIds, List txnToWriteIds) {
+    this.params = new ArrayList<>();
+    for (long txnId : txnIds) {
+      params.add(new Object[]{ txnId, dbName, tableName, writeId });
+      txnToWriteIds.add(new TxnToWriteId(txnId, writeId));
+      writeId++;
+    }
+  }
+
+  
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) {
+    return "INSERT INTO \"TXN_TO_WRITE_ID\" (\"T2W_TXNID\",  \"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\") VALUES (?, ?, ?, ?)";
+  }
+
+  @Override
+  public List getQueryParameters() {
+    return params;
+  }
+
+  @Override
+  public ParameterizedPreparedStatementSetter getPreparedStatementSetter() {
+    return (ps, argument) -> {
+      ps.setLong(1, (Long)argument[0]);
+      ps.setString(2, argument[1].toString());
+      ps.setString(3, argument[2].toString());
+      ps.setLong(4, (Long)argument[3]);
+    };
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return null;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteInvalidOpenTxnsCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteInvalidOpenTxnsCommand.java
new file mode 100644
index 000000000000..3a218ca9e51d
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteInvalidOpenTxnsCommand.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.txn.jdbc.InClauseBatchCommand;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.List;
+
+public class DeleteInvalidOpenTxnsCommand extends InClauseBatchCommand {
+
+  public DeleteInvalidOpenTxnsCommand(List txnids) {
+    super("DELETE FROM \"TXNS\" WHERE \"TXN_ID\" IN (:txnIds)", 
+        new MapSqlParameterSource().addValue("txnIds", txnids),
+        "txnIds", Long::compareTo);
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteReplTxnMapEntryCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteReplTxnMapEntryCommand.java
new file mode 100644
index 000000000000..556a509b1f05
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/DeleteReplTxnMapEntryCommand.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.util.function.Function;
+
+public class DeleteReplTxnMapEntryCommand implements ParameterizedCommand {
+    
+  private final long sourceTxnId;
+  private final String replicationPolicy;
+
+  public DeleteReplTxnMapEntryCommand(long sourceTxnId, String replicationPolicy) {
+    this.sourceTxnId = sourceTxnId;
+    this.replicationPolicy = replicationPolicy;
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return null;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "DELETE FROM \"REPL_TXN_MAP\" WHERE \"RTM_SRC_TXN_ID\" = :sourceTxnId AND \"RTM_REPL_POLICY\" = :replPolicy";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("sourceTxnId", sourceTxnId)
+        .addValue("replPolicy", replicationPolicy);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/InsertCompactionInfoCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionInfoCommand.java
similarity index 96%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/InsertCompactionInfoCommand.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionInfoCommand.java
index da0e2126a3f7..4af24c5d4a50 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/InsertCompactionInfoCommand.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionInfoCommand.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
@@ -34,7 +34,7 @@ public class InsertCompactionInfoCommand implements ParameterizedCommand {
 
   private final long compactionEndTime;
 
-  // language=PostgreSQL
+  //language=SQL
   private static final String INSERT =
       "INSERT INTO \"COMPLETED_COMPACTIONS\" " +
           "   (\"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", " +
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionRequestCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionRequestCommand.java
new file mode 100644
index 000000000000..7e2d9bf52b5c
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompactionRequestCommand.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.apache.hadoop.hive.metastore.utils.StringableMap;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.Types;
+import java.util.function.Function;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class InsertCompactionRequestCommand implements ParameterizedCommand {
+  
+  private final long id;
+  private final CompactionState compactionState;
+  private final CompactionRequest rqst;
+  private Long highestWriteId = null;
+  private Long txnId = null;
+
+  public InsertCompactionRequestCommand(long id, CompactionState compactionState, CompactionRequest rqst) {
+    this.id = id;
+    this.compactionState = compactionState;
+    this.rqst = rqst;
+  }
+  
+  public InsertCompactionRequestCommand withTxnDetails(long highestWriteId, long txnId) {
+    this.highestWriteId = highestWriteId;
+    this.txnId = txnId;
+    return this;
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return ParameterizedCommand.EXACTLY_ONE_ROW;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", \"CQ_STATE\", " +
+        "\"CQ_TYPE\", \"CQ_POOL_NAME\", \"CQ_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\", \"CQ_TBLPROPERTIES\", \"CQ_RUN_AS\", " +
+        "\"CQ_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TXN_ID\", \"CQ_ENQUEUE_TIME\") " +
+        "VALUES(:id, :dbName, :tableName, :partition, :state, :type, :poolName, :buckets, :orderBy, :tblProperties, " +
+        ":runAs, :initiatorId, :initiatorVersion, :highestWriteId, :txnId, " + getEpochFn(databaseProduct) + ")";
+  }
+  
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    try {
+      return new MapSqlParameterSource()
+          .addValue("id", id)
+          .addValue("dbName", rqst.getDbname(), Types.VARCHAR)
+          .addValue("tableName", rqst.getTablename(), Types.VARCHAR)
+          .addValue("partition", rqst.getPartitionname(), Types.VARCHAR)
+          .addValue("state", compactionState.getSqlConst(), Types.VARCHAR)
+          .addValue("type", TxnUtils.thriftCompactionType2DbType(rqst.getType()), Types.VARCHAR)
+          .addValue("poolName", rqst.getPoolName(), Types.VARCHAR)
+          .addValue("buckets", rqst.isSetNumberOfBuckets() ? rqst.getNumberOfBuckets() : null, Types.INTEGER)
+          .addValue("orderBy", rqst.getOrderByClause(), Types.VARCHAR)
+          .addValue("tblProperties", rqst.getProperties() == null ? null : new StringableMap(rqst.getProperties()), Types.VARCHAR)
+          .addValue("runAs", rqst.getRunas(), Types.VARCHAR)
+          .addValue("initiatorId", rqst.getInitiatorId(), Types.VARCHAR)
+          .addValue("initiatorVersion", rqst.getInitiatorVersion(), Types.VARCHAR)
+          .addValue("highestWriteId", highestWriteId, Types.BIGINT)
+          .addValue("txnId", txnId, Types.BIGINT);
+    } catch (MetaException e) {
+      throw new MetaWrapperException(e);
+    }
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompletedTxnComponentsCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompletedTxnComponentsCommand.java
new file mode 100644
index 000000000000..9f9086b755db
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertCompletedTxnComponentsCommand.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedBatchCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter;
+
+import java.util.List;
+import java.util.function.Function;
+
+public class InsertCompletedTxnComponentsCommand implements ParameterizedBatchCommand {
+
+  private final long txnId;
+  private final char isUpdateDelete;
+  private final List infos;
+
+  public InsertCompletedTxnComponentsCommand(long txnId, char isUpdateDelete, List infos) {
+    this.txnId = txnId;
+    this.isUpdateDelete = isUpdateDelete;
+    this.infos = infos;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) {
+    return
+        "INSERT INTO \"COMPLETED_TXN_COMPONENTS\" " +
+        "(\"CTC_TXNID\", \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", \"CTC_WRITEID\", \"CTC_UPDATE_DELETE\") " +
+        "VALUES (?, ?, ?, ?, ?, ?)";
+  }
+
+  @Override
+  public List getQueryParameters() {
+    return infos;
+  }
+
+  @Override
+  public ParameterizedPreparedStatementSetter getPreparedStatementSetter() {
+    return (ps, argument) -> {
+      ps.setLong(1, txnId);
+      ps.setString(2, argument.getDatabase());
+      ps.setString(3, argument.getTable());
+      ps.setString(4, argument.getPartition());
+      ps.setLong(5, argument.getWriteId());
+      ps.setString(6, Character.toString(isUpdateDelete));
+    };
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return ParameterizedCommand.EXACTLY_ONE_ROW;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertHiveLocksCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertHiveLocksCommand.java
new file mode 100644
index 000000000000..6ef5465616d0
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertHiveLocksCommand.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedBatchCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.apache.hadoop.hive.metastore.utils.LockTypeUtil;
+import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+import static org.apache.hadoop.hive.metastore.txn.entities.LockInfo.LOCK_WAITING;
+
+public class InsertHiveLocksCommand implements ParameterizedBatchCommand {
+  
+  private final LockRequest lockRequest;
+  private final long tempExtLockId;
+
+  public InsertHiveLocksCommand(LockRequest lockRequest, long tempExtLockId) {
+    this.lockRequest = lockRequest;
+    this.tempExtLockId = tempExtLockId;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) {
+    try {
+      //language=SQL
+      return String.format( 
+          "INSERT INTO \"HIVE_LOCKS\" ( " +
+          "\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", " +
+          "\"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", \"HL_USER\", \"HL_HOST\", \"HL_AGENT_INFO\") " +
+          "VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s, ?, ?, ?)", lockRequest.getTxnid() != 0 ? "0" : getEpochFn(databaseProduct));
+    } catch (MetaException e) {
+      throw new MetaWrapperException(e);
+    }
+  }
+
+  @Override
+  public List getQueryParameters() {
+    List params = new ArrayList<>(lockRequest.getComponentSize());
+    long intLockId = 0;
+    for (LockComponent lc : lockRequest.getComponent()) {
+      String lockType = LockTypeUtil.getEncodingAsStr(lc.getType());
+      params.add(new Object[] {tempExtLockId, ++intLockId, lockRequest.getTxnid(), StringUtils.lowerCase(lc.getDbname()),
+          StringUtils.lowerCase(lc.getTablename()), TxnUtils.normalizePartitionCase(lc.getPartitionname()),
+          Character.toString(LOCK_WAITING), lockType, lockRequest.getUser(), lockRequest.getHostname(), lockRequest.getAgentInfo()});
+    }
+    return params;
+  }
+
+  @Override
+  public ParameterizedPreparedStatementSetter getPreparedStatementSetter() {
+    return (ps, argument) -> {
+      ps.setLong(1, (Long)argument[0]);
+      ps.setLong(2, (Long)argument[1]);
+      ps.setLong(3, (Long)argument[2]);
+      ps.setString(4, (String)argument[3]);
+      ps.setString(5, (String)argument[4]);
+      ps.setString(6, (String)argument[5]);
+      ps.setString(7, (String)argument[6]);
+      ps.setString(8, (String)argument[7]);
+      ps.setString(9, (String)argument[8]);
+      ps.setString(10, (String)argument[9]);
+      ps.setString(11, (String)argument[10]);
+    };
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return ParameterizedCommand.EXACTLY_ONE_ROW;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertTxnComponentsCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertTxnComponentsCommand.java
new file mode 100644
index 000000000000..2f4da649c5ad
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/InsertTxnComponentsCommand.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.txn.entities.OperationType;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedBatchCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.springframework.jdbc.core.ParameterizedPreparedStatementSetter;
+
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+public class InsertTxnComponentsCommand implements ParameterizedBatchCommand {
+  
+  private final LockRequest lockRequest;
+  private final Map, Long> writeIds;
+  private final AddDynamicPartitions dynamicPartitions;
+
+  public InsertTxnComponentsCommand(LockRequest lockRequest, Map, Long> writeIds) {
+    this.lockRequest = lockRequest;
+    this.writeIds = writeIds;
+    this.dynamicPartitions = null;
+  }
+
+  public InsertTxnComponentsCommand(AddDynamicPartitions dynamicPartitions) {
+    this.dynamicPartitions = dynamicPartitions;
+    this.lockRequest = null;
+    this.writeIds = null;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) {
+    return "INSERT INTO \"TXN_COMPONENTS\" (" +
+        "\"TC_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_OPERATION_TYPE\", \"TC_WRITEID\")" +
+        " VALUES (?, ?, ?, ?, ?, ?)";
+  }
+
+  @Override
+  public List getQueryParameters() {
+    return dynamicPartitions == null ? getQueryParametersByLockRequest() : getQueryParametersByDynamicPartitions();
+  }
+
+  @Override
+  public ParameterizedPreparedStatementSetter getPreparedStatementSetter() {
+    return (ps, argument) -> {
+      ps.setLong(1, (Long)argument[0]);
+      ps.setString(2, (String)argument[1]);
+      ps.setString(3, (String)argument[2]);
+      ps.setString(4, (String)argument[3]);
+      ps.setString(5, (String)argument[4]);
+      ps.setObject(6, argument[5], Types.BIGINT);
+    };
+  }
+
+  @Override
+  public Function resultPolicy() {
+    return ParameterizedCommand.EXACTLY_ONE_ROW;
+  }
+  
+  private List getQueryParametersByLockRequest() {
+    assert lockRequest != null;
+    List params = new ArrayList<>(lockRequest.getComponentSize());
+    Set> alreadyAddedTables = new HashSet<>();
+
+    for (LockComponent lc : lockRequest.getComponent()) {
+      if (lc.isSetIsTransactional() && !lc.isIsTransactional()) {
+        //we don't prevent using non-acid resources in a txn, but we do lock them
+        continue;
+      }
+      if (!shouldUpdateTxnComponent(lockRequest.getTxnid(), lockRequest, lc)) {
+        continue;
+      }
+
+      Function> getWriteIdKey = lockComponent ->
+          Pair.of(StringUtils.lowerCase(lockComponent.getDbname()), StringUtils.lowerCase(lockComponent.getTablename()));
+
+      String dbName = StringUtils.lowerCase(lc.getDbname());
+      String tblName = StringUtils.lowerCase(lc.getTablename());
+      String partName = TxnUtils.normalizePartitionCase(lc.getPartitionname());
+      OperationType opType = OperationType.fromDataOperationType(lc.getOperationType());
+      Pair writeIdKey = getWriteIdKey.apply(lc);
+
+
+      Predicate isDynPart = lockComponent -> lockComponent.isSetIsDynamicPartitionWrite() && lockComponent.isIsDynamicPartitionWrite();
+      Set> isDynPartUpdate = lockRequest.getComponent().stream().filter(isDynPart)
+          .filter(lockComponent -> lockComponent.getOperationType() == DataOperationType.UPDATE || lockComponent.getOperationType() == DataOperationType.DELETE)
+          .map(getWriteIdKey)
+          .collect(Collectors.toSet());
+
+      if (isDynPart.test(lc)) {
+        partName = null;
+        if (alreadyAddedTables.contains(writeIdKey)) {
+          continue;
+        }
+        opType = isDynPartUpdate.contains(writeIdKey) ? OperationType.UPDATE : OperationType.INSERT;
+      }
+      Long writeId = writeIds.get(writeIdKey);
+
+      params.add(new Object[]{lockRequest.getTxnid(), dbName, tblName, partName, opType.getSqlConst(), writeId});
+      alreadyAddedTables.add(writeIdKey);
+    }
+    return params;    
+  }
+
+  private List getQueryParametersByDynamicPartitions() {
+    assert dynamicPartitions != null;
+    //for RU this may be null so we should default it to 'u' which is most restrictive
+    OperationType ot = OperationType.UPDATE;
+    if (dynamicPartitions.isSetOperationType()) {
+      ot = OperationType.fromDataOperationType(dynamicPartitions.getOperationType());
+    }
+    
+    List params = new ArrayList<>(dynamicPartitions.getPartitionnamesSize());
+    for (String partName : dynamicPartitions.getPartitionnames()) {
+      params.add(new Object[]{
+          dynamicPartitions.getTxnid(),
+          dynamicPartitions.getDbname().toLowerCase(),
+          dynamicPartitions.getTablename().toLowerCase(),
+          partName,
+          ot.getSqlConst(),
+          dynamicPartitions.getWriteid()
+      });
+    }
+    return params;
+  }
+
+  private boolean shouldUpdateTxnComponent(long txnid, LockRequest rqst, LockComponent lc) {
+    if(!lc.isSetOperationType()) {
+      //request came from old version of the client
+      return true; //this matches old behavior
+    }
+    else {
+      switch (lc.getOperationType()) {
+        case INSERT:
+        case UPDATE:
+        case DELETE:
+          return true;
+        case SELECT:
+          return false;
+        case NO_TXN:
+              /*this constant is a bit of a misnomer since we now always have a txn context.  It
+               just means the operation is such that we don't care what tables/partitions it
+               affected as it doesn't trigger a compaction or conflict detection.  A better name
+               would be NON_TRANSACTIONAL.*/
+          return false;
+        default:
+          //since we have an open transaction, only 4 values above are expected
+          throw new IllegalStateException("Unexpected DataOperationType: " + lc.getOperationType()
+              + " agentInfo=" + rqst.getAgentInfo() + " " + JavaUtils.txnIdToString(txnid));
+      }
+    }
+  }
+  
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/RemoveCompactionMetricsDataCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveCompactionMetricsDataCommand.java
similarity index 94%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/RemoveCompactionMetricsDataCommand.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveCompactionMetricsDataCommand.java
index 8e07104760d2..a3fd4a41abc2 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/RemoveCompactionMetricsDataCommand.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveCompactionMetricsDataCommand.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
 import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
 import org.springframework.jdbc.core.namedparam.SqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/RemoveDuplicateCompleteTxnComponentsCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java
similarity index 98%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/RemoveDuplicateCompleteTxnComponentsCommand.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java
index 0ff9fa3a4b35..d2cd6353fc2f 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/RemoveDuplicateCompleteTxnComponentsCommand.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveTxnsFromMinHistoryLevelCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveTxnsFromMinHistoryLevelCommand.java
new file mode 100644
index 000000000000..2b69264809a6
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveTxnsFromMinHistoryLevelCommand.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ConditionalCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.InClauseBatchCommand;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.List;
+
+public class RemoveTxnsFromMinHistoryLevelCommand extends InClauseBatchCommand implements ConditionalCommand {
+
+  public RemoveTxnsFromMinHistoryLevelCommand(List txnids) {
+    super("DELETE FROM \"MIN_HISTORY_LEVEL\" WHERE \"MHL_TXNID\" IN (:txnIds)",
+        new MapSqlParameterSource().addValue("txnIds", txnids), "txnIds", Long::compareTo);
+  }
+
+  @Override
+  public boolean shouldBeUsed(DatabaseProduct databaseProduct) {
+    return TxnHandler.ConfVars.useMinHistoryLevel();
+  }
+
+  @Override
+  public void onError(DatabaseProduct databaseProduct, Exception e) {
+    if (databaseProduct.isTableNotExistsError(e)) {
+      TxnHandler.ConfVars.setUseMinHistoryLevel(false);
+    }
+  }  
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveWriteIdsFromMinHistoryCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveWriteIdsFromMinHistoryCommand.java
new file mode 100644
index 000000000000..0460af7f38a3
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveWriteIdsFromMinHistoryCommand.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.commands;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ConditionalCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.InClauseBatchCommand;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.List;
+
+public class RemoveWriteIdsFromMinHistoryCommand extends InClauseBatchCommand implements ConditionalCommand {
+
+  public RemoveWriteIdsFromMinHistoryCommand(List txnids) {
+    super("DELETE FROM \"MIN_HISTORY_WRITE_ID\" WHERE \"MH_TXNID\" IN (:txnIds)",
+        new MapSqlParameterSource().addValue("txnIds", txnids), "txnIds", Long::compareTo);
+  }
+
+  @Override
+  public boolean shouldBeUsed(DatabaseProduct databaseProduct) {
+    return TxnHandler.ConfVars.useMinHistoryWriteId();
+  }
+
+  @Override
+  public void onError(DatabaseProduct databaseProduct, Exception e) {
+    if (databaseProduct.isTableNotExistsError(e)) {
+      // If the table does not exists anymore, we disable the flag and start to work the new way
+      // This enables to switch to the new functionality without a restart
+      TxnHandler.ConfVars.setUseMinHistoryWriteId(false);
+    }
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortCompactionFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortCompactionFunction.java
new file mode 100644
index 000000000000..bd4fa91961db
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortCompactionFunction.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.AbortCompactResponse;
+import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest;
+import org.apache.hadoop.hive.metastore.api.AbortCompactionResponseElement;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertCompactionInfoCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.DbTimeHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryCallProperties;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryFunction;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.springframework.transaction.TransactionDefinition.PROPAGATION_REQUIRED;
+
+public class AbortCompactionFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortCompactionFunction.class);
+
+  public static final String SELECT_COMPACTION_QUEUE_BY_COMPID =
+      "SELECT XX.* FROM ( SELECT " +
+          "   \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\", " +
+          "   \"CQ_STATE\" AS \"CC_STATE\", \"CQ_TYPE\" AS \"CC_TYPE\", \"CQ_TBLPROPERTIES\" AS \"CC_TBLPROPERTIES\", \"CQ_WORKER_ID\" AS \"CC_WORKER_ID\", " +
+          "   \"CQ_START\" AS \"CC_START\", \"CQ_RUN_AS\" AS \"CC_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" AS \"CC_HIGHEST_WRITE_ID\", \"CQ_META_INFO\" AS \"CC_META_INFO\"," +
+          "   \"CQ_HADOOP_JOB_ID\" AS \"CC_HADOOP_JOB_ID\", \"CQ_ERROR_MESSAGE\" AS \"CC_ERROR_MESSAGE\",  \"CQ_ENQUEUE_TIME\" AS \"CC_ENQUEUE_TIME\"," +
+          "   \"CQ_WORKER_VERSION\" AS \"CC_WORKER_VERSION\", \"CQ_INITIATOR_ID\" AS \"CC_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" AS \"CC_INITIATOR_VERSION\", " +
+          "   \"CQ_RETRY_RETENTION\" AS \"CC_RETRY_RETENTION\", \"CQ_NEXT_TXN_ID\" AS \"CC_NEXT_TXN_ID\", \"CQ_TXN_ID\" AS \"CC_TXN_ID\", " +
+          "   \"CQ_COMMIT_TIME\" AS \"CC_COMMIT_TIME\", \"CQ_POOL_NAME\" AS \"CC_POOL_NAME\",  " +
+          "   \"CQ_NUMBER_OF_BUCKETS\" AS \"CC_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\" AS \"CC_ORDER_BY\" " +
+          "   FROM " +
+          "   \"COMPACTION_QUEUE\" " +
+          "   UNION ALL " +
+          "   SELECT " +
+          "   \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", " +
+          "   \"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_RUN_AS\", " +
+          "   \"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", " +
+          "   \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\", " +
+          "    -1 , \"CC_NEXT_TXN_ID\", \"CC_TXN_ID\", \"CC_NEXT_TXN_ID\", \"CC_POOL_NAME\", " +
+          "   \"CC_NUMBER_OF_BUCKETS\", \"CC_ORDER_BY\" " +
+          "   FROM   " +
+          "   \"COMPLETED_COMPACTIONS\") XX WHERE \"CC_ID\" IN (:ids) ";
+  
+
+  private final AbortCompactionRequest reqst;
+  private final SqlRetryHandler sqlRetryHandler;
+
+  public AbortCompactionFunction(AbortCompactionRequest reqst, SqlRetryHandler sqlRetryHandler) {
+    this.reqst = reqst;
+    this.sqlRetryHandler = sqlRetryHandler;
+  }
+
+  @Override
+  public AbortCompactResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    Map abortCompactionResponseElements = new HashMap<>();
+    AbortCompactResponse response = new AbortCompactResponse(new HashMap<>());
+    response.setAbortedcompacts(abortCompactionResponseElements);
+
+    reqst.getCompactionIds().forEach(x -> abortCompactionResponseElements.put(x, getAbortCompactionResponseElement(x,"Error","No Such Compaction Id Available")));
+
+    List eligibleCompactionsToAbort = 
+        findEligibleCompactionsToAbort(jdbcResource, abortCompactionResponseElements, reqst.getCompactionIds());
+    
+    CompactionAborter aborter = new CompactionAborter(jdbcResource);    
+    for (CompactionInfo compactionInfo : eligibleCompactionsToAbort) {
+      try {
+        AbortCompactionResponseElement responseElement = sqlRetryHandler.executeWithRetry(
+            new SqlRetryCallProperties().withCallerId("abortCompaction"), 
+            aborter.withCompactionInfo(compactionInfo));
+        abortCompactionResponseElements.put(compactionInfo.id, responseElement);
+      } catch (TException e) {
+        throw new MetaException(e.getMessage());
+      }
+    }
+    return response;
+  }
+
+  private List findEligibleCompactionsToAbort(MultiDataSourceJdbcResource jdbcResource,
+      Map abortCompactionResponseElements, List requestedCompId) {
+
+    return jdbcResource.getJdbcTemplate().query(
+        SELECT_COMPACTION_QUEUE_BY_COMPID,
+        new MapSqlParameterSource().addValue("ids", requestedCompId),
+        rs -> {
+          List compactionInfoList = new ArrayList<>();
+          while (rs.next()) {
+            char compState = rs.getString(5).charAt(0);
+            long compID = rs.getLong(1);
+            if (CompactionState.INITIATED.equals(CompactionState.fromSqlConst(compState))) {
+              compactionInfoList.add(CompactionInfo.loadFullFromCompactionQueue(rs));
+            } else {
+              abortCompactionResponseElements.put(compID, getAbortCompactionResponseElement(compID,"Error",
+                  "Error while aborting compaction as compaction is in state-" + CompactionState.fromSqlConst(compState)));
+            }
+          }
+          return compactionInfoList;
+        });    
+  }
+
+  private AbortCompactionResponseElement getAbortCompactionResponseElement(long compactionId, String status, String message) {
+    AbortCompactionResponseElement resEle = new AbortCompactionResponseElement(compactionId);
+    resEle.setMessage(message);
+    resEle.setStatus(status);
+    return resEle;
+  }
+
+
+  private class CompactionAborter implements SqlRetryFunction {
+
+    private final MultiDataSourceJdbcResource jdbcResource;
+    private CompactionInfo compactionInfo;
+
+    public CompactionAborter(MultiDataSourceJdbcResource jdbcResource) {
+      this.jdbcResource = jdbcResource;
+    }
+
+    public CompactionAborter withCompactionInfo(CompactionInfo compactionInfo) {
+      this.compactionInfo = compactionInfo;
+      return this;
+    }
+
+    @Override
+    public AbortCompactionResponseElement execute() {
+      try (TransactionContext context = jdbcResource.getTransactionManager().getNewTransaction(PROPAGATION_REQUIRED)) {
+        compactionInfo.state = TxnStore.ABORTED_STATE;
+        compactionInfo.errorMessage = "Compaction Aborted by Abort Comapction request.";
+        int updCount;
+        try {
+          updCount = jdbcResource.execute(new InsertCompactionInfoCommand(compactionInfo, jdbcResource.execute(new DbTimeHandler()).getTime()));
+        } catch (Exception e) {
+          LOG.error("Unable to update compaction record: {}.", compactionInfo);
+          return getAbortCompactionResponseElement(compactionInfo.id, "Error",
+              "Error while aborting compaction:Unable to update compaction record in COMPLETED_COMPACTIONS");
+        }
+        LOG.debug("Inserted {} entries into COMPLETED_COMPACTIONS", updCount);
+        try {
+          updCount = jdbcResource.getJdbcTemplate().update("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = :id",
+              new MapSqlParameterSource().addValue("id", compactionInfo.id));
+          if (updCount != 1) {
+            LOG.error("Unable to update compaction record: {}. updCnt={}", compactionInfo, updCount);
+            return getAbortCompactionResponseElement(compactionInfo.id, "Error",
+                "Error while aborting compaction: Unable to update compaction record in COMPACTION_QUEUE");
+          } else {
+            jdbcResource.getTransactionManager().commit(context);
+            return getAbortCompactionResponseElement(compactionInfo.id, "Success",
+                "Successfully aborted compaction");
+          }
+        } catch (DataAccessException e) {
+          return getAbortCompactionResponseElement(compactionInfo.id, "Error",
+              "Error while aborting compaction:" + e.getMessage());
+        }
+      }
+    }
+
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnFunction.java
new file mode 100644
index 000000000000..72c19e4c6e72
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnFunction.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.DeleteReplTxnMapEntryCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.FindTxnStateHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetOpenTxnTypeAndLockHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.TargetTxnIdListHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+
+public class AbortTxnFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortTxnFunction.class);
+
+  public AbortTxnFunction(AbortTxnRequest rqst) {
+    this.rqst = rqst;
+  }
+
+  private final AbortTxnRequest rqst;
+  
+  @Override
+  public TxnType execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, NoSuchTxnException, TxnAbortedException {
+    long txnid = rqst.getTxnid();
+    TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE;
+    long sourceTxnId = -1;
+    boolean isReplayedReplTxn = TxnType.REPL_CREATED.equals(rqst.getTxn_type());
+    boolean isHiveReplTxn = rqst.isSetReplPolicy() && TxnType.DEFAULT.equals(rqst.getTxn_type());
+    if (isReplayedReplTxn) {
+      assert (rqst.isSetReplPolicy());
+      sourceTxnId = rqst.getTxnid();
+      List targetTxnIds = jdbcResource.execute(new TargetTxnIdListHandler(rqst.getReplPolicy(), Collections.singletonList(sourceTxnId)));
+      if (targetTxnIds.isEmpty()) {
+        // Idempotent case where txn was already closed or abort txn event received without
+        // corresponding open txn event.
+        LOG.info("Target txn id is missing for source txn id : {} and repl policy {}", sourceTxnId,
+            rqst.getReplPolicy());
+        return null;
+      }
+      assert targetTxnIds.size() == 1;
+      txnid = targetTxnIds.get(0);
+    }
+
+    TxnType txnType = jdbcResource.execute(new GetOpenTxnTypeAndLockHandler(jdbcResource.getSqlGenerator(), txnid));
+    if (txnType == null) {
+      TxnStatus status = jdbcResource.execute(new FindTxnStateHandler(txnid));
+      if (status == TxnStatus.ABORTED) {
+        if (isReplayedReplTxn) {
+          // in case of replication, idempotent is taken care by getTargetTxnId
+          LOG.warn("Invalid state ABORTED for transactions started using replication replay task");
+          jdbcResource.execute(new DeleteReplTxnMapEntryCommand(sourceTxnId, rqst.getReplPolicy()));
+        }
+        LOG.info("abortTxn({}) requested by it is already {}", JavaUtils.txnIdToString(txnid), TxnStatus.ABORTED);
+        return null;
+      }
+      TxnUtils.raiseTxnUnexpectedState(status, txnid);
+    }
+
+    if (isReplayedReplTxn) {
+      txnErrorMsg = TxnErrorMsg.ABORT_REPLAYED_REPL_TXN;
+    } else if (isHiveReplTxn) {
+      txnErrorMsg = TxnErrorMsg.ABORT_DEFAULT_REPL_TXN;
+    } else if (rqst.isSetErrorCode()) {
+      txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode());
+    }
+
+    new AbortTxnsFunction(Collections.singletonList(txnid), false, true,
+        isReplayedReplTxn, txnErrorMsg).execute(jdbcResource);
+
+    if (isReplayedReplTxn) {
+      jdbcResource.execute(new DeleteReplTxnMapEntryCommand(sourceTxnId, rqst.getReplPolicy()));
+    }
+    
+    return txnType;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnsFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnsFunction.java
new file mode 100644
index 000000000000..c280b85222c6
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AbortTxnsFunction.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.repl.ReplConst;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveTxnsFromMinHistoryLevelCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveWriteIdsFromMinHistoryCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetDatabaseIdHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.UncategorizedSQLException;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.executeQueriesInBatch;
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.executeQueriesInBatchNoCount;
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class AbortTxnsFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortTxnsFunction.class);
+
+  private final List txnids;
+  private final boolean checkHeartbeat;
+  private final boolean skipCount;
+  private final boolean isReplReplayed;
+  private final TxnErrorMsg txnErrorMsg;
+
+  /**
+   * TODO: expose this as an operation to client.  Useful for streaming API to abort all remaining
+   * transactions in a batch on IOExceptions.
+   * Caller must rollback the transaction if not all transactions were aborted since this will not
+   * attempt to delete associated locks in this case.
+   *
+   * @param txnids list of transactions to abort
+   * @param checkHeartbeat value used by {@code  org.apache.hadoop.hive.metastore.txn.TxnHandler#performTimeOuts()} 
+   *                       to ensure this doesn't Abort txn which were heartbeated after #performTimeOuts() select 
+   *                       and this operation.
+   * @param skipCount If true, the method always returns 0, otherwise returns the number of actually aborted txns
+   */
+  public AbortTxnsFunction(List txnids, boolean checkHeartbeat, boolean skipCount, boolean isReplReplayed, 
+                           TxnErrorMsg txnErrorMsg) {
+    this.txnids = txnids;
+    this.checkHeartbeat = checkHeartbeat;
+    this.skipCount = skipCount;
+    this.isReplReplayed = isReplReplayed;
+    this.txnErrorMsg = txnErrorMsg;
+  }
+
+  /**
+   * @param jdbcResource A {@link MultiDataSourceJdbcResource} instance responsible for providing all the necessary 
+   *                     resources to be able to perform transactional database calls.
+   * @return 0 if skipCount is true, the number of aborted transactions otherwise
+   */
+  @Override
+  public Integer execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    if (txnids.isEmpty()) {
+      return 0;
+    }
+    Configuration conf = jdbcResource.getConf();
+    Collections.sort(txnids);    
+    LOG.debug("Aborting {} transaction(s) {} due to {}", txnids.size(), txnids, txnErrorMsg);
+    
+    int maxBatchSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.JDBC_MAX_BATCH_SIZE);    
+    jdbcResource.execute(new RemoveTxnsFromMinHistoryLevelCommand(txnids));
+    jdbcResource.execute(new RemoveWriteIdsFromMinHistoryCommand(txnids));
+    
+    Connection dbConn = jdbcResource.getConnection();
+    try {
+      DatabaseProduct dbProduct = jdbcResource.getDatabaseProduct();
+      //This is an update statement, thus at any Isolation level will take Write locks so will block
+      //all other ops using S4U on TXNS row.
+      List queries = new ArrayList<>();
+      StringBuilder prefix = new StringBuilder();
+      StringBuilder suffix = new StringBuilder();
+
+      // add update txns queries to query list
+      prefix.append("UPDATE \"TXNS\" SET \"TXN_STATE\" = ").append(TxnStatus.ABORTED)
+          .append(" , \"TXN_META_INFO\" = ").append(txnErrorMsg.toSqlString())
+          .append(" WHERE \"TXN_STATE\" = ").append(TxnStatus.OPEN).append(" AND ");
+      if (checkHeartbeat) {
+        suffix.append(" AND \"TXN_LAST_HEARTBEAT\" < ")
+            .append(getEpochFn(dbProduct)).append("-")
+            .append(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS));
+      }
+      TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"TXN_ID\"", true, false);
+      int numUpdateQueries = queries.size();
+
+      // add delete hive locks queries to query list
+      prefix.setLength(0);
+      suffix.setLength(0);
+      prefix.append("DELETE FROM \"HIVE_LOCKS\" WHERE ");
+      TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"HL_TXNID\"", false, false);
+
+      //If this abort is for REPL_CREATED TXN initiated outside the replication flow, then clean the corresponding entry
+      //from REPL_TXN_MAP and mark that database as replication incompatible.
+      if (!isReplReplayed) {
+        for (String database : getDbNamesForReplayedTxns(jdbcResource, dbConn, txnids)) {
+          markDbAsReplIncompatible(jdbcResource, database);
+        }
+        // Delete mapping from REPL_TXN_MAP if it exists.
+        prefix.setLength(0);
+        suffix.setLength(0);
+        prefix.append("DELETE FROM \"REPL_TXN_MAP\" WHERE ");
+        TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"RTM_TARGET_TXN_ID\"", false, false);
+      }
+
+      int numAborted = 0;
+      try (Statement stmt = dbConn.createStatement()) {
+        // execute all queries in the list in one batch
+        if (skipCount) {
+          executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize);
+        } else {
+          List affectedRowsByQuery = executeQueriesInBatch(stmt, queries, maxBatchSize);
+          numAborted = getUpdateCount(numUpdateQueries, affectedRowsByQuery);
+        }
+      }
+
+      if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
+        Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_ABORTED_TXNS).inc(txnids.size());
+      }
+      LOG.warn("Aborted {} transaction(s) {} due to {}", txnids.size(), txnids, txnErrorMsg);
+      return numAborted;
+    } catch (SQLException e) {
+      throw new UncategorizedSQLException(null, null, e);
+    }
+  }
+
+  private Set getDbNamesForReplayedTxns(MultiDataSourceJdbcResource jdbcResource, Connection dbConn, 
+                                                List targetTxnIds) throws SQLException {
+    Set dbNames = new HashSet<>();
+    if (targetTxnIds.isEmpty()) {
+      return dbNames;
+    }
+    List inQueries = new ArrayList<>();
+    StringBuilder prefix = new StringBuilder();
+    prefix.append("SELECT \"RTM_REPL_POLICY\" FROM \"REPL_TXN_MAP\" WHERE ");
+    TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), inQueries, prefix, new StringBuilder(), targetTxnIds,
+        "\"RTM_TARGET_TXN_ID\"", false, false);
+    for (String query : inQueries) {
+      LOG.debug("Going to execute select <{}>", query);
+      try (PreparedStatement pst = jdbcResource.getSqlGenerator().prepareStmtWithParameters(dbConn, query, null);
+           ResultSet rs = pst.executeQuery()) {
+        while (rs.next()) {
+          dbNames.add(MetaStoreUtils.getDbNameFromReplPolicy(rs.getString(1)));
+        }
+      }
+    }
+    return dbNames;
+  }
+
+  private void markDbAsReplIncompatible(MultiDataSourceJdbcResource jdbcResource, String database) throws SQLException, MetaException {
+    try (Statement stmt = jdbcResource.getConnection().createStatement()){
+      String catalog = MetaStoreUtils.getDefaultCatalog(jdbcResource.getConf());
+      String s = jdbcResource.getSqlGenerator().getDbProduct().getPrepareTxnStmt();
+      if (s != null) {
+        stmt.execute(s);
+      }
+      long dbId = jdbcResource.execute(new GetDatabaseIdHandler(database, catalog));
+      new UpdataDatabasePropFunction(database, dbId, ReplConst.REPL_INCOMPATIBLE, ReplConst.TRUE).execute(jdbcResource);
+    }
+  }
+
+  private int getUpdateCount(int numUpdateQueries, List affectedRowsByQuery) {
+    return affectedRowsByQuery.stream()
+        .limit(numUpdateQueries)
+        .mapToInt(Integer::intValue)
+        .sum();
+  }
+
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AcquireTxnLockFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AcquireTxnLockFunction.java
new file mode 100644
index 000000000000..eb2d47eea4c0
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AcquireTxnLockFunction.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Statement;
+
+public class AcquireTxnLockFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AcquireTxnLockFunction.class.getName());
+  
+  private final boolean shared;
+
+  public AcquireTxnLockFunction(boolean shared) {
+    this.shared = shared;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    String sqlStmt = jdbcResource.getSqlGenerator().createTxnLockStatement(shared);
+    jdbcResource.getJdbcTemplate().getJdbcTemplate().execute((Statement stmt) -> {
+      stmt.execute(sqlStmt);
+      return null;
+    });
+    LOG.debug("TXN lock locked by '{}' in mode {}", JavaUtils.hostname(), shared);
+    return null;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AllocateTableWriteIdsFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AllocateTableWriteIdsFunction.java
new file mode 100644
index 000000000000..247c9345ea29
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/AllocateTableWriteIdsFunction.java
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
+import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.AddWriteIdsToTxnToWriteIdCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.TargetTxnIdListHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.InClauseBatchCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.UncategorizedSQLException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class AllocateTableWriteIdsFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbortTxnFunction.class);
+
+  private final AllocateTableWriteIdsRequest rqst;
+  private final List transactionalListeners;
+
+  public AllocateTableWriteIdsFunction(AllocateTableWriteIdsRequest rqst, List transactionalListeners) {
+    this.rqst = rqst;
+    this.transactionalListeners = transactionalListeners;
+  }
+
+  @Override
+  public AllocateTableWriteIdsResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    List txnIds;
+    String dbName = rqst.getDbName().toLowerCase();
+    String tblName = rqst.getTableName().toLowerCase();
+    boolean shouldReallocate = rqst.isReallocate();
+    Connection dbConn = jdbcResource.getConnection();
+    List txnToWriteIds = new ArrayList<>();
+    List srcTxnToWriteIds = null;
+
+    if (rqst.isSetReplPolicy()) {
+      srcTxnToWriteIds = rqst.getSrcTxnToWriteIdList();
+      List srcTxnIds = new ArrayList<>();
+      assert (rqst.isSetSrcTxnToWriteIdList());
+      assert (!rqst.isSetTxnIds());
+      assert (!srcTxnToWriteIds.isEmpty());
+
+      for (TxnToWriteId txnToWriteId : srcTxnToWriteIds) {
+        srcTxnIds.add(txnToWriteId.getTxnId());
+      }
+      txnIds = jdbcResource.execute(new TargetTxnIdListHandler(rqst.getReplPolicy(), srcTxnIds));
+      if (srcTxnIds.size() != txnIds.size()) {
+        // Idempotent case where txn was already closed but gets allocate write id event.
+        // So, just ignore it and return empty list.
+        LOG.info("Idempotent case: Target txn id is missing for source txn id : {} and repl policy {}", srcTxnIds,
+            rqst.getReplPolicy());
+        return new AllocateTableWriteIdsResponse(txnToWriteIds);
+      }
+    } else {
+      assert (!rqst.isSetSrcTxnToWriteIdList());
+      assert (rqst.isSetTxnIds());
+      txnIds = rqst.getTxnIds();
+    }
+
+    //Easiest check since we can't differentiate do we handle singleton list or list with multiple txn ids.
+    if (txnIds.size() > 1) {
+      Collections.sort(txnIds); //easier to read logs and for assumption done in replication flow
+    }
+
+    // Check if all the input txns are in valid state.
+    // Write IDs should be allocated only for open and not read-only transactions.
+    if (!isTxnsOpenAndNotReadOnly(jdbcResource, txnIds)) {
+      String errorMsg = "Write ID allocation on " + TableName.getDbTable(dbName, tblName)
+          + " failed for input txns: "
+          + getAbortedAndReadOnlyTxns(jdbcResource, txnIds)
+          + getCommittedTxns(jdbcResource, txnIds);
+      LOG.error(errorMsg);
+
+      throw new IllegalStateException("Write ID allocation failed on " + TableName.getDbTable(dbName, tblName)
+          + " as not all input txns in open state or read-only");
+    }
+
+    List queries = new ArrayList<>();
+    StringBuilder prefix = new StringBuilder();
+    StringBuilder suffix = new StringBuilder();
+    long writeId;
+    int allocatedTxnsCount = 0;
+    List params = Arrays.asList(dbName, tblName);
+
+    if (shouldReallocate) {
+      // during query recompilation after lock acquistion, it is important to realloc new writeIds
+      // to ensure writeIds are committed in increasing order.
+      jdbcResource.execute(new InClauseBatchCommand<>(
+          "DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = :dbName AND \"T2W_TABLE\" = :tableName AND " +
+              "\"T2W_TXNID\" IN (:txnIds)", 
+          new MapSqlParameterSource()
+              .addValue("dbName", dbName)
+              .addValue("tableName", tblName)
+              .addValue("txnIds", txnIds),
+          "txnIds", Long::compareTo));
+    } else {
+      // Traverse the TXN_TO_WRITE_ID to see if any of the input txns already have allocated a
+      // write id for the same db.table. If yes, then need to reuse it else have to allocate new one
+      // The write id would have been already allocated in case of multi-statement txns where
+      // first write on a table will allocate write id and rest of the writes should re-use it.
+      prefix.append("SELECT \"T2W_TXNID\", \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE")
+          .append(" \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND ");
+      TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), queries, prefix, suffix,
+          txnIds, "\"T2W_TXNID\"", false, false);
+      for (String query : queries) {
+        try (PreparedStatement pStmt = jdbcResource.getSqlGenerator().prepareStmtWithParameters(dbConn, query, params)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Going to execute query <" + query.replace("?", "'{}'") + ">", dbName, tblName);
+          }
+          try (ResultSet rs = pStmt.executeQuery()) {
+            while (rs.next()) {
+              // If table write ID is already allocated for the given transaction, then just use it
+              long txnId = rs.getLong(1);
+              writeId = rs.getLong(2);
+              txnToWriteIds.add(new TxnToWriteId(txnId, writeId));
+              allocatedTxnsCount++;
+              LOG.info("Reused already allocated writeID: {} for txnId: {}", writeId, txnId);
+            }            
+          }          
+        } catch (SQLException e) {
+          throw new UncategorizedSQLException(null, null, e);
+        }
+      }
+    }
+
+    // Batch allocation should always happen atomically. Either write ids for all txns is allocated or none.
+    long numOfWriteIds = txnIds.size();
+    assert ((allocatedTxnsCount == 0) || (numOfWriteIds == allocatedTxnsCount));
+    if (allocatedTxnsCount == numOfWriteIds) {
+      // If all the txns in the list have pre-allocated write ids for the given table, then just return.
+      // This is for idempotent case.
+      return new AllocateTableWriteIdsResponse(txnToWriteIds);
+    }
+
+    long srcWriteId = 0;
+    if (rqst.isSetReplPolicy()) {
+      // In replication flow, we always need to allocate write ID equal to that of source.
+      assert (srcTxnToWriteIds != null);
+      srcWriteId = srcTxnToWriteIds.get(0).getWriteId();
+    }
+
+
+    // There are some txns in the list which does not have write id allocated and hence go ahead and do it.
+    // Get the next write id for the given table and update it with new next write id.
+    // This is select for update query which takes a lock if the table entry is already there in NEXT_WRITE_ID
+    String query = jdbcResource.getSqlGenerator().addForUpdateClause(
+        "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = :dbName AND \"NWI_TABLE\" = :tableName");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Going to execute query {}", query);
+    }
+    
+    Long nextWriteId = jdbcResource.getJdbcTemplate().query(query, 
+        new MapSqlParameterSource()
+            .addValue("dbName", dbName)
+            .addValue("tableName", tblName),
+        (ResultSet rs) -> rs.next() ? rs.getLong(1) : null);
+    
+    if (nextWriteId == null) {
+      query = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") " +
+          "VALUES (:dbName, :tableName, :nextId)";
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to execute query {}", query);
+      }
+      
+      // First allocation of write id should add the table to the next_write_id meta table
+      // The initial value for write id should be 1 and hence we add 1 with number of write ids allocated here
+      // For repl flow, we need to force set the incoming write id.
+      writeId = (srcWriteId > 0) ? srcWriteId : 1;
+      jdbcResource.getJdbcTemplate().update(query,
+          new MapSqlParameterSource()
+              .addValue("dbName", dbName)
+              .addValue("tableName", tblName)
+              .addValue("nextId", writeId + numOfWriteIds));      
+    } else {
+      query = "UPDATE \"NEXT_WRITE_ID\" SET \"NWI_NEXT\" = :nextId WHERE \"NWI_DATABASE\" = :dbName AND \"NWI_TABLE\" = :tableName";
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to execute query {}", query);
+      }
+
+      writeId = (srcWriteId > 0) ? srcWriteId : nextWriteId;
+      // Update the NEXT_WRITE_ID for the given table after incrementing by number of write ids allocated
+      jdbcResource.getJdbcTemplate().update(query,
+          new MapSqlParameterSource()
+              .addValue("dbName", dbName)
+              .addValue("tableName", tblName)
+              .addValue("nextId", writeId + numOfWriteIds));
+
+      // For repl flow, if the source write id is mismatching with target next write id, then current
+      // metadata in TXN_TO_WRITE_ID is stale for this table and hence need to clean-up TXN_TO_WRITE_ID.
+      // This is possible in case of first incremental repl after bootstrap where concurrent write
+      // and drop table was performed at source during bootstrap dump.
+      if ((srcWriteId > 0) && (srcWriteId != nextWriteId)) {
+        query = "DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = :dbName AND \"T2W_TABLE\" = :tableName";
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Going to execute query {}", query);
+        }
+
+        jdbcResource.getJdbcTemplate().update(query,
+            new MapSqlParameterSource()
+                .addValue("dbName", dbName)
+                .addValue("tableName", tblName));
+      }
+    }
+
+    // Map the newly allocated write ids against the list of txns which doesn't have pre-allocated write ids
+    jdbcResource.execute(new AddWriteIdsToTxnToWriteIdCommand(dbName, tblName, writeId, txnIds, txnToWriteIds));
+
+    if (transactionalListeners != null) {
+      MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+          EventMessage.EventType.ALLOC_WRITE_ID,
+          new AllocWriteIdEvent(txnToWriteIds, dbName, tblName),
+          dbConn, jdbcResource.getSqlGenerator());
+    }
+
+    LOG.info("Allocated write ids for dbName={}, tblName={} (txnIds: {})", dbName, tblName, rqst.getTxnIds());
+    return new AllocateTableWriteIdsResponse(txnToWriteIds);
+  }
+
+  /**
+   * Checks if all the txns in the list are in open state and not read-only.
+   * @param txnIds list of txns to be evaluated for open state/read-only status
+   * @return If all the txns in open state and not read-only, then return true else false
+   */
+  private boolean isTxnsOpenAndNotReadOnly(MultiDataSourceJdbcResource jdbcResource, List txnIds) {
+    List queries = new ArrayList<>();
+    StringBuilder prefix = new StringBuilder();
+
+    // Get the count of txns from the given list that are in open state and not read-only.
+    // If the returned count is same as the input number of txns, then all txns are in open state and not read-only.
+    prefix.append("SELECT COUNT(*) FROM \"TXNS\" WHERE \"TXN_STATE\" = ").append(TxnStatus.OPEN)
+        .append(" AND \"TXN_TYPE\" != ").append(TxnType.READ_ONLY.getValue()).append(" AND ");
+
+    TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), queries, prefix, new StringBuilder(),
+        txnIds, "\"TXN_ID\"", false, false);
+
+    AtomicLong count = new AtomicLong(0);
+    for (String query : queries) {
+      LOG.debug("Going to execute query <{}>", query);
+      jdbcResource.getJdbcTemplate().query(query, rs -> {
+        while (rs.next()) {
+          count.set(count.get() + rs.getLong(1));
+        }
+        return null;
+      });
+    }
+    return count.get() == txnIds.size();
+  }
+
+  /**
+   * Get txns from the list that are either aborted or read-only.
+   * @param txnIds list of txns to be evaluated for aborted state/read-only status
+   */
+  private String getAbortedAndReadOnlyTxns(MultiDataSourceJdbcResource jdbcResource, List txnIds) {
+    List queries = new ArrayList<>();
+    StringBuilder prefix = new StringBuilder();
+
+    // Check if any of the txns in the list are either aborted or read-only.
+    prefix.append("SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\" FROM \"TXNS\" WHERE ");
+    TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), queries, prefix, new StringBuilder(),
+        txnIds, "\"TXN_ID\"", false, false);
+    StringBuilder txnInfo = new StringBuilder();
+
+    for (String query : queries) {
+      LOG.debug("Going to execute query <{}>", query);
+      jdbcResource.getJdbcTemplate().query(query, rs -> {
+        while (rs.next()) {
+          long txnId = rs.getLong(1);
+          TxnStatus txnState = TxnStatus.fromString(rs.getString(2));
+          TxnType txnType = TxnType.findByValue(rs.getInt(3));
+
+          if (txnState != TxnStatus.OPEN) {
+            txnInfo.append("{").append(txnId).append(",").append(txnState).append("}");
+          } else if (txnType == TxnType.READ_ONLY) {
+            txnInfo.append("{").append(txnId).append(",read-only}");
+          }
+        }
+        return null;
+      });
+    }
+    return txnInfo.toString();
+  }
+
+  /**
+   * Get txns from the list that are committed.
+   * @param txnIds list of txns to be evaluated for committed state
+   */
+  private String getCommittedTxns(MultiDataSourceJdbcResource jdbcResource, List txnIds) {
+    List queries = new ArrayList<>();
+    StringBuilder prefix = new StringBuilder();
+
+    // Check if any of the txns in the list are committed.
+    prefix.append("SELECT \"CTC_TXNID\" FROM \"COMPLETED_TXN_COMPONENTS\" WHERE ");
+    TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), queries, prefix, new StringBuilder(),
+        txnIds, "\"CTC_TXNID\"", false, false);
+    StringBuilder txnInfo = new StringBuilder();
+
+    for (String query : queries) {
+      LOG.debug("Going to execute query <{}>", query);
+      jdbcResource.getJdbcTemplate().query(query, rs -> {
+        while (rs.next()) {
+          long txnId = rs.getLong(1);
+          txnInfo.append("{").append(txnId).append(",c}");
+        }          
+        return null;
+      });
+    }
+    return txnInfo.toString();
+  }
+  
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CheckLockFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CheckLockFunction.java
new file mode 100644
index 000000000000..af05e1fdd292
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CheckLockFunction.java
@@ -0,0 +1,342 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.entities.LockInfo;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetLocksByLockId;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.apache.hadoop.hive.metastore.utils.LockTypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+import static org.apache.hadoop.hive.metastore.txn.entities.LockInfo.LOCK_ACQUIRED;
+
+public class CheckLockFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CheckLockFunction.class);
+
+  private static final String EXCL_CTAS_ERR_MSG =
+      "Failed to initiate a concurrent CTAS operation with the same table name, lockInfo : %s";
+  private static final String ZERO_WAIT_READ_ERR_MSG = "Unable to acquire read lock due to an existing exclusive lock {%s}";
+  
+  private final long extLockId;
+  private final long txnId;
+  private final boolean zeroWaitReadEnabled;
+  private final boolean isExclusiveCTAS;
+
+  public CheckLockFunction(long extLockId, long txnId, 
+                           boolean zeroWaitReadEnabled, boolean isExclusiveCTAS) {
+    this.extLockId = extLockId;
+    this.txnId = txnId;
+    this.zeroWaitReadEnabled = zeroWaitReadEnabled;
+    this.isExclusiveCTAS = isExclusiveCTAS;
+  }
+
+  @SuppressWarnings("squid:S2583")
+  @Override
+  public LockResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, NoSuchTxnException, TxnAbortedException, NoSuchLockException {
+    LockResponse response = new LockResponse();
+    /**
+     * todo: Longer term we should pass this from client somehow - this would be an optimization;  once
+     * that is in place make sure to build and test "writeSet" below using OperationType not LockType
+     * With Static Partitions we assume that the query modifies exactly the partitions it locked.  (not entirely
+     * realistic since Update/Delete may have some predicate that filters out all records out of
+     * some partition(s), but plausible).  For DP, we acquire locks very wide (all known partitions),
+     * but for most queries only a fraction will actually be updated.  #addDynamicPartitions() tells
+     * us exactly which ones were written to.  Thus using this trick to kill a query early for
+     * DP queries may be too restrictive.
+     */
+    boolean isPartOfDynamicPartitionInsert = true;
+    List locksBeingChecked = getLocksFromLockId(jdbcResource, extLockId); //being acquired now
+    response.setLockid(extLockId);
+
+    //This is the set of entities that the statement represented by extLockId wants to update
+    List writeSet = new ArrayList<>();
+
+    for (LockInfo info : locksBeingChecked) {
+      if (!isPartOfDynamicPartitionInsert && info.getType() == LockType.SHARED_WRITE) {
+        writeSet.add(info);
+      }
+    }
+    if (!writeSet.isEmpty()) {
+      if (writeSet.get(0).getTxnId() == 0) {
+        //Write operation always start a txn
+        throw new IllegalStateException("Found Write lock for " + JavaUtils.lockIdToString(extLockId) + " but no txnid");
+      }
+
+
+      Object[] args = new Object[writeSet.size() * 4 + 1];
+      int index = 0;
+      args[index++] = writeSet.get(0).getTxnId();
+      StringBuilder sb = new StringBuilder(" \"WS_DATABASE\", \"WS_TABLE\", \"WS_PARTITION\", " +
+          "\"WS_TXNID\", \"WS_COMMIT_ID\" " +
+          "FROM \"WRITE_SET\" WHERE WS_COMMIT_ID >= ? AND (");//see commitTxn() for more info on this inequality
+      for (int i = 0; i < writeSet.size(); i++) {
+        sb.append("(\"WS_DATABASE\" = ? AND \"WS_TABLE\" = ? AND (\"WS_PARTITION\" = ? OR ? IS NULL)");
+        if (i < writeSet.size() - 1) {
+          sb.append(" OR ");
+        }
+        sb.append(")");
+        LockInfo info = writeSet.get(i);
+        args[index++] = info.getDb();
+        args[index++] = info.getTable();
+        args[index++] = info.getPartition();
+        args[index++] = info.getPartition();
+      }
+
+      WriteSetInfo wsInfo = jdbcResource.getJdbcTemplate().getJdbcTemplate().query(sb.toString(), args, (ResultSet rs) -> {
+        WriteSetInfo info = null;
+        if (rs.next()) {
+          info = new WriteSetInfo();
+          info.database = rs.getString("WS_DATABASE");
+          info.table = rs.getString("WS_TABLE");
+          info.partition = rs.getString("WS_PARTITION");
+          info.txnId = rs.getLong("WS_TXNID");
+          info.commitId = rs.getLong("WS_COMMIT_ID");
+        }
+        return info;
+      });
+
+      if (wsInfo != null) {
+        /**
+         * if here, it means we found an already committed txn which overlaps with the current one and
+         * it updated the same resource the current txn wants to update.  By First-committer-wins
+         * rule, current txn will not be allowed to commit so  may as well kill it now;  This is just an
+         * optimization to prevent wasting cluster resources to run a query which is known to be DOA.
+         * {@link #commitTxn(CommitTxnRequest)} has the primary responsibility to ensure this.
+         * checkLock() runs at READ_COMMITTED, so you could have another (Hive) txn running commitTxn()
+         * in parallel and thus writing to WRITE_SET.  commitTxn() logic is properly mutexed to ensure
+         * that we don't "miss" any WW conflicts. We could've mutexed the checkLock() and commitTxn()
+         * as well but this reduces concurrency for very little gain.
+         * Note that update/delete (which runs as dynamic partition insert) acquires a lock on the table,
+         * but WRITE_SET has entries for actual partitions updated. Thus this optimization will "miss"
+         * the WW conflict, but it will be caught in commitTxn() where actual partitions written are known.
+         * This is OK since we want 2 concurrent updates that update different sets of partitions to both commit.
+         */
+        String resourceName = wsInfo.database + '/' + wsInfo.table;
+        if (wsInfo.partition != null) {
+          resourceName += '/' + wsInfo.partition;
+        }
+
+        String msg = "Aborting " + JavaUtils.txnIdToString(writeSet.get(0).getTxnId()) +
+            " since a concurrent committed transaction [" + JavaUtils.txnIdToString(wsInfo.txnId) + "," + wsInfo.commitId +
+            "] has already updated resource '" + resourceName + "'";
+        LOG.info(msg);
+        int count = new AbortTxnsFunction(Collections.singletonList(writeSet.get(0).getTxnId()),
+            false, false, false, TxnErrorMsg.ABORT_CONCURRENT).execute(jdbcResource);
+        if (count != 1) {
+          throw new IllegalStateException(msg + " FAILED!");
+        }
+        throw new TxnAbortedException(msg);
+      }
+    }
+
+    String queryStr =
+        " \"EX\".*, \"REQ\".\"HL_LOCK_INT_ID\" \"LOCK_INT_ID\", \"REQ\".\"HL_LOCK_TYPE\" \"LOCK_TYPE\" FROM (" +
+            " SELECT \"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\"," +
+            " \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\" FROM \"HIVE_LOCKS\"" +
+            " WHERE \"HL_LOCK_EXT_ID\" < " + extLockId + ") \"EX\"" +
+            " INNER JOIN (" +
+            " SELECT \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\"," +
+            " \"HL_LOCK_TYPE\" FROM \"HIVE_LOCKS\"" +
+            " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + ") \"REQ\"" +
+            " ON \"EX\".\"HL_DB\" = \"REQ\".\"HL_DB\"" +
+            " AND (\"EX\".\"HL_TABLE\" IS NULL OR \"REQ\".\"HL_TABLE\" IS NULL" +
+            " OR \"EX\".\"HL_TABLE\" = \"REQ\".\"HL_TABLE\"" +
+            " AND (\"EX\".\"HL_PARTITION\" IS NULL OR \"REQ\".\"HL_PARTITION\" IS NULL" +
+            " OR \"EX\".\"HL_PARTITION\" = \"REQ\".\"HL_PARTITION\"))" +
+        /*different locks from same txn should not conflict with each other,
+          txnId=0 means it's a select or IUD which does not write to ACID table*/
+            " WHERE (\"REQ\".\"HL_TXNID\" = 0 OR \"EX\".\"HL_TXNID\" != \"REQ\".\"HL_TXNID\")" +
+            " AND ";
+
+      /**EXCLUSIVE lock on partition should prevent SHARED_READ on the table, however there is no reason
+        for an EXCLUSIVE on a table to prevent SHARED_READ on a database. Similarly, EXCLUSIVE on a partition
+        should not conflict with SHARED_READ on a database.
+        SHARED_READ is usually acquired on a database to make sure it's not dropped, while some operation
+        is performed on that db (e.g. show tables, created table, etc).
+        EXCLUSIVE on an object may mean it's being dropped or overwritten.*/
+    String[] whereStr = {
+        // shared-read
+        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.sharedRead() + " AND \"EX\".\"HL_LOCK_TYPE\"=" +
+            LockTypeUtil.exclusive() + " AND NOT (\"EX\".\"HL_TABLE\" IS NOT NULL AND \"REQ\".\"HL_TABLE\" IS NULL)",
+        // exclusive
+        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.exclusive() +
+            " AND NOT (\"EX\".\"HL_TABLE\" IS NULL AND \"EX\".\"HL_LOCK_TYPE\"=" +
+            LockTypeUtil.sharedRead() + " AND \"REQ\".\"HL_TABLE\" IS NOT NULL)",
+        // shared-write
+        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.sharedWrite() + " AND \"EX\".\"HL_LOCK_TYPE\" IN (" +
+            LockTypeUtil.exclWrite() + "," + LockTypeUtil.exclusive() + ")",
+        // excl-write
+        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.exclWrite() + " AND \"EX\".\"HL_LOCK_TYPE\"!=" +
+            LockTypeUtil.sharedRead()
+    };
+
+    List subQuery = new ArrayList<>();
+    for (String subCond : whereStr) {
+      subQuery.add("(" + jdbcResource.getSqlGenerator().addLimitClause(1, queryStr + subCond) + ")");
+    }
+    String query = String.join(" UNION ALL ", subQuery);
+
+    Boolean success = Objects.requireNonNull(jdbcResource.getJdbcTemplate().query(query, new MapSqlParameterSource(), (ResultSet rs) -> {
+      if (rs.next()) {
+        try {
+          // We acquire all locks for a given query atomically; if 1 blocks, all remain in Waiting state.
+          LockInfo blockedBy = new LockInfo(rs);
+          long intLockId = rs.getLong("LOCK_INT_ID");
+          char lockChar = rs.getString("LOCK_TYPE").charAt(0);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Failure to acquire lock({} intLockId:{} {}), blocked by ({})", JavaUtils.lockIdToString(extLockId),
+                intLockId, JavaUtils.txnIdToString(txnId), blockedBy);
+          }
+
+          LockType lockType = LockTypeUtil.getLockTypeFromEncoding(lockChar)
+              .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar));
+
+          if ((zeroWaitReadEnabled && LockType.SHARED_READ == lockType || isExclusiveCTAS) && TxnUtils.isValidTxn(txnId)) {
+            jdbcResource.getJdbcTemplate().update("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = :extLockId",
+                new MapSqlParameterSource().addValue("extLockId", extLockId));
+
+            response.setErrorMessage(String.format(
+                isExclusiveCTAS ? EXCL_CTAS_ERR_MSG : ZERO_WAIT_READ_ERR_MSG, blockedBy));
+            response.setState(LockState.NOT_ACQUIRED);
+            return false;
+          }
+
+          int updCnt = jdbcResource.getJdbcTemplate().update("UPDATE \"HIVE_LOCKS\"" +
+                  " SET \"HL_BLOCKEDBY_EXT_ID\" = :blockedByExtLockId, \"HL_BLOCKEDBY_INT_ID\" = :blockedByIntLockId " +
+                  " WHERE \"HL_LOCK_EXT_ID\" = :extLockId AND \"HL_LOCK_INT_ID\" = :intLockId",
+              new MapSqlParameterSource()
+                  .addValue("blockedByExtLockId", blockedBy.getExtLockId())
+                  .addValue("blockedByIntLockId", blockedBy.getIntLockId())
+                  .addValue("extLockId", extLockId)
+                  .addValue("intLockId", intLockId));
+
+          if (updCnt != 1) {
+            LOG.error("Failure to update lock (extLockId={}, intLockId={}) with the blocking lock's IDs " +
+                "(extLockId={}, intLockId={})", extLockId, intLockId, blockedBy.getExtLockId(), blockedBy.getIntLockId());
+            throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnId) + " "
+                + JavaUtils.lockIdToString(extLockId) + " " + intLockId);
+          }
+          response.setState(LockState.WAITING);
+          return false;
+        } catch (MetaException e) {
+          throw new MetaWrapperException(e);
+        }
+      }
+      return true;
+    }), "This never should be null, it's just to suppress warnings");
+
+    if (!success) {
+      return response;
+    }
+
+    // If here, there were no locks that would block any item from 'locksBeingChecked' - acquire them all
+    acquire(jdbcResource, locksBeingChecked);
+
+    // We acquired all the locks, so commit and return acquired.
+    LOG.debug("Successfully acquired locks: {}", locksBeingChecked);
+    response.setState(LockState.ACQUIRED);
+    return response;
+  }
+
+  // NEVER call this function without first calling heartbeat(long, long)
+  private List getLocksFromLockId(MultiDataSourceJdbcResource jdbcResource, long extLockId) throws MetaException {
+    List locks = jdbcResource.execute(new GetLocksByLockId(extLockId, -1, jdbcResource.getSqlGenerator()));
+    if (locks.isEmpty()) {
+      throw new MetaException("This should never happen!  We already " +
+          "checked the lock(" + JavaUtils.lockIdToString(extLockId) + ") existed but now we can't find it!");
+    }
+    LOG.debug("Found {} locks for extLockId={}. Locks: {}", locks.size(), extLockId, locks);
+    return locks;
+  }
+
+  private void acquire(MultiDataSourceJdbcResource jdbcResource, List locksBeingChecked)
+      throws NoSuchLockException, MetaException {
+    if (CollectionUtils.isEmpty(locksBeingChecked)) {
+      return;
+    }
+    long txnId = locksBeingChecked.get(0).getTxnId();
+    long extLockId = locksBeingChecked.get(0).getExtLockId();
+    int rc = jdbcResource.getJdbcTemplate().update("UPDATE \"HIVE_LOCKS\" SET \"HL_LOCK_STATE\" = :state, " +
+        //if lock is part of txn, heartbeat info is in txn record
+        "\"HL_LAST_HEARTBEAT\" = " + (TxnUtils.isValidTxn(txnId) ? 0 : getEpochFn(jdbcResource.getDatabaseProduct())) +
+        ",\"HL_ACQUIRED_AT\" = " + getEpochFn(jdbcResource.getDatabaseProduct()) +
+        ",\"HL_BLOCKEDBY_EXT_ID\"=NULL,\"HL_BLOCKEDBY_INT_ID\"=NULL" +
+        " WHERE \"HL_LOCK_EXT_ID\" = :extLockId", 
+        new MapSqlParameterSource()
+            .addValue("state", Character.toString(LOCK_ACQUIRED), Types.CHAR)
+            .addValue("extLockId", extLockId));
+    
+    if (rc < locksBeingChecked.size()) {
+      LOG.error("Failure to acquire all locks (acquired: {}, total needed: {}).", rc, locksBeingChecked.size());
+      /*select all locks for this ext ID and see which ones are missing*/
+      Set notFoundIds = locksBeingChecked.stream()
+          .map(lockInfo -> Long.toString(lockInfo.getIntLockId()))
+          .collect(Collectors.toSet());
+      List foundIds = Objects.requireNonNull(jdbcResource.getJdbcTemplate().query(
+          "SELECT \"HL_LOCK_INT_ID\" FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = :extLockId",
+          new MapSqlParameterSource().addValue("extLockId", extLockId), rs -> {
+            List ids = new ArrayList<>();
+            while (rs.next()) {
+              ids.add(rs.getString("HL_LOCK_INT_ID"));
+            }
+            return ids;
+          }), "This never should be null, it's just to suppress warnings");
+      
+      foundIds.forEach(notFoundIds::remove);
+      String errorMsg = String.format("No such lock(s): (%s: %s) %s",
+          JavaUtils.lockIdToString(extLockId), String.join(", ", notFoundIds), JavaUtils.txnIdToString(txnId));
+      throw new NoSuchLockException(errorMsg);
+    }
+  }
+
+  static class WriteSetInfo {
+    String database;
+    String table;
+    String partition;
+    Long txnId;
+    Long commitId;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CleanTxnToWriteIdTableFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CleanTxnToWriteIdTableFunction.java
similarity index 90%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CleanTxnToWriteIdTableFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CleanTxnToWriteIdTableFunction.java
index 3268f0d2ef18..6457cd27f04a 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CleanTxnToWriteIdTableFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CleanTxnToWriteIdTableFunction.java
@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
 import org.slf4j.Logger;
@@ -47,21 +48,19 @@ public class CleanTxnToWriteIdTableFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CleanupRecordsFunction.class);
+  private static final EnumSet HIVE_OBJECT_TYPES = 
+      EnumSet.of(HiveObjectType.DATABASE, HiveObjectType.TABLE, HiveObjectType.PARTITION);
+
+  @SuppressWarnings("squid:S3599")
+  //language=SQL
+  private static final Map, String> DELETE_COMMANDS =
+      new LinkedHashMap, String>() {{
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HIVE_OBJECT_TYPES.contains(hiveObjectType),
+            "DELETE FROM \"TXN_COMPONENTS\" WHERE " +
+                "\"TC_DATABASE\" = :dbName AND " +
+                "(\"TC_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+                "(\"TC_PARTITION\" = :partName OR :partName IS NULL)");
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HIVE_OBJECT_TYPES.contains(hiveObjectType),
+            "DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE " +
+                "(\"CTC_DATABASE\" = :dbName) AND " +
+                "(\"CTC_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+                "(\"CTC_PARTITION\" = :partName OR :partName IS NULL)");
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HIVE_OBJECT_TYPES.contains(hiveObjectType),
+            "DELETE FROM \"COMPACTION_QUEUE\" WHERE " +
+                "\"CQ_DATABASE\" = :dbName AND " +
+                "(\"CQ_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+                "(\"CQ_PARTITION\" = :partName OR :partName IS NULL) AND " +
+                "(\"CQ_TXN_ID\" != :txnId OR :txnId IS NULL)");
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HIVE_OBJECT_TYPES.contains(hiveObjectType),
+            "DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE " +
+                "\"CC_DATABASE\" = :dbName AND " +
+                "(\"CC_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+                "(\"CC_PARTITION\" = :partName OR :partName IS NULL)");
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HiveObjectType.DATABASE.equals(hiveObjectType) ||
+                (HiveObjectType.TABLE.equals(hiveObjectType) && !keepTxnToWriteIdMetaData),
+            "DELETE FROM \"TXN_TO_WRITE_ID\" WHERE " +
+                "\"T2W_DATABASE\" = :dbName AND " +
+                "(\"T2W_TABLE\" = :tableName OR :tableName IS NULL)");
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HiveObjectType.DATABASE.equals(hiveObjectType) ||
+                HiveObjectType.TABLE.equals(hiveObjectType) && !keepTxnToWriteIdMetaData,
+            "DELETE FROM \"NEXT_WRITE_ID\" WHERE " +
+                "\"NWI_DATABASE\" = :dbName AND " +
+                "(\"NWI_TABLE\" = :tableName OR :tableName IS NULL)");
+        put((hiveObjectType, keepTxnToWriteIdMetaData) -> HIVE_OBJECT_TYPES.contains(hiveObjectType),
+            "DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE " +
+                "\"CMC_DATABASE\" = :dbName AND " +
+                "(\"CMC_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+                "(\"CMC_PARTITION\" = :partName OR :partName IS NULL)");
+      }};
+
+  private final HiveObjectType type;
+  private final Database db;
+  private final Table table;
+  private final Iterator partitionIterator;
+  private final String defaultCatalog;
+  private final boolean keepTxnToWriteIdMetaData;
+  private final Long txnId;
+
+  public CleanupRecordsFunction(HiveObjectType type, Database db, Table table, Iterator partitionIterator,
+                                String defaultCatalog, boolean keepTxnToWriteIdMetaData, Long txnId) {
+    this.type = type;
+    this.db = db;
+    this.table = table;
+    this.partitionIterator = partitionIterator;
+    this.defaultCatalog = defaultCatalog;
+    this.keepTxnToWriteIdMetaData = keepTxnToWriteIdMetaData;
+    this.txnId = txnId;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    // cleanup should be done only for objects belonging to default catalog
+    List paramSources = new ArrayList<>();
+
+    switch (type) {
+      case DATABASE: {
+        if (!defaultCatalog.equals(db.getCatalogName())) {
+          LOG.debug("Skipping cleanup because db: " + db.getName() + " belongs to catalog "
+              + "other than default catalog: " + db.getCatalogName());
+          return null;
+        }
+        paramSources.add(new MapSqlParameterSource()
+            .addValue("dbName", db.getName().toLowerCase())
+            .addValue("tableName", null, Types.VARCHAR)
+            .addValue("partName", null, Types.VARCHAR)
+            .addValue("txnId", txnId, Types.BIGINT));
+        break;
+      }
+      case TABLE: {
+        if (!defaultCatalog.equals(table.getCatName())) {
+          LOG.debug("Skipping cleanup because table: {} belongs to catalog other than default catalog: {}",
+              table.getTableName(), table.getCatName());
+          return null;
+        }
+        paramSources.add(new MapSqlParameterSource()
+            .addValue("dbName", table.getDbName().toLowerCase())
+            .addValue("tableName", table.getTableName().toLowerCase(), Types.VARCHAR)
+            .addValue("partName", null, Types.VARCHAR)
+            .addValue("txnId", null, Types.BIGINT));
+        break;
+      }
+      case PARTITION: {
+        if (!defaultCatalog.equals(table.getCatName())) {
+          LOG.debug("Skipping cleanup because partitions belong to catalog other than default catalog: {}",
+              table.getCatName());
+          return null;
+        }
+        List partCols = table.getPartitionKeys();  // partition columns
+        List partVals;                                  // partition values
+        while (partitionIterator.hasNext()) {
+          Partition partition = partitionIterator.next();
+          partVals = partition.getValues();
+          paramSources.add(new MapSqlParameterSource()
+              .addValue("dbName", table.getDbName().toLowerCase())
+              .addValue("tableName", table.getTableName().toLowerCase(), Types.VARCHAR)
+              .addValue("partName", Warehouse.makePartName(partCols, partVals), Types.VARCHAR)
+              .addValue("txnId", null, Types.BIGINT));
+        }
+      }
+    }
+
+    try {
+      for (MapSqlParameterSource parameterSource : paramSources) {
+        for (Map.Entry, String> item : DELETE_COMMANDS.entrySet()) {
+          if (item.getKey().apply(type, keepTxnToWriteIdMetaData)) {
+            jdbcResource.getJdbcTemplate().update(item.getValue(), parameterSource);
+          }
+        }
+      }
+    } catch (DataAccessException e) {
+      Throwable ex = e;
+      do {
+        String message = ex.getMessage();
+        if (StringUtils.isNotBlank(message) && message.contains("does not exist")) {
+          LOG.warn("Cannot perform cleanup since metastore table does not exist");
+          return null;
+        }
+        ex = ex.getCause();
+      } while (ex != null);
+      throw e;
+    }
+    return null;
+  }
+
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CommitTxnFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CommitTxnFunction.java
new file mode 100644
index 000000000000..6b5b81aace90
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CommitTxnFunction.java
@@ -0,0 +1,636 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hive.common.repl.ReplConst;
+import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
+import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.ReplLastIdInfo;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.CommitCompactionEvent;
+import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.OperationType;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.DeleteReplTxnMapEntryCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertCompletedTxnComponentsCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveTxnsFromMinHistoryLevelCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveWriteIdsFromMinHistoryCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.FindTxnStateHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetCompactionInfoHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetHighWaterMarkHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetOpenTxnTypeAndLockHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.TargetTxnIdListHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.RollbackException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+public class CommitTxnFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CommitTxnFunction.class);
+
+  private final CommitTxnRequest rqst;
+  private final List transactionalListeners;
+
+  public CommitTxnFunction(CommitTxnRequest rqst, List transactionalListeners) {
+    this.rqst = rqst;
+    this.transactionalListeners = transactionalListeners;
+  }
+
+  @Override
+  public TxnType execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, NoSuchTxnException, TxnAbortedException {
+    char isUpdateDelete = 'N';
+    long txnid = rqst.getTxnid();
+    long sourceTxnId = -1;
+
+    boolean isReplayedReplTxn = TxnType.REPL_CREATED.equals(rqst.getTxn_type());
+    boolean isHiveReplTxn = rqst.isSetReplPolicy() && TxnType.DEFAULT.equals(rqst.getTxn_type());
+    // Get the current TXN
+    TransactionContext context = jdbcResource.getTransactionManager().getActiveTransaction();
+    Long commitId = null;
+
+    if (rqst.isSetReplLastIdInfo()) {
+      updateReplId(jdbcResource, rqst.getReplLastIdInfo());
+    }
+
+    if (isReplayedReplTxn) {
+      assert (rqst.isSetReplPolicy());
+      sourceTxnId = rqst.getTxnid();
+      List targetTxnIds = jdbcResource.execute(new TargetTxnIdListHandler(rqst.getReplPolicy(), Collections.singletonList(sourceTxnId)));
+      if (targetTxnIds.isEmpty()) {
+        // Idempotent case where txn was already closed or commit txn event received without
+        // corresponding open txn event.
+        LOG.info("Target txn id is missing for source txn id : {} and repl policy {}", sourceTxnId,
+            rqst.getReplPolicy());
+        throw new RollbackException(null);
+      }
+      assert targetTxnIds.size() == 1;
+      txnid = targetTxnIds.get(0);
+    }
+
+    /**
+     * Runs at READ_COMMITTED with S4U on TXNS row for "txnid".  S4U ensures that no other
+     * operation can change this txn (such acquiring locks). While lock() and commitTxn()
+     * should not normally run concurrently (for same txn) but could due to bugs in the client
+     * which could then corrupt internal transaction manager state.  Also competes with abortTxn().
+     */
+    TxnType txnType = jdbcResource.execute(new GetOpenTxnTypeAndLockHandler(jdbcResource.getSqlGenerator(), txnid));
+    if (txnType == null) {
+      //if here, txn was not found (in expected state)
+      TxnStatus actualTxnStatus = jdbcResource.execute(new FindTxnStateHandler(txnid));
+      if (actualTxnStatus == TxnStatus.COMMITTED) {
+        if (isReplayedReplTxn) {
+          // in case of replication, idempotent is taken care by getTargetTxnId
+          LOG.warn("Invalid state COMMITTED for transactions started using replication replay task");
+        }
+        /**
+         * This makes the operation idempotent
+         * (assume that this is most likely due to retry logic)
+         */
+        LOG.info("Nth commitTxn({}) msg", JavaUtils.txnIdToString(txnid));
+        return null;
+      }
+      TxnUtils.raiseTxnUnexpectedState(actualTxnStatus, txnid);
+    }
+
+    String conflictSQLSuffix = "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" IN (" +
+        OperationType.UPDATE + "," + OperationType.DELETE + ")";
+    long tempCommitId = TxnUtils.generateTemporaryId();
+
+    if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) {
+      new AcquireTxnLockFunction(false).execute(jdbcResource);
+      commitId = jdbcResource.execute(new GetHighWaterMarkHandler());
+
+    } else if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn) {
+      String writeSetInsertSql = "INSERT INTO \"WRITE_SET\" (\"WS_DATABASE\", \"WS_TABLE\", \"WS_PARTITION\"," +
+          "   \"WS_TXNID\", \"WS_COMMIT_ID\", \"WS_OPERATION_TYPE\")" +
+          " SELECT DISTINCT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_TXNID\", " + tempCommitId + ", \"TC_OPERATION_TYPE\" ";
+
+      boolean isUpdateOrDelete = Boolean.TRUE.equals(jdbcResource.getJdbcTemplate().query(
+          jdbcResource.getSqlGenerator().addLimitClause(1, "\"TC_OPERATION_TYPE\" " + conflictSQLSuffix),
+          ResultSet::next));
+      
+      if (isUpdateOrDelete) {
+        isUpdateDelete = 'Y';
+        //if here it means currently committing txn performed update/delete and we should check WW conflict
+        /**
+         * "select distinct" is used below because
+         * 1. once we get to multi-statement txns, we only care to record that something was updated once
+         * 2. if {@link #addDynamicPartitions(AddDynamicPartitions)} is retried by caller it may create
+         *  duplicate entries in TXN_COMPONENTS
+         * but we want to add a PK on WRITE_SET which won't have unique rows w/o this distinct
+         * even if it includes all of its columns
+         *
+         * First insert into write_set using a temporary commitID, which will be updated in a separate call,
+         * see: {@link #updateWSCommitIdAndCleanUpMetadata(Statement, long, TxnType, Long, long)}}.
+         * This should decrease the scope of the S4U lock on the next_txn_id table.
+         */
+        Object undoWriteSetForCurrentTxn = context.createSavepoint();
+        jdbcResource.getJdbcTemplate().update(
+            writeSetInsertSql + (TxnHandler.ConfVars.useMinHistoryLevel() ? conflictSQLSuffix :
+            "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"= :txnId AND \"TC_OPERATION_TYPE\" <> :type"),
+            new MapSqlParameterSource()
+                .addValue("txnId", txnid)
+                .addValue("type", OperationType.COMPACT.getSqlConst()));
+
+        /**
+         * This S4U will mutex with other commitTxn() and openTxns().
+         * -1 below makes txn intervals look like [3,3] [4,4] if all txns are serial
+         * Note: it's possible to have several txns have the same commit id.  Suppose 3 txns start
+         * at the same time and no new txns start until all 3 commit.
+         * We could've incremented the sequence for commitId as well but it doesn't add anything functionally.
+         */
+        new AcquireTxnLockFunction(false).execute(jdbcResource);
+        commitId = jdbcResource.execute(new GetHighWaterMarkHandler());
+
+        if (!rqst.isExclWriteEnabled()) {
+          /**
+           * see if there are any overlapping txns that wrote the same element, i.e. have a conflict
+           * Since entire commit operation is mutexed wrt other start/commit ops,
+           * committed.ws_commit_id <= current.ws_commit_id for all txns
+           * thus if committed.ws_commit_id < current.ws_txnid, transactions do NOT overlap
+           * For example, [17,20] is committed, [6,80] is being committed right now - these overlap
+           * [17,20] committed and [21,21] committing now - these do not overlap.
+           * [17,18] committed and [18,19] committing now - these overlap  (here 18 started while 17 was still running)
+           */
+          WriteSetInfo info = checkForWriteConflict(jdbcResource, txnid);
+          if (info != null) {
+            //found a conflict, so let's abort the txn
+            String committedTxn = "[" + JavaUtils.txnIdToString(info.txnId) + "," + info.committedCommitId + "]";
+            StringBuilder resource = new StringBuilder(info.database).append("/").append(info.table);
+            if (info.partition != null) {
+              resource.append('/').append(info.partition);
+            }
+            String msg = "Aborting [" + JavaUtils.txnIdToString(txnid) + "," + commitId + "]" + " due to a write conflict on " + resource +
+                " committed by " + committedTxn + " " + info.currentOperationType + "/" + info.committedOperationType;
+            //remove WRITE_SET info for current txn since it's about to abort
+            context.rollbackToSavepoint(undoWriteSetForCurrentTxn);
+            LOG.info(msg);
+            //todo: should make abortTxns() write something into TXNS.TXN_META_INFO about this
+            if (new AbortTxnsFunction(Collections.singletonList(txnid), false, false, 
+                isReplayedReplTxn, TxnErrorMsg.ABORT_WRITE_CONFLICT).execute(jdbcResource) != 1) {
+              throw new IllegalStateException(msg + " FAILED!");
+            }
+            throw new TxnAbortedException(msg);
+          }
+        }
+      } else if (!TxnHandler.ConfVars.useMinHistoryLevel()) {
+        jdbcResource.getJdbcTemplate().update(writeSetInsertSql + "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" = :txnId AND \"TC_OPERATION_TYPE\" <> :type",
+            new MapSqlParameterSource()
+                .addValue("txnId", txnid)
+                .addValue("type", OperationType.COMPACT.getSqlConst()));
+        commitId = jdbcResource.execute(new GetHighWaterMarkHandler());
+      }
+    } else {
+      /*
+       * current txn didn't update/delete anything (may have inserted), so just proceed with commit
+       *
+       * We only care about commit id for write txns, so for RO (when supported) txns we don't
+       * have to mutex on NEXT_TXN_ID.
+       * Consider: if RO txn is after a W txn, then RO's openTxns() will be mutexed with W's
+       * commitTxn() because both do S4U on NEXT_TXN_ID and thus RO will see result of W txn.
+       * If RO < W, then there is no reads-from relationship.
+       * In replication flow we don't expect any write write conflict as it should have been handled at source.
+       */
+      assert true;
+    }
+
+
+    if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn && !MetaStoreServerUtils.isCompactionTxn(txnType)) {
+      moveTxnComponentsToCompleted(jdbcResource, txnid, isUpdateDelete);
+    } else if (isReplayedReplTxn) {
+      if (rqst.isSetWriteEventInfos() && !rqst.getWriteEventInfos().isEmpty()) {
+        jdbcResource.execute(new InsertCompletedTxnComponentsCommand(txnid, isUpdateDelete, rqst.getWriteEventInfos()));
+      }
+      jdbcResource.execute(new DeleteReplTxnMapEntryCommand(sourceTxnId, rqst.getReplPolicy()));
+    }
+    updateWSCommitIdAndCleanUpMetadata(jdbcResource, txnid, txnType, commitId, tempCommitId);
+    jdbcResource.execute(new RemoveTxnsFromMinHistoryLevelCommand(ImmutableList.of(txnid)));
+    jdbcResource.execute(new RemoveWriteIdsFromMinHistoryCommand(ImmutableList.of(txnid)));
+    if (rqst.isSetKeyValue()) {
+      updateKeyValueAssociatedWithTxn(jdbcResource, rqst);
+    }
+
+    if (!isHiveReplTxn) {
+      createCommitNotificationEvent(jdbcResource, txnid , txnType);
+    }
+
+    LOG.debug("Going to commit");
+
+    if (MetastoreConf.getBoolVar(jdbcResource.getConf(), MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
+      Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_COMMITTED_TXNS).inc();
+    }
+    return txnType;
+  }
+
+  private void updateReplId(MultiDataSourceJdbcResource jdbcResource, ReplLastIdInfo replLastIdInfo) throws MetaException {
+    String lastReplId = Long.toString(replLastIdInfo.getLastReplId());
+    String catalog = replLastIdInfo.isSetCatalog() ? normalizeIdentifier(replLastIdInfo.getCatalog()) :
+        MetaStoreUtils.getDefaultCatalog(jdbcResource.getConf());
+    String db = normalizeIdentifier(replLastIdInfo.getDatabase());
+    String table = replLastIdInfo.isSetTable() ? normalizeIdentifier(replLastIdInfo.getTable()) : null;
+    List partList = replLastIdInfo.isSetPartitionList() ? replLastIdInfo.getPartitionList() : null;
+
+    String s = jdbcResource.getSqlGenerator().getDbProduct().getPrepareTxnStmt();
+    if (s != null) {
+      jdbcResource.getJdbcTemplate().execute(s, ps -> null);
+    }
+
+    // not used select for update as it will be updated by single thread only from repl load
+    long dbId = updateDatabaseProp(jdbcResource, catalog, db, ReplConst.REPL_TARGET_TABLE_PROPERTY, lastReplId);
+    if (table != null) {
+      long tableId = updateTableProp(jdbcResource, catalog, db, dbId, table, ReplConst.REPL_TARGET_TABLE_PROPERTY, lastReplId);
+      if (partList != null && !partList.isEmpty()) {
+        updatePartitionProp(jdbcResource, tableId, partList, ReplConst.REPL_TARGET_TABLE_PROPERTY, lastReplId);
+      }
+    }
+  }
+
+  private long updateDatabaseProp(MultiDataSourceJdbcResource jdbcResource, String catalog, String database, 
+                                  String prop, String propValue) throws MetaException {
+    String query = 
+        "SELECT d.\"DB_ID\", dp.\"PARAM_KEY\", dp.\"PARAM_VALUE\" FROM \"DATABASE_PARAMS\" dp\n" +
+            "RIGHT JOIN \"DBS\" d ON dp.\"DB_ID\" = d.\"DB_ID\" " +
+        "WHERE \"NAME\" = :dbName  and \"CTLG_NAME\" = :catalog";
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Going to execute query <" + query + ">");
+    }
+    DbEntityParam dbEntityParam = jdbcResource.getJdbcTemplate().query(query,
+        new MapSqlParameterSource()
+            .addValue("dbName", database)
+            .addValue("catalog", catalog),
+        //no row means database no found
+        rs -> rs.next()
+            ? new DbEntityParam(rs.getLong("DB_ID"), rs.getString("PARAM_KEY"), rs.getString("PARAM_VALUE"))
+            : null);
+
+    if (dbEntityParam == null) {
+      throw new MetaException("DB with name " + database + " does not exist in catalog " + catalog);
+    }
+
+    //TODO: would be better to replace with MERGE or UPSERT
+    String command;
+    if (dbEntityParam.key == null) {
+      command = "INSERT INTO \"DATABASE_PARAMS\" VALUES (:dbId, :key, :value)";
+    } else if (!dbEntityParam.value.equals(propValue)) {
+      command = "UPDATE \"DATABASE_PARAMS\" SET \"PARAM_VALUE\" = :value WHERE \"DB_ID\" = :dbId AND \"PARAM_KEY\" = :key";
+    } else {
+      LOG.info("Database property: {} with value: {} already updated for db: {}", prop, propValue, database);
+      return dbEntityParam.id;      
+    }
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Updating {} for db: {}  using command {}", prop, database, command);
+    }
+    SqlParameterSource params = new MapSqlParameterSource()
+        .addValue("dbId", dbEntityParam.id)
+        .addValue("key", prop)
+        .addValue("value", propValue);
+    if (jdbcResource.getJdbcTemplate().update(command, params) != 1) {
+      //only one row insert or update should happen
+      throw new RuntimeException("DATABASE_PARAMS is corrupted for database: " + database);
+    }
+    return dbEntityParam.id;
+  }
+
+  private long updateTableProp(MultiDataSourceJdbcResource jdbcResource, String catalog, String db, long dbId,
+                                  String table, String prop, String propValue) throws MetaException {
+    String query = 
+        "SELECT t.\"TBL_ID\", tp.\"PARAM_KEY\", tp.\"PARAM_VALUE\" FROM \"TABLE_PARAMS\" tp " +
+            "RIGHT JOIN \"TBLS\" t ON tp.\"TBL_ID\" = t.\"TBL_ID\" " +
+        "WHERE t.\"DB_ID\" = :dbId AND t.\"TBL_NAME\" = :tableName";
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Going to execute query <" + query + ">");
+    }
+    DbEntityParam dbEntityParam = jdbcResource.getJdbcTemplate().query(query,
+        new MapSqlParameterSource()
+            .addValue("tableName", table)
+            .addValue("dbId", dbId),
+        //no row means table no found
+        rs -> rs.next() 
+            ? new DbEntityParam(rs.getLong("TBL_ID"), rs.getString("PARAM_KEY"), rs.getString("PARAM_VALUE")) 
+            : null);
+
+    if (dbEntityParam == null) {
+      throw new MetaException("Table with name " + table + " does not exist in db " + catalog + "." + db);
+    }
+
+    //TODO: would be better to replace with MERGE or UPSERT
+    String command;
+    if (dbEntityParam.key == null) {
+      command = "INSERT INTO \"TABLE_PARAMS\" VALUES (:tblId, :key, :value)";
+    } else if (!dbEntityParam.value.equals(propValue)) {
+      command = "UPDATE \"TABLE_PARAMS\" SET \"PARAM_VALUE\" = :value WHERE \"TBL_ID\" = :dbId AND \"PARAM_KEY\" = :key";
+    } else {
+      LOG.info("Database property: {} with value: {} already updated for db: {}", prop, propValue, db);
+      return dbEntityParam.id;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Updating {} for table: {}  using command {}", prop, table, command);
+    }
+    SqlParameterSource params = new MapSqlParameterSource()
+        .addValue("tblId", dbEntityParam.id)
+        .addValue("key", prop)
+        .addValue("value", propValue);
+    if (jdbcResource.getJdbcTemplate().update(command, params) != 1) {
+      //only one row insert or update should happen
+      throw new RuntimeException("TABLE_PARAMS is corrupted for table: " + table);
+    }
+    return dbEntityParam.id;
+  }
+  
+  private void updatePartitionProp(MultiDataSourceJdbcResource jdbcResource, long tableId,
+                                   List partList, String prop, String propValue) {
+    List queries = new ArrayList<>();
+    StringBuilder prefix = new StringBuilder();
+    StringBuilder suffix = new StringBuilder();
+    //language=SQL
+    prefix.append(
+        "SELECT p.\"PART_ID\", pp.\"PARAM_KEY\", pp.\"PARAM_VALUE\" FROM \"PARTITION_PARAMS\" pp\n" +
+        "RIGHT JOIN \"PARTITIONS\" p ON pp.\"PART_ID\" = p.\"PART_ID\" WHERE p.\"TBL_ID\" = :tblId AND pp.\"PARAM_KEY\" = :key");
+
+    // Populate the complete query with provided prefix and suffix
+    TxnUtils.buildQueryWithINClauseStrings(jdbcResource.getConf(), queries, prefix, suffix, partList,
+        "\"PART_NAME\"", true, false);
+    SqlParameterSource params = new MapSqlParameterSource()
+        .addValue("tblId", tableId)
+        .addValue("key", prop);
+    List partitionParams = new ArrayList<>();
+    for(String query : queries) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to execute query <" + query + ">");
+      }
+      jdbcResource.getJdbcTemplate().query(query, params,
+          (ResultSet rs) -> {
+            while (rs.next()) {
+              partitionParams.add(new DbEntityParam(rs.getLong("PART_ID"), rs.getString("PARAM_KEY"), rs.getString("PARAM_VALUE")));
+            }
+          });
+    }
+
+    //TODO: would be better to replace with MERGE or UPSERT
+    int maxBatchSize = MetastoreConf.getIntVar(jdbcResource.getConf(), MetastoreConf.ConfVars.JDBC_MAX_BATCH_SIZE);
+    //all insert in one batch
+    int[][] inserts = jdbcResource.getJdbcTemplate().getJdbcTemplate().batchUpdate(
+        "INSERT INTO \"PARTITION_PARAMS\" VALUES (?, ?, ?)",
+        partitionParams.stream().filter(p -> p.key == null).collect(Collectors.toList()), maxBatchSize,
+        (ps, argument) -> {
+          ps.setLong(1, argument.id);
+          ps.setString(2, argument.key);
+          ps.setString(3, propValue);
+        });
+    //all update in one batch
+    int[][] updates =jdbcResource.getJdbcTemplate().getJdbcTemplate().batchUpdate(
+        "UPDATE \"PARTITION_PARAMS\" SET \"PARAM_VALUE\" = ? WHERE \"PART_ID\" = ? AND \"PARAM_KEY\" = ?",
+        partitionParams.stream().filter(p -> p.key != null && !propValue.equals(p.value)).collect(Collectors.toList()), maxBatchSize,
+        (ps, argument) -> {
+          ps.setString(1, propValue);
+          ps.setLong(2, argument.id);
+          ps.setString(3, argument.key);
+        });
+
+    if (Arrays.stream(inserts).flatMapToInt(IntStream::of).sum() + Arrays.stream(updates).flatMapToInt(IntStream::of).sum() != partList.size()) {
+      throw new RuntimeException("PARTITION_PARAMS is corrupted, update failed");      
+    }    
+  }
+
+  private WriteSetInfo checkForWriteConflict(MultiDataSourceJdbcResource jdbcResource, long txnid) throws MetaException {
+    String writeConflictQuery = jdbcResource.getSqlGenerator().addLimitClause(1, 
+        "\"COMMITTED\".\"WS_TXNID\", \"COMMITTED\".\"WS_COMMIT_ID\", " +
+        "\"COMMITTED\".\"WS_DATABASE\", \"COMMITTED\".\"WS_TABLE\", \"COMMITTED\".\"WS_PARTITION\", " +
+        "\"CUR\".\"WS_COMMIT_ID\" \"CUR_WS_COMMIT_ID\", \"CUR\".\"WS_OPERATION_TYPE\" \"CUR_OP\", " +
+        "\"COMMITTED\".\"WS_OPERATION_TYPE\" \"COMMITTED_OP\" FROM \"WRITE_SET\" \"COMMITTED\" INNER JOIN \"WRITE_SET\" \"CUR\" " +
+        "ON \"COMMITTED\".\"WS_DATABASE\"=\"CUR\".\"WS_DATABASE\" AND \"COMMITTED\".\"WS_TABLE\"=\"CUR\".\"WS_TABLE\" " +
+        //For partitioned table we always track writes at partition level (never at table)
+        //and for non partitioned - always at table level, thus the same table should never
+        //have entries with partition key and w/o
+        "AND (\"COMMITTED\".\"WS_PARTITION\"=\"CUR\".\"WS_PARTITION\" OR (\"COMMITTED\".\"WS_PARTITION\" IS NULL AND \"CUR\".\"WS_PARTITION\" IS NULL)) " +
+        "WHERE \"CUR\".\"WS_TXNID\" <= \"COMMITTED\".\"WS_COMMIT_ID\" " + //txns overlap; could replace ws_txnid
+        // with txnid, though any decent DB should infer this
+        "AND \"CUR\".\"WS_TXNID\"= :txnId " + //make sure RHS of join only has rows we just inserted as
+        // part of this commitTxn() op
+        "AND \"COMMITTED\".\"WS_TXNID\" <> :txnId " + //and LHS only has committed txns
+        //U+U and U+D and D+D is a conflict and we don't currently track I in WRITE_SET at all
+        //it may seem like D+D should not be in conflict but consider 2 multi-stmt txns
+        //where each does "delete X + insert X, where X is a row with the same PK.  This is
+        //equivalent to an update of X but won't be in conflict unless D+D is in conflict.
+        //The same happens when Hive splits U=I+D early so it looks like 2 branches of a
+        //multi-insert stmt (an Insert and a Delete branch).  It also 'feels'
+        // un-serializable to allow concurrent deletes
+        "AND (\"COMMITTED\".\"WS_OPERATION_TYPE\" IN(:opUpdate, :opDelete) " +
+        "AND \"CUR\".\"WS_OPERATION_TYPE\" IN(:opUpdate, :opDelete))");
+    LOG.debug("Going to execute query: <{}>", writeConflictQuery);
+    return jdbcResource.getJdbcTemplate().query(writeConflictQuery,
+        new MapSqlParameterSource()
+            .addValue("txnId", txnid)
+            .addValue("opUpdate", OperationType.UPDATE.getSqlConst())
+            .addValue("opDelete", OperationType.DELETE.getSqlConst()),
+        (ResultSet rs) -> {
+          if(rs.next()) {
+            return new WriteSetInfo(rs.getLong("WS_TXNID"), rs.getLong("CUR_WS_COMMIT_ID"),
+                rs.getLong("WS_COMMIT_ID"), rs.getString("CUR_OP"), rs.getString("COMMITTED_OP"),
+                rs.getString("WS_DATABASE"), rs.getString("WS_TABLE"), rs.getString("WS_PARTITION"));
+          } else {
+            return null;
+          }
+        });
+  }
+
+  private void moveTxnComponentsToCompleted(MultiDataSourceJdbcResource jdbcResource, long txnid, char isUpdateDelete) {
+    // Move the record from txn_components into completed_txn_components so that the compactor
+    // knows where to look to compact.
+    String query = "INSERT INTO \"COMPLETED_TXN_COMPONENTS\" (\"CTC_TXNID\", \"CTC_DATABASE\", " +
+        "\"CTC_TABLE\", \"CTC_PARTITION\", \"CTC_WRITEID\", \"CTC_UPDATE_DELETE\") SELECT \"TC_TXNID\", " +
+        "\"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_WRITEID\", :flag FROM \"TXN_COMPONENTS\" " +
+        "WHERE \"TC_TXNID\" = :txnid AND \"TC_OPERATION_TYPE\" <> :type";
+    //we only track compactor activity in TXN_COMPONENTS to handle the case where the
+    //compactor txn aborts - so don't bother copying it to COMPLETED_TXN_COMPONENTS
+    LOG.debug("Going to execute insert <{}>", query);
+    int affectedRows = jdbcResource.getJdbcTemplate().update(query,
+        new MapSqlParameterSource()
+            .addValue("flag", Character.toString(isUpdateDelete), Types.CHAR)
+            .addValue("txnid", txnid)
+            .addValue("type", OperationType.COMPACT.getSqlConst(), Types.CHAR));
+
+    if (affectedRows < 1) {
+      //this can be reasonable for an empty txn START/COMMIT or read-only txn
+      //also an IUD with DP that didn't match any rows.
+      LOG.info("Expected to move at least one record from txn_components to "
+          + "completed_txn_components when committing txn! {}", JavaUtils.txnIdToString(txnid));
+    }
+  }
+
+  private void updateKeyValueAssociatedWithTxn(MultiDataSourceJdbcResource jdbcResource, CommitTxnRequest rqst) {
+    if (!rqst.getKeyValue().getKey().startsWith(TxnStore.TXN_KEY_START)) {
+      String errorMsg = "Error updating key/value in the sql backend with"
+          + " txnId=" + rqst.getTxnid() + ","
+          + " tableId=" + rqst.getKeyValue().getTableId() + ","
+          + " key=" + rqst.getKeyValue().getKey() + ","
+          + " value=" + rqst.getKeyValue().getValue() + "."
+          + " key should start with " + TxnStore.TXN_KEY_START + ".";
+      LOG.warn(errorMsg);
+      throw new IllegalArgumentException(errorMsg);
+    }
+    String query = "UPDATE \"TABLE_PARAMS\" SET \"PARAM_VALUE\" = :value WHERE \"TBL_ID\" = :id AND \"PARAM_KEY\" = :key";
+    LOG.debug("Going to execute update <{}>", query);
+    int affectedRows = jdbcResource.getJdbcTemplate().update(query,
+        new MapSqlParameterSource()
+            .addValue("value", rqst.getKeyValue().getValue())
+            .addValue("id", rqst.getKeyValue().getTableId())
+            .addValue("key", rqst.getKeyValue().getKey()));
+    
+    if (affectedRows != 1) {
+      String errorMsg = "Error updating key/value in the sql backend with"
+          + " txnId=" + rqst.getTxnid() + ","
+          + " tableId=" + rqst.getKeyValue().getTableId() + ","
+          + " key=" + rqst.getKeyValue().getKey() + ","
+          + " value=" + rqst.getKeyValue().getValue() + "."
+          + " Only one row should have been affected but "
+          + affectedRows + " rows where affected.";
+      LOG.warn(errorMsg);
+      throw new IllegalStateException(errorMsg);
+    }
+  }
+
+  /**
+   * See overridden method in CompactionTxnHandler also.
+   */
+  private void updateWSCommitIdAndCleanUpMetadata(MultiDataSourceJdbcResource jdbcResource, long txnid, TxnType txnType,
+                                                    Long commitId, long tempId) throws MetaException {
+    List queryBatch = new ArrayList<>(6);
+    // update write_set with real commitId
+    if (commitId != null) {
+      queryBatch.add("UPDATE \"WRITE_SET\" SET \"WS_COMMIT_ID\" = " + commitId +
+          " WHERE \"WS_COMMIT_ID\" = " + tempId + " AND \"WS_TXNID\" = " + txnid);
+    }
+    // clean up txn related metadata
+    if (txnType != TxnType.READ_ONLY) {
+      queryBatch.add("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" = " + txnid);
+    }
+    queryBatch.add("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_TXNID\" = " + txnid);
+    // DO NOT remove the transaction from the TXN table, the cleaner will remove it when appropriate
+    queryBatch.add("UPDATE \"TXNS\" SET \"TXN_STATE\" = " + TxnStatus.COMMITTED + " WHERE \"TXN_ID\" = " + txnid);
+    if (txnType == TxnType.MATER_VIEW_REBUILD) {
+      queryBatch.add("DELETE FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE \"MRL_TXN_ID\" = " + txnid);
+    }
+    if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) {
+      queryBatch.add("UPDATE \"COMPACTION_QUEUE\" SET \"CQ_NEXT_TXN_ID\" = " + commitId + ", \"CQ_COMMIT_TIME\" = " +
+          getEpochFn(jdbcResource.getDatabaseProduct()) + " WHERE \"CQ_TXN_ID\" = " + txnid);
+    }
+    
+    // execute all in one batch
+    jdbcResource.getJdbcTemplate().getJdbcTemplate().batchUpdate(queryBatch.toArray(new String[0]));
+  }
+
+  /**
+   * Create Notifiaction Events on txn commit
+   * @param txnid committed txn
+   * @param txnType transaction type
+   * @throws MetaException ex
+   */
+  private void createCommitNotificationEvent(MultiDataSourceJdbcResource jdbcResource, long txnid, TxnType txnType)
+      throws MetaException {
+    if (transactionalListeners != null) {
+      MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+          EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, txnType), jdbcResource.getConnection(), jdbcResource.getSqlGenerator());
+
+      CompactionInfo compactionInfo = jdbcResource.execute(new GetCompactionInfoHandler(txnid, true));
+      if (compactionInfo != null) {
+        MetaStoreListenerNotifier
+            .notifyEventWithDirectSql(transactionalListeners, EventMessage.EventType.COMMIT_COMPACTION,
+                new CommitCompactionEvent(txnid, compactionInfo), jdbcResource.getConnection(), jdbcResource.getSqlGenerator());
+      } else {
+        LOG.warn("No compaction queue record found for Compaction type transaction commit. txnId:" + txnid);
+      }
+      
+    }
+  }
+
+  private static class DbEntityParam {
+    final long id;
+    final String key;
+    final String value;
+
+    public DbEntityParam(long id, String key, String value) {
+      this.id = id;
+      this.key = key;
+      this.value = value;
+    }
+  }
+  
+  private static class WriteSetInfo {
+    final long txnId;
+    final long currentCommitId;
+    final long committedCommitId;
+    final String currentOperationType;
+    final String committedOperationType;
+    final String database;
+    final String table;
+    final String partition;
+
+    public WriteSetInfo(long txnId, long currentCommitId, long committedCommitId, 
+                        String currentOperationType, String committedOperationType, 
+                        String database, String table, String partition) {
+      this.txnId = txnId;
+      this.currentCommitId = currentCommitId;
+      this.committedCommitId = committedCommitId;
+      this.currentOperationType = currentOperationType;
+      this.committedOperationType = committedOperationType;
+      this.database = database;
+      this.table = table;
+      this.partition = partition;
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CompactFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CompactFunction.java
new file mode 100644
index 000000000000..5331dc9562ed
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/CompactFunction.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertCompactionRequestCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.INITIATED_RESPONSE;
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.INITIATED_STATE;
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.READY_FOR_CLEANING;
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.REFUSED_RESPONSE;
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.WORKING_STATE;
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getFullTableName;
+
+public class CompactFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactFunction.class);
+  
+  private final CompactionRequest rqst;
+  private final long openTxnTimeOutMillis;
+  private final TxnStore.MutexAPI mutexAPI;
+
+  public CompactFunction(CompactionRequest rqst, long openTxnTimeOutMillis, TxnStore.MutexAPI mutexAPI) {
+    this.rqst = rqst;
+    this.openTxnTimeOutMillis = openTxnTimeOutMillis;
+    this.mutexAPI = mutexAPI;
+  }
+
+  @Override
+  public CompactionResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    // Put a compaction request in the queue.
+    TxnStore.MutexAPI.LockHandle handle = null;
+    try {
+      /**
+       * MUTEX_KEY.CompactionScheduler lock ensures that there is only 1 entry in
+       * Initiated/Working state for any resource.  This ensures that we don't run concurrent
+       * compactions for any resource.
+       */
+      handle = mutexAPI.acquireLock(TxnStore.MUTEX_KEY.CompactionScheduler.name());
+
+      GetValidWriteIdsRequest request = new GetValidWriteIdsRequest(
+          Collections.singletonList(getFullTableName(rqst.getDbname(), rqst.getTablename())));
+      final ValidCompactorWriteIdList tblValidWriteIds = TxnUtils.createValidCompactWriteIdList(
+          new GetValidWriteIdsFunction(request, openTxnTimeOutMillis).execute(jdbcResource).getTblValidWriteIds().get(0));
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ValidCompactWriteIdList: {}", tblValidWriteIds.writeToString());
+      }
+
+      NamedParameterJdbcTemplate npJdbcTemplate = jdbcResource.getJdbcTemplate();
+      Pair existing = npJdbcTemplate.query(
+          "SELECT \"CQ_ID\", \"CQ_STATE\" FROM \"COMPACTION_QUEUE\" WHERE (\"CQ_STATE\" IN(:states) OR" +
+              " (\"CQ_STATE\" = :readyForCleaningState AND \"CQ_HIGHEST_WRITE_ID\" = :highestWriteId)) AND" +
+              " \"CQ_DATABASE\"= :dbName AND \"CQ_TABLE\"= :tableName AND ((:partition is NULL AND \"CQ_PARTITION\" IS NULL) OR \"CQ_PARTITION\" = :partition)",
+          new MapSqlParameterSource()
+              .addValue("states", Arrays.asList(Character.toString(INITIATED_STATE), Character.toString(WORKING_STATE)))
+              .addValue("readyForCleaningState", READY_FOR_CLEANING, Types.VARCHAR)
+              .addValue("highestWriteId", tblValidWriteIds.getHighWatermark())
+              .addValue("dbName", rqst.getDbname())
+              .addValue("tableName", rqst.getTablename())
+              .addValue("partition", rqst.getPartitionname(), Types.VARCHAR),
+          rs -> {
+            if (rs.next()) {
+              return new ImmutablePair<>(rs.getLong("CQ_ID"), rs.getString("CQ_STATE"));
+            }
+            return null;
+          });
+      if (existing != null) {
+        String state = CompactionState.fromSqlConst(existing.getValue()).toString();
+        LOG.info("Ignoring request to compact {}/{}/{} since it is already {} with id={}", rqst.getDbname(),
+            rqst.getTablename(), rqst.getPartitionname(), state, existing.getKey());
+        CompactionResponse resp = new CompactionResponse(-1, REFUSED_RESPONSE, false);
+        resp.setErrormessage("Compaction is already scheduled with state='" + state + "' and id=" + existing.getKey());
+        return resp;
+      }
+
+      long id = new GenerateCompactionQueueIdFunction().execute(jdbcResource);
+      jdbcResource.execute(new InsertCompactionRequestCommand(id, CompactionState.INITIATED, rqst));
+      return new CompactionResponse(id, INITIATED_RESPONSE, true);
+    } finally {
+      if (handle != null) {
+        handle.releaseLocks();
+      }
+    }
+  }
+
+
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnqueueLockFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnqueueLockFunction.java
new file mode 100644
index 000000000000..28a69cc6e619
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnqueueLockFunction.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.txn.TxnLockManager;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertHiveLocksCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertTxnComponentsCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetOpenTxnTypeAndLockHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetWriteIdsHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.dao.EmptyResultDataAccessException;
+import org.springframework.jdbc.core.JdbcTemplate;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.Objects;
+
+public class EnqueueLockFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(EnqueueLockFunction.class);  
+
+  private static final String INCREMENT_NEXT_LOCK_ID_QUERY = "UPDATE \"NEXT_LOCK_ID\" SET \"NL_NEXT\" = %s";
+  private static final String UPDATE_HIVE_LOCKS_EXT_ID_QUERY = "UPDATE \"HIVE_LOCKS\" SET \"HL_LOCK_EXT_ID\" = %s " +
+      "WHERE \"HL_LOCK_EXT_ID\" = %s";
+
+  private final LockRequest lockRequest;
+
+  public EnqueueLockFunction(LockRequest lockRequest) {
+    this.lockRequest = lockRequest;
+  }
+
+  /**
+   * This enters locks into the queue in {@link org.apache.hadoop.hive.metastore.txn.entities.LockInfo#LOCK_WAITING} mode.
+   * Isolation Level Notes:
+   * 1. We use S4U (withe read_committed) to generate the next (ext) lock id.  This serializes
+   * any 2 {@code enqueueLockWithRetry()} calls.
+   * 2. We use S4U on the relevant TXNS row to block any concurrent abort/commit/etc. operations
+   * @see TxnLockManager#checkLock(long, long, boolean, boolean)
+   */
+  @Override
+  public Long execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, TxnAbortedException, NoSuchTxnException {
+    long txnid = lockRequest.getTxnid();
+    if (TxnUtils.isValidTxn(txnid)) {
+      //this also ensures that txn is still there in expected state
+      TxnType txnType = jdbcResource.execute(new GetOpenTxnTypeAndLockHandler(jdbcResource.getSqlGenerator(), txnid));
+      if (txnType == null) {
+        new EnsureValidTxnFunction(txnid).execute(jdbcResource);
+      }
+    }
+        /* Insert txn components and hive locks (with a temp extLockId) first, before getting the next lock ID in a select-for-update.
+           This should minimize the scope of the S4U and decrease the table lock duration. */
+    if (txnid > 0) {
+      jdbcResource.execute(new InsertTxnComponentsCommand(lockRequest, jdbcResource.execute(new GetWriteIdsHandler(lockRequest))));
+    }
+    long tempExtLockId = TxnUtils.generateTemporaryId();
+    jdbcResource.execute(new InsertHiveLocksCommand(lockRequest, tempExtLockId));
+
+    /* Get the next lock id.
+     * This has to be atomic with adding entries to HIVE_LOCK entries (1st add in W state) to prevent a race.
+     * Suppose ID gen is a separate txn and 2 concurrent lock() methods are running.  1st one generates nl_next=7,
+     * 2nd nl_next=8.  Then 8 goes first to insert into HIVE_LOCKS and acquires the locks.  Then 7 unblocks,
+     * and add it's W locks, but it won't see locks from 8 since to be 'fair' {@link #checkLock(java.sql.Connection, long)}
+     * doesn't block on locks acquired later than one it's checking*/
+    long extLockId = getNextLockIdForUpdate(jdbcResource);
+    incrementLockIdAndUpdateHiveLocks(jdbcResource.getJdbcTemplate().getJdbcTemplate(), extLockId, tempExtLockId);
+
+    jdbcResource.getTransactionManager().getActiveTransaction().createSavepoint();
+
+    return extLockId;
+  }
+
+  private long getNextLockIdForUpdate(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    String s = jdbcResource.getSqlGenerator().addForUpdateClause("SELECT \"NL_NEXT\" FROM \"NEXT_LOCK_ID\"");
+    LOG.debug("Going to execute query <{}>", s);
+    
+    try {
+      return Objects.requireNonNull(
+          jdbcResource.getJdbcTemplate().queryForObject(s, new MapSqlParameterSource(), Long.class),
+          "This never should be null, it's just to suppress warnings");
+    } catch (EmptyResultDataAccessException e) {
+      LOG.error("Failure to get next lock ID for update! SELECT query returned empty ResultSet.");
+      throw new MetaException("Transaction tables not properly " +
+          "initialized, no record found in next_lock_id");      
+    }
+  }
+
+  private void incrementLockIdAndUpdateHiveLocks(JdbcTemplate jdbcTemplate, long extLockId, long tempId) {    
+    String incrCmd = String.format(INCREMENT_NEXT_LOCK_ID_QUERY, (extLockId + 1));
+    // update hive locks entries with the real EXT_LOCK_ID (replace temp ID)
+    String updateLocksCmd = String.format(UPDATE_HIVE_LOCKS_EXT_ID_QUERY, extLockId, tempId);
+    LOG.debug("Going to execute updates in batch: <{}>, and <{}>", incrCmd, updateLocksCmd);
+    jdbcTemplate.batchUpdate(incrCmd, updateLocksCmd);
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnsureValidTxnFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnsureValidTxnFunction.java
new file mode 100644
index 000000000000..05c5ba58388c
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/EnsureValidTxnFunction.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+public class EnsureValidTxnFunction implements TransactionalFunction {
+
+  private final long txnId;
+
+  public EnsureValidTxnFunction(long txnId) {
+    this.txnId = txnId;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, NoSuchTxnException, TxnAbortedException {
+    SqlParameterSource paramSource = new MapSqlParameterSource().addValue("txnId", txnId);
+    // We need to check whether this transaction is valid and open
+    TxnStatus status = jdbcResource.getJdbcTemplate().query("SELECT \"TXN_STATE\" FROM \"TXNS\" WHERE \"TXN_ID\" = :txnId",
+        paramSource, rs -> rs.next() ? TxnStatus.fromString(rs.getString("TXN_STATE")) : null);
+
+    if (status == null) {
+      // todo: add LIMIT 1 instead of count - should be more efficient
+      boolean alreadyCommitted = Boolean.TRUE.equals(jdbcResource.getJdbcTemplate().query("SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_TXNID\" = :txnId",
+          paramSource, rs -> {
+            // todo: strictly speaking you can commit an empty txn, thus 2nd conjunct is wrong but
+            // only possible for for multi-stmt txns
+            return rs.next() && rs.getInt(1) > 0;
+          }));
+
+      if (alreadyCommitted) {
+        // makes the message more informative - helps to find bugs in client code
+        throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnId)
+            + " is already committed.");
+      }
+      throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnId));
+    } else {
+      if (status == TxnStatus.ABORTED) {
+        throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnId)
+            + " already aborted");
+        // todo: add time of abort, which is not currently tracked. Requires schema change
+      }
+    }
+    return null;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/FindPotentialCompactionsFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/FindPotentialCompactionsFunction.java
similarity index 87%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/FindPotentialCompactionsFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/FindPotentialCompactionsFunction.java
index e23c16cb6feb..7d9bdbf7e2df 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/FindPotentialCompactionsFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/FindPotentialCompactionsFunction.java
@@ -15,12 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.AbortedTxnHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.CompactionCandidateHandler;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
 
@@ -48,7 +50,7 @@ public Set execute(MultiDataSourceJdbcResource jdbcResource) thr
     Set candidates = new HashSet<>(jdbcResource.execute(
         new CompactionCandidateHandler(lastChecked, fetchSize)));
     int remaining = fetchSize - candidates.size();
-    if (collectAbortedTxns) {
+    if (collectAbortedTxns && remaining > 0) {
       candidates.addAll(jdbcResource.execute(new AbortedTxnHandler(abortedTimeThreshold, abortedThreshold, remaining)));
     }
     return candidates;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GenerateCompactionQueueIdFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GenerateCompactionQueueIdFunction.java
new file mode 100644
index 000000000000..6195d5196a77
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GenerateCompactionQueueIdFunction.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+public class GenerateCompactionQueueIdFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateCompactionQueueIdFunction.class);
+
+  public GenerateCompactionQueueIdFunction() {}
+
+  @Override
+  public Long execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException  {
+    // Get the id for the next entry in the queue
+    String sql = jdbcResource.getSqlGenerator().addForUpdateClause("SELECT \"NCQ_NEXT\" FROM \"NEXT_COMPACTION_QUEUE_ID\"");
+    LOG.debug("going to execute SQL <{}>", sql);
+
+    Long allocatedId = jdbcResource.getJdbcTemplate().query(sql, rs -> {
+      if (!rs.next()) {
+        throw new IllegalStateException("Transaction tables not properly initiated, "
+            + "no record found in next_compaction_queue_id");
+      }
+      long id = rs.getLong(1);
+
+      int count = jdbcResource.getJdbcTemplate().update("UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = :newId WHERE \"NCQ_NEXT\" = :id",
+          new MapSqlParameterSource()
+              .addValue("id", id)
+              .addValue("newId", id + 1));
+
+      if (count != 1) {
+        //TODO: Eliminate this id generation by implementing: https://issues.apache.org/jira/browse/HIVE-27121
+        LOG.info("The returned compaction ID ({}) already taken, obtaining new", id);
+        return null;
+      }
+      return id;
+    });
+    if (allocatedId == null) {
+      return execute(jdbcResource);
+    } else {
+      return allocatedId;
+    }
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetMaterializationInvalidationInfoFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetMaterializationInvalidationInfoFunction.java
new file mode 100644
index 000000000000..f88fea62813b
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetMaterializationInvalidationInfoFunction.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hadoop.hive.common.MaterializationSnapshot;
+import org.apache.hadoop.hive.common.ValidReadTxnList;
+import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.Materialization;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+public class GetMaterializationInvalidationInfoFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GetMaterializationInvalidationInfoFunction.class);
+
+  private final CreationMetadata creationMetadata;
+  private final String validTxnListStr;
+
+  public GetMaterializationInvalidationInfoFunction(CreationMetadata creationMetadata, String validTxnListStr) {
+    this.creationMetadata = creationMetadata;
+    this.validTxnListStr = validTxnListStr;
+  }
+
+  @Override
+  public Materialization execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    if (creationMetadata.getTablesUsed().isEmpty()) {
+      // Bail out
+      LOG.warn("Materialization creation metadata does not contain any table");
+      return null;
+    }
+
+    // We are composing a query that returns a single row if an update happened after
+    // the materialization was created. Otherwise, query returns 0 rows.
+
+    // Parse validReaderWriteIdList from creation metadata
+    MaterializationSnapshot mvSnapshot = MaterializationSnapshot.fromJson(creationMetadata.getValidTxnList());
+    if (mvSnapshot.getTableSnapshots() != null && !mvSnapshot.getTableSnapshots().isEmpty()) {
+      // Incremental rebuild of MVs on Iceberg sources is not supported.
+      return null;
+    }
+    final ValidTxnWriteIdList validReaderWriteIdList = new ValidTxnWriteIdList(mvSnapshot.getValidTxnList());
+
+    // Parse validTxnList
+    final ValidReadTxnList currentValidTxnList = new ValidReadTxnList(validTxnListStr);
+    // Get the valid write id list for the tables in current state
+    final List currentTblValidWriteIdsList = new ArrayList<>();
+    for (String fullTableName : creationMetadata.getTablesUsed()) {
+      currentTblValidWriteIdsList.add(new GetValidWriteIdsForTableFunction(currentValidTxnList, fullTableName).execute(jdbcResource));
+    }
+    final ValidTxnWriteIdList currentValidReaderWriteIdList = TxnCommonUtils.createValidTxnWriteIdList(
+        currentValidTxnList.getHighWatermark(), currentTblValidWriteIdsList);
+
+    List params = new ArrayList<>();
+    StringBuilder queryUpdateDelete = new StringBuilder();
+    StringBuilder queryCompletedCompactions = new StringBuilder();
+    StringBuilder queryCompactionQueue = new StringBuilder();
+    // compose a query that select transactions containing an update...
+    queryUpdateDelete.append("SELECT \"CTC_UPDATE_DELETE\" FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_UPDATE_DELETE\" ='Y' AND (");
+    queryCompletedCompactions.append("SELECT 1 FROM \"COMPLETED_COMPACTIONS\" WHERE (");
+    queryCompactionQueue.append("SELECT 1 FROM \"COMPACTION_QUEUE\" WHERE (");
+    int i = 0;
+    for (String fullyQualifiedName : creationMetadata.getTablesUsed()) {
+      ValidWriteIdList tblValidWriteIdList =
+          validReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName);
+      if (tblValidWriteIdList == null) {
+        LOG.warn("ValidWriteIdList for table {} not present in creation metadata, this should not happen", fullyQualifiedName);
+        return null;
+      }
+
+      // First, we check whether the low watermark has moved for any of the tables.
+      // If it has, we return true, since it is not incrementally refreshable, e.g.,
+      // one of the commits that are not available may be an update/delete.
+      ValidWriteIdList currentTblValidWriteIdList =
+          currentValidReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName);
+      if (currentTblValidWriteIdList == null) {
+        LOG.warn("Current ValidWriteIdList for table {} not present in creation metadata, this should not happen", fullyQualifiedName);
+        return null;
+      }
+      if (!Objects.equals(currentTblValidWriteIdList.getMinOpenWriteId(), tblValidWriteIdList.getMinOpenWriteId())) {
+        LOG.debug("Minimum open write id do not match for table {}", fullyQualifiedName);
+        return null;
+      }
+
+      // ...for each of the tables that are part of the materialized view,
+      // where the transaction had to be committed after the materialization was created...
+      if (i != 0) {
+        queryUpdateDelete.append("OR");
+        queryCompletedCompactions.append("OR");
+        queryCompactionQueue.append("OR");
+      }
+      String[] names = TxnUtils.getDbTableName(fullyQualifiedName);
+      assert (names.length == 2);
+      queryUpdateDelete.append(" (\"CTC_DATABASE\"=? AND \"CTC_TABLE\"=?");
+      queryCompletedCompactions.append(" (\"CC_DATABASE\"=? AND \"CC_TABLE\"=?");
+      queryCompactionQueue.append(" (\"CQ_DATABASE\"=? AND \"CQ_TABLE\"=?");
+      params.add(names[0]);
+      params.add(names[1]);
+      queryUpdateDelete.append(" AND (\"CTC_WRITEID\" > " + tblValidWriteIdList.getHighWatermark());
+      queryCompletedCompactions.append(" AND (\"CC_HIGHEST_WRITE_ID\" > " + tblValidWriteIdList.getHighWatermark());
+      queryUpdateDelete.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " :
+          " OR \"CTC_WRITEID\" IN(" + StringUtils.join(",",
+              Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ) ");
+      queryCompletedCompactions.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " :
+          " OR \"CC_HIGHEST_WRITE_ID\" IN(" + StringUtils.join(",",
+              Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ) ");
+      queryUpdateDelete.append(") ");
+      queryCompletedCompactions.append(") ");
+      queryCompactionQueue.append(") ");
+      i++;
+    }
+    // ... and where the transaction has already been committed as per snapshot taken
+    // when we are running current query
+    queryUpdateDelete.append(") AND \"CTC_TXNID\" <= " + currentValidTxnList.getHighWatermark());
+    queryUpdateDelete.append(currentValidTxnList.getInvalidTransactions().length == 0 ? " " :
+        " AND \"CTC_TXNID\" NOT IN(" + StringUtils.join(",",
+            Arrays.asList(ArrayUtils.toObject(currentValidTxnList.getInvalidTransactions()))) + ") ");
+    queryCompletedCompactions.append(")");
+    queryCompactionQueue.append(") ");
+
+    boolean hasUpdateDelete = executeBoolean(jdbcResource, queryUpdateDelete.toString(), params,
+        "Unable to retrieve materialization invalidation information: completed transaction components.");
+
+    // Execute query
+    queryCompletedCompactions.append(" UNION ");
+    queryCompletedCompactions.append(queryCompactionQueue.toString());
+    List paramsTwice = new ArrayList<>(params);
+    paramsTwice.addAll(params);
+    boolean hasCompaction = executeBoolean(jdbcResource, queryCompletedCompactions.toString(), paramsTwice,
+        "Unable to retrieve materialization invalidation information: compactions");
+
+    return new Materialization(hasUpdateDelete, hasCompaction);
+  }
+
+  private boolean executeBoolean(MultiDataSourceJdbcResource jdbcResource, String queryText, List params, String errorMessage) throws MetaException {
+    try (PreparedStatement pst = jdbcResource.getSqlGenerator().prepareStmtWithParameters(jdbcResource.getConnection(), queryText, params)) {
+      LOG.debug("Going to execute query <{}>", queryText);
+      pst.setMaxRows(1);
+      try (ResultSet rs = pst.executeQuery()) {
+        return rs.next();
+      }
+    } catch (SQLException ex) {
+      LOG.warn(errorMessage, ex);
+      throw new MetaException(errorMessage + " " + StringUtils.stringifyException(ex));
+    }
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsForTableFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsForTableFunction.java
new file mode 100644
index 000000000000..a7f2b64606ba
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsForTableFunction.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.ResultSetExtractor;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.nio.ByteBuffer;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class GetValidWriteIdsForTableFunction implements TransactionalFunction {
+
+  private final ValidTxnList validTxnList;
+  private final String fullTableName;
+
+  public GetValidWriteIdsForTableFunction(ValidTxnList validTxnList, String fullTableName) {
+    this.validTxnList = validTxnList;
+    this.fullTableName = fullTableName;
+  }
+
+  // Method to get the Valid write ids list for the given table
+  // Input fullTableName is expected to be of format .
+  @Override
+  public TableValidWriteIds execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    String[] names = TxnUtils.getDbTableName(fullTableName);
+    assert (names.length == 2);
+
+    // Find the writeId high watermark based upon txnId high watermark. If found, then, need to
+    // traverse through all write Ids less than writeId HWM to make exceptions list.
+    // The writeHWM = min(NEXT_WRITE_ID.nwi_next-1, max(TXN_TO_WRITE_ID.t2w_writeid under txnHwm))
+    long writeIdHwm = Objects.requireNonNull(jdbcResource.getJdbcTemplate().query(
+        "SELECT MAX(\"T2W_WRITEID\") FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_TXNID\" <= :txnHwm "
+            + " AND \"T2W_DATABASE\" = :db AND \"T2W_TABLE\" = :table",
+        new MapSqlParameterSource()
+            .addValue("txnHwm", validTxnList.getHighWatermark())
+            .addValue("db", names[0])
+            .addValue("table", names[1]), new HwmExtractor()));
+
+    // If no writeIds allocated by txns under txnHwm, then find writeHwm from NEXT_WRITE_ID.
+    if (writeIdHwm <= 0) {
+      // Need to subtract 1 as nwi_next would be the next write id to be allocated but we need highest
+      // allocated write id.
+      writeIdHwm = Objects.requireNonNull(jdbcResource.getJdbcTemplate().query(
+          "SELECT \"NWI_NEXT\" -1 FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = :db AND \"NWI_TABLE\" = :table",
+          new MapSqlParameterSource()
+              .addValue("db", names[0])
+              .addValue("table", names[1]), new HwmExtractor()));
+    }
+
+    final List invalidWriteIdList = new ArrayList<>();
+    final BitSet abortedBits = new BitSet();
+    final AtomicLong minOpenWriteId = new AtomicLong(Long.MAX_VALUE);
+    final AtomicBoolean foundValidUncompactedWrite = new AtomicBoolean(false);
+
+    // As writeIdHwm is known, query all writeIds under the writeId HWM.
+    // If any writeId under HWM is allocated by txn > txnId HWM or belongs to open/aborted txns,
+    // then will be added to invalid list. The results should be sorted in ascending order based
+    // on write id. The sorting is needed as exceptions list in ValidWriteIdList would be looked-up
+    // using binary search.
+    jdbcResource.getJdbcTemplate().query(
+        "SELECT \"T2W_TXNID\", \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_WRITEID\" <= :writeIdHwm" +
+            " AND \"T2W_DATABASE\" = :db AND \"T2W_TABLE\" = :table ORDER BY \"T2W_WRITEID\" ASC",
+        new MapSqlParameterSource()
+            .addValue("writeIdHwm", writeIdHwm)
+            .addValue("db", names[0])
+            .addValue("table", names[1]), rs -> {
+          while (rs.next()) {
+            long txnId = rs.getLong(1);
+            long writeId = rs.getLong(2);
+            if (validTxnList.isTxnValid(txnId)) {
+              // Skip if the transaction under evaluation is already committed.
+              foundValidUncompactedWrite.set(true);
+              continue;
+            }
+            // The current txn is either in open or aborted state.
+            // Mark the write ids state as per the txn state.
+            invalidWriteIdList.add(writeId);
+            if (validTxnList.isTxnAborted(txnId)) {
+              abortedBits.set(invalidWriteIdList.size() - 1);
+            } else {
+              minOpenWriteId.set(Math.min(minOpenWriteId.get(), writeId));
+            }
+          }
+          return null;
+        });
+
+    // If we have compacted writes and some invalid writes on the table,
+    // return the lowest invalid write as a writeIdHwm and set it as invalid.
+    if (!foundValidUncompactedWrite.get()) {
+      long writeId = invalidWriteIdList.isEmpty() ? -1 : invalidWriteIdList.get(0);
+      invalidWriteIdList.clear();
+      abortedBits.clear();
+
+      if (writeId != -1) {
+        invalidWriteIdList.add(writeId);
+        writeIdHwm = writeId;
+        if (writeId != minOpenWriteId.get()) {
+          abortedBits.set(0);
+        }
+      }
+    }
+    ByteBuffer byteBuffer = ByteBuffer.wrap(abortedBits.toByteArray());
+    TableValidWriteIds owi = new TableValidWriteIds(fullTableName, writeIdHwm, invalidWriteIdList, byteBuffer);
+    if (minOpenWriteId.get() < Long.MAX_VALUE) {
+      owi.setMinOpenWriteId(minOpenWriteId.get());
+    }
+    return owi;
+  }
+
+  private static class HwmExtractor implements ResultSetExtractor {
+
+    @Override
+    public Long extractData(ResultSet rs) throws SQLException, DataAccessException {
+      if (rs.next()) {
+        return rs.getLong(1);
+      } else {
+        // Need to initialize to 0 to make sure if nobody modified this table, then current txn
+        // shouldn't read any data.
+        // If there is a conversion from non-acid to acid table, then by default 0 would be assigned as
+        // writeId for data from non-acid table and so writeIdHwm=0 would ensure those data are readable by any txns.
+        return 0L;
+      }
+    }
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsFunction.java
new file mode 100644
index 000000000000..2992963e95ac
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/GetValidWriteIdsFunction.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.common.ValidReadTxnList;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetOpenTxnsListHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class GetValidWriteIdsFunction implements TransactionalFunction {
+
+  private final GetValidWriteIdsRequest rqst;
+  private final long openTxnTimeOutMillis;
+
+  public GetValidWriteIdsFunction(GetValidWriteIdsRequest rqst, long openTxnTimeOutMillis) {
+    this.rqst = rqst;
+    this.openTxnTimeOutMillis = openTxnTimeOutMillis;
+  }
+
+  public GetValidWriteIdsResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    ValidTxnList validTxnList;
+
+    // We should prepare the valid write ids list based on validTxnList of current txn.
+    // If no txn exists in the caller, then they would pass null for validTxnList and so it is
+    // required to get the current state of txns to make validTxnList
+    if (rqst.isSetValidTxnList()) {
+      assert !rqst.isSetWriteId();
+      validTxnList = new ValidReadTxnList(rqst.getValidTxnList());
+    } else if (rqst.isSetWriteId()) {
+      validTxnList = TxnCommonUtils.createValidReadTxnList(getOpenTxns(jdbcResource), 
+          getTxnId(jdbcResource, rqst.getFullTableNames().get(0), rqst.getWriteId())); 
+    } else {
+      // Passing 0 for currentTxn means, this validTxnList is not wrt to any txn
+      validTxnList = TxnCommonUtils.createValidReadTxnList(getOpenTxns(jdbcResource), 0);
+    }
+
+    // Get the valid write id list for all the tables read by the current txn
+    List tblValidWriteIdsList = new ArrayList<>();
+    for (String fullTableName : rqst.getFullTableNames()) {
+      tblValidWriteIdsList.add(new GetValidWriteIdsForTableFunction(validTxnList, fullTableName).execute(jdbcResource));
+    }
+    return new GetValidWriteIdsResponse(tblValidWriteIdsList);
+  }
+
+  private long getTxnId(MultiDataSourceJdbcResource jdbcResource, String fullTableName, Long writeId) throws MetaException {
+    String[] names = TxnUtils.getDbTableName(fullTableName);
+    assert (names.length == 2);
+    Long txnId = jdbcResource.getJdbcTemplate().query(
+        "SELECT \"T2W_TXNID\" FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = :db AND "
+            + "\"T2W_TABLE\" = :table AND \"T2W_WRITEID\" = :writeId",
+        new MapSqlParameterSource()
+            .addValue("db", names[0])
+            .addValue("table", names[1])
+            .addValue("writeId", writeId),
+        (ResultSet rs) -> {
+          if(rs.next()) {
+            long id = rs.getLong(1);
+            return rs.wasNull() ? null : id;
+          }
+          return null;
+        });
+    if (txnId == null) {
+      throw new MetaException("invalid write id " + writeId + " for table " + fullTableName);
+    }
+    return txnId;
+  }
+  
+  private GetOpenTxnsResponse getOpenTxns(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    return jdbcResource.execute(new GetOpenTxnsListHandler(false, openTxnTimeOutMillis))
+        .toOpenTxnsResponse(Collections.singletonList(TxnType.READ_ONLY));
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatLockFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatLockFunction.java
new file mode 100644
index 000000000000..779de4256bfc
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatLockFunction.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+/**
+ * Heartbeats on the lock table.  This commits, so do not enter it with any state.
+ * Should not be called on a lock that belongs to transaction.
+ */
+public class HeartbeatLockFunction implements TransactionalFunction {
+
+  private final long extLockId;
+
+  public HeartbeatLockFunction(long extLockId) {
+    this.extLockId = extLockId;
+  }
+  
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) 
+      throws MetaException, NoSuchTxnException, TxnAbortedException, NoSuchLockException {
+    // If the lock id is 0, then there are no locks in this heartbeat
+    if (extLockId == 0) {
+      return null;
+    }
+    
+    int rc = jdbcResource.getJdbcTemplate().update("UPDATE \"HIVE_LOCKS\" SET \"HL_LAST_HEARTBEAT\" = " +
+        getEpochFn(jdbcResource.getDatabaseProduct()) + " WHERE \"HL_LOCK_EXT_ID\" = :extLockId",
+        new MapSqlParameterSource().addValue("extLockId", extLockId));
+    if (rc < 1) {
+      throw new NoSuchLockException("No such lock: " + JavaUtils.lockIdToString(extLockId));
+    }
+    jdbcResource.getTransactionManager().getActiveTransaction().createSavepoint();    
+    return null;
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnFunction.java
new file mode 100644
index 000000000000..979900a7e273
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnFunction.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.Types;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+/**
+ * Heartbeats on the txn table. This commits, so do not enter it with any state. 
+ */
+public class HeartbeatTxnFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HeartbeatTxnFunction.class);
+
+  private final long txnId;
+
+  public HeartbeatTxnFunction(long txnId) {
+    this.txnId = txnId;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException, NoSuchTxnException, TxnAbortedException, NoSuchLockException {
+    // If the txnid is 0, then there are no transactions in this heartbeat
+    if (txnId == 0) {
+      return null;
+    }
+    
+    int rc = jdbcResource.getJdbcTemplate().update(
+        "UPDATE \"TXNS\" SET \"TXN_LAST_HEARTBEAT\" = " + getEpochFn(jdbcResource.getDatabaseProduct()) +
+        " WHERE \"TXN_ID\" = :txnId AND \"TXN_STATE\" = :state", 
+        new MapSqlParameterSource()
+            .addValue("txnId", txnId)
+            .addValue("state", TxnStatus.OPEN.getSqlConst(), Types.CHAR));
+    
+    if (rc < 1) {
+      new EnsureValidTxnFunction(txnId).execute(jdbcResource); // This should now throw some useful exception.
+      LOG.error("Can neither heartbeat txn (txnId={}) nor confirm it as invalid.", txnId);
+      throw new NoSuchTxnException("No such txn: " + txnId);
+    }
+    
+    LOG.debug("Successfully heartbeated for txnId={}", txnId);
+    jdbcResource.getTransactionManager().getActiveTransaction().createSavepoint();
+    return null;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnRangeFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnRangeFunction.java
new file mode 100644
index 000000000000..ff3c801b643c
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/HeartbeatTxnRangeFunction.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class HeartbeatTxnRangeFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HeartbeatTxnRangeFunction.class);
+
+  private final HeartbeatTxnRangeRequest rqst;
+
+  public HeartbeatTxnRangeFunction(HeartbeatTxnRangeRequest rqst) {
+    this.rqst = rqst;
+  }
+
+  @Override
+  public HeartbeatTxnRangeResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    HeartbeatTxnRangeResponse rsp = new HeartbeatTxnRangeResponse();
+    Set nosuch = new HashSet<>();
+    Set aborted = new HashSet<>();
+    rsp.setNosuch(nosuch);
+    rsp.setAborted(aborted);
+    /**
+     * READ_COMMITTED is sufficient since {@link #heartbeatTxn(java.sql.Connection, long)}
+     * only has 1 update statement in it and
+     * we only update existing txns, i.e. nothing can add additional txns that this operation
+     * would care about (which would have required SERIALIZABLE)
+     */
+    /*do fast path first (in 1 statement) if doesn't work, rollback and do the long version*/
+    List queries = new ArrayList<>();
+    int numTxnsToHeartbeat = (int) (rqst.getMax() - rqst.getMin() + 1);
+    List txnIds = new ArrayList<>(numTxnsToHeartbeat);
+    for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) {
+      txnIds.add(txn);
+    }
+    TransactionContext context = jdbcResource.getTransactionManager().getActiveTransaction();
+    Object savePoint = context.createSavepoint();
+    TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), queries,
+        new StringBuilder("UPDATE \"TXNS\" SET \"TXN_LAST_HEARTBEAT\" = " + getEpochFn(jdbcResource.getDatabaseProduct()) +
+            " WHERE \"TXN_STATE\" = " + TxnStatus.OPEN + " AND "),
+        new StringBuilder(""), txnIds, "\"TXN_ID\"", true, false);
+    int updateCnt = 0;
+    for (String query : queries) {
+      LOG.debug("Going to execute update <{}>", query);
+      updateCnt += jdbcResource.getJdbcTemplate().update(query, new MapSqlParameterSource());
+    }
+    if (updateCnt == numTxnsToHeartbeat) {
+      //fast pass worked, i.e. all txns we were asked to heartbeat were Open as expected
+      context.rollbackToSavepoint(savePoint);
+      return rsp;
+    }
+    //if here, do the slow path so that we can return info txns which were not in expected state
+    context.rollbackToSavepoint(savePoint);
+    for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) {
+      try {
+        new HeartbeatTxnFunction(txn).execute(jdbcResource);
+      } catch (NoSuchTxnException e) {
+        nosuch.add(txn);
+      } catch (TxnAbortedException e) {
+        aborted.add(txn);
+      } catch (NoSuchLockException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return rsp;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/LockMaterializationRebuildFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/LockMaterializationRebuildFunction.java
new file mode 100644
index 000000000000..ce7257eae648
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/LockMaterializationRebuildFunction.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.ResultSet;
+import java.time.Instant;
+
+public class LockMaterializationRebuildFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LockMaterializationRebuildFunction.class);
+
+  private final String dbName;
+  private final String tableName;
+  private final long txnId;
+  private final TxnStore.MutexAPI mutexAPI;
+
+  public LockMaterializationRebuildFunction(String dbName, String tableName, long txnId, TxnStore.MutexAPI mutexAPI) {
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.txnId = txnId;
+    this.mutexAPI = mutexAPI;
+  }
+
+  @Override
+  public LockResponse execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Acquiring lock for materialization rebuild with {} for {}",
+          JavaUtils.txnIdToString(txnId), TableName.getDbTable(dbName, tableName));
+    }
+
+    /**
+     * MUTEX_KEY.MaterializationRebuild lock ensures that there is only 1 entry in
+     * Initiated/Working state for any resource. This ensures we do not run concurrent
+     * rebuild operations on any materialization.
+     */
+    try (TxnStore.MutexAPI.LockHandle ignored = mutexAPI.acquireLock(TxnStore.MUTEX_KEY.MaterializationRebuild.name())){
+      MapSqlParameterSource params = new MapSqlParameterSource()
+          .addValue("dbName", dbName)
+          .addValue("tableName", tableName);
+
+      String selectQ = "SELECT \"MRL_TXN_ID\" FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE" +
+          " \"MRL_DB_NAME\" = :dbName AND \"MRL_TBL_NAME\" = :tableName";
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to execute query {}", selectQ);
+      }
+      boolean found = Boolean.TRUE.equals(jdbcResource.getJdbcTemplate().query(selectQ, params, ResultSet::next));
+      
+      if(found) {
+        LOG.info("Ignoring request to rebuild {}/{} since it is already being rebuilt", dbName, tableName);
+        return new LockResponse(txnId, LockState.NOT_ACQUIRED);
+      }
+      
+      String insertQ = "INSERT INTO \"MATERIALIZATION_REBUILD_LOCKS\" " +
+          "(\"MRL_TXN_ID\", \"MRL_DB_NAME\", \"MRL_TBL_NAME\", \"MRL_LAST_HEARTBEAT\") " +
+          "VALUES (:txnId, :dbName, :tableName, " + Instant.now().toEpochMilli() + ")";
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to execute update {}", insertQ);
+      }
+      jdbcResource.getJdbcTemplate().update(insertQ, params.addValue("txnId", txnId));
+      return new LockResponse(txnId, LockState.ACQUIRED);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/MarkCleanedFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MarkCleanedFunction.java
similarity index 90%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/MarkCleanedFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MarkCleanedFunction.java
index 52b8b9a32f85..9f985d7c459e 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/MarkCleanedFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MarkCleanedFunction.java
@@ -15,15 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
-import org.apache.hadoop.hive.metastore.txn.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
-import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.InClauseBatchCommand;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
 import org.slf4j.Logger;
@@ -42,11 +41,9 @@ public class MarkCleanedFunction implements TransactionalFunction {
   private static final Logger LOG = LoggerFactory.getLogger(MarkCleanedFunction.class);
 
   private final CompactionInfo info;
-  private final Configuration conf;
 
-  public MarkCleanedFunction(CompactionInfo info, Configuration conf) {
+  public MarkCleanedFunction(CompactionInfo info) {
     this.info = info;
-    this.conf = conf;
   }
 
   @Override
@@ -111,11 +108,11 @@ public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaExcepti
     }
 
     // Do cleanup of metadata in TXN_COMPONENTS table.
-    removeTxnComponents(info, jdbcTemplate);
+    removeTxnComponents(info, jdbcResource);
     return null;
   }
 
-  private void removeTxnComponents(CompactionInfo info, NamedParameterJdbcTemplate jdbcTemplate) {
+  private void removeTxnComponents(CompactionInfo info, MultiDataSourceJdbcResource jdbcResource) throws MetaException {
     /*
      * compaction may remove data from aborted txns above tc_writeid bit it only guarantees to
      * remove it up to (inclusive) tc_writeid, so it's critical to not remove metadata about
@@ -131,7 +128,7 @@ private void removeTxnComponents(CompactionInfo info, NamedParameterJdbcTemplate
 
     int totalCount = 0;
     if (!info.hasUncompactedAborts && info.highestWriteId != 0) {
-      totalCount = jdbcTemplate.update(
+      totalCount = jdbcResource.getJdbcTemplate().update(
           "DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" IN ( "
               + "SELECT \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = :state) "
               + "AND \"TC_DATABASE\" = :db AND \"TC_TABLE\" = :table "
@@ -139,13 +136,14 @@ private void removeTxnComponents(CompactionInfo info, NamedParameterJdbcTemplate
               + "AND \"TC_WRITEID\" <= :id",
           params.addValue("id", info.highestWriteId));
     } else if (CollectionUtils.isNotEmpty(info.writeIds)) {
-      totalCount = TxnUtils.executeStatementWithInClause(conf, jdbcTemplate,
+      params.addValue("ids", new ArrayList<>(info.writeIds));
+      totalCount = jdbcResource.execute(new InClauseBatchCommand<>(
           "DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" IN ( "
               + "SELECT \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = :state) "
               + "AND \"TC_DATABASE\" = :db AND \"TC_TABLE\" = :table "
               + "AND (:partition is NULL OR \"TC_PARTITION\" = :partition) "
-              + "AND \"TC_WRITEID\" IN (:ids)",
-          params, "ids", new ArrayList<>(info.writeIds), Long::compareTo);
+              + "AND \"TC_WRITEID\" IN (:ids)", 
+          params, "ids", Long::compareTo));
     }
     LOG.debug("Removed {} records from txn_components", totalCount);
   }
@@ -168,11 +166,11 @@ private void removeCompactionAndAbortRetryEntries(CompactionInfo info, NamedPara
           .addValue("table", info.tableName)
           .addValue("type", Character.toString(TxnStore.ABORT_TXN_CLEANUP_TYPE), Types.CHAR)
           .addValue("partition", info.partName, Types.VARCHAR);
-    }    
+    }
 
     LOG.debug("Going to execute update <{}>", query);
-    int rc = jdbcTemplate.update(query,params);
+    int rc = jdbcTemplate.update(query, params);
     LOG.debug("Removed {} records in COMPACTION_QUEUE", rc);
-  }  
-  
+  }
+
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MinOpenTxnIdWaterMarkFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MinOpenTxnIdWaterMarkFunction.java
new file mode 100644
index 000000000000..0407729af01b
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/MinOpenTxnIdWaterMarkFunction.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.OpenTxnTimeoutLowBoundaryTxnIdHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.Types;
+import java.util.Objects;
+
+public class MinOpenTxnIdWaterMarkFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MinOpenTxnIdWaterMarkFunction.class);
+  
+  private final long openTxnTimeOutMillis;
+
+  public MinOpenTxnIdWaterMarkFunction(long openTxnTimeOutMillis) {
+    this.openTxnTimeOutMillis = openTxnTimeOutMillis;
+  }
+
+  @Override
+  public Long execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    /**
+     * We try to find the highest transactionId below everything was committed or aborted.
+     * For that we look for the lowest open transaction in the TXNS and the TxnMinTimeout boundary,
+     * because it is guaranteed there won't be open transactions below that.
+     */
+    long minOpenTxn = Objects.requireNonNull(jdbcResource.getJdbcTemplate().query(
+        "SELECT MIN(\"TXN_ID\") FROM \"TXNS\" WHERE \"TXN_STATE\"= :status",
+        new MapSqlParameterSource().addValue("status", TxnStatus.OPEN.getSqlConst(), Types.CHAR),
+        (ResultSet rs) -> {
+          if (!rs.next()) {
+            throw new IllegalStateException("Scalar query returned no rows?!?!!");
+          }
+          long id = rs.getLong(1);
+          if (rs.wasNull()) {
+            id = Long.MAX_VALUE;
+          }
+          return id;
+        }));
+    long lowWaterMark = jdbcResource.execute(new OpenTxnTimeoutLowBoundaryTxnIdHandler(openTxnTimeOutMillis));
+    LOG.debug("MinOpenTxnIdWaterMark calculated with minOpenTxn {}, lowWaterMark {}", minOpenTxn, lowWaterMark);
+    return Long.min(minOpenTxn, lowWaterMark + 1);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/NextCompactionFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/NextCompactionFunction.java
similarity index 97%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/NextCompactionFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/NextCompactionFunction.java
index cf07b1198d1d..f39ba8b3d147 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/NextCompactionFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/NextCompactionFunction.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.metastore.api.FindNextCompactRequest;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java
new file mode 100644
index 000000000000..69173857f43e
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+public class OnRenameFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(OnRenameFunction.class);
+  
+  //language=SQL
+  private static final String[] UPDATE_COMMANNDS = new String[]{
+      "UPDATE \"TXN_COMPONENTS\" SET " +
+          "\"TC_PARTITION\" = COALESCE(:newPartName, \"TC_PARTITION\"), " +
+          "\"TC_TABLE\" = COALESCE(:newTableName, \"TC_TABLE\"), " +
+          "\"TC_DATABASE\" = COALESCE(:newDbName, \"TC_DATABASE\") WHERE " +
+          "(\"TC_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"TC_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"TC_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"COMPLETED_TXN_COMPONENTS\" SET " +
+          "\"CTC_PARTITION\" = COALESCE(:newPartName, \"CTC_PARTITION\"), " +
+          "\"CTC_TABLE\" = COALESCE(:newTableName, \"CTC_TABLE\"), " +
+          "\"CTC_DATABASE\" = COALESCE(:newDbName, \"CTC_DATABASE\") WHERE " +
+          "(\"CTC_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"CTC_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"CTC_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"HIVE_LOCKS\" SET " +
+          "\"HL_PARTITION\" = COALESCE(:newPartName, \"HL_PARTITION\"), " +
+          "\"HL_TABLE\" = COALESCE(:newTableName, \"HL_TABLE\"), " +
+          "\"HL_DB\" = COALESCE(:newDbName, \"HL_DB\") WHERE " +
+          "(\"HL_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"HL_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"HL_DB\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"COMPACTION_QUEUE\" SET " +
+          "\"CQ_PARTITION\" = COALESCE(:newPartName, \"CQ_PARTITION\"), " +
+          "\"CQ_TABLE\" = COALESCE(:newTableName, \"CQ_TABLE\"), " +
+          "\"CQ_DATABASE\" = COALESCE(:newDbName, \"CQ_DATABASE\") WHERE " +
+          "(\"CQ_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"CQ_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"CQ_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"COMPLETED_COMPACTIONS\" SET " +
+          "\"CC_PARTITION\" = COALESCE(:newPartName, \"CC_PARTITION\"), " +
+          "\"CC_TABLE\" = COALESCE(:newTableName, \"CC_TABLE\"), " +
+          "\"CC_DATABASE\" = COALESCE(:newDbName, \"CC_DATABASE\") WHERE " +
+          "(\"CC_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"CC_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"CC_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"WRITE_SET\" SET " +
+          "\"WS_PARTITION\" = COALESCE(:newPartName, \"WS_PARTITION\"), " +
+          "\"WS_TABLE\" = COALESCE(:newTableName, \"WS_TABLE\"), " +
+          "\"WS_DATABASE\" = COALESCE(:newDbName, \"WS_DATABASE\") WHERE " +
+          "(\"WS_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"WS_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"WS_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"TXN_TO_WRITE_ID\" SET " +
+          "\"T2W_TABLE\" = COALESCE(:newTableName, \"T2W_TABLE\"), " +
+          "\"T2W_DATABASE\" = COALESCE(:newDbName, \"T2W_DATABASE\") WHERE " +
+          "(\"T2W_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"T2W_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"NEXT_WRITE_ID\" SET " +
+          "\"NWI_TABLE\" = COALESCE(:newTableName, \"NWI_TABLE\"), " +
+          "\"NWI_DATABASE\" = COALESCE(:newDbName, \"NWI_DATABASE\") WHERE " +
+          "(\"NWI_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"NWI_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+      "UPDATE \"COMPACTION_METRICS_CACHE\" SET " +
+          "\"CMC_PARTITION\" = COALESCE(:newPartName, \"CMC_PARTITION\"), " +
+          "\"CMC_TABLE\" = COALESCE(:newTableName, \"CMC_TABLE\"), " +
+          "\"CMC_DATABASE\" = COALESCE(:newDbName, \"CMC_DATABASE\") WHERE " +
+          "(\"CMC_PARTITION\" = :oldPartName OR :oldPartName IS NULL) AND " +
+          "(\"CMC_TABLE\" = :oldTableName OR :oldTableName IS NULL) AND " +
+          "(\"CMC_DATABASE\" = :oldDbName OR :oldDbName IS NULL)",
+  };
+
+  private final String oldCatName;
+  private final String oldDbName;
+  private final String oldTabName;
+  private final String oldPartName;
+  private final String newCatName;
+  private final String newDbName;
+  private final String newTabName;
+  private final String newPartName;
+
+  public OnRenameFunction(String oldCatName, String oldDbName, String oldTabName, String oldPartName, 
+                          String newCatName, String newDbName, String newTabName, String newPartName) {
+    this.oldCatName = oldCatName;
+    this.oldDbName = oldDbName;
+    this.oldTabName = oldTabName;
+    this.oldPartName = oldPartName;
+    this.newCatName = newCatName;
+    this.newDbName = newDbName;
+    this.newTabName = newTabName;
+    this.newPartName = newPartName;
+  }
+
+  @SuppressWarnings("squid:S2259")
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    String callSig = "onRename(" +
+        oldCatName + "," + oldDbName + "," + oldTabName + "," + oldPartName + "," +
+        newCatName + "," + newDbName + "," + newTabName + "," + newPartName + ")";
+
+    if(newPartName != null) {
+      assert oldPartName != null && oldTabName != null && oldDbName != null && oldCatName != null : callSig;
+    }
+    if(newTabName != null) {
+      assert oldTabName != null && oldDbName != null && oldCatName != null : callSig;
+    }
+    if(newDbName != null) {
+      assert oldDbName != null && oldCatName != null : callSig;
+    }
+
+    MapSqlParameterSource paramSource = new MapSqlParameterSource()
+        .addValue("oldDbName", StringUtils.lowerCase(oldDbName))
+        .addValue("newDbName", StringUtils.lowerCase(newDbName))
+        .addValue("oldTableName", StringUtils.lowerCase(oldTabName))
+        .addValue("newTableName", StringUtils.lowerCase(newTabName))
+        .addValue("oldPartName", oldPartName)
+        .addValue("newPartName", newPartName);
+    try {
+      for (String command : UPDATE_COMMANNDS) {
+        jdbcResource.getJdbcTemplate().update(command, paramSource);
+      }
+    } catch (DataAccessException e) {
+      //TODO: this seems to be very hacky, and as a result retry attempts won't happen, because DataAccessExceptions are
+      // caught and either swallowed or wrapped in MetaException. Also, only a single test fails without this block:
+      // org.apache.hadoop.hive.metastore.client.TestDatabases.testAlterDatabaseNotNullableFields
+      // It may worth investigate if this catch block is really needed. 
+      if (e.getMessage() != null && e.getMessage().contains("does not exist")) {
+        LOG.warn("Cannot perform {} since metastore table does not exist", callSig);
+      } else {
+        throw new MetaException("Unable to " + callSig + ":" + org.apache.hadoop.util.StringUtils.stringifyException(e));
+      }
+    }
+    return null;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OpenTxnsFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OpenTxnsFunction.java
new file mode 100644
index 000000000000..e677309799a0
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OpenTxnsFunction.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
+import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.TargetTxnIdListHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.UncategorizedSQLException;
+import org.springframework.jdbc.core.JdbcTemplate;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class OpenTxnsFunction implements TransactionalFunction> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(OpenTxnsFunction.class);
+
+  private static final String TXN_TMP_STATE = "_";
+  private static final String TXNS_INSERT_QRY = "INSERT INTO \"TXNS\" " +
+      "(\"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", \"TXN_USER\", \"TXN_HOST\", \"TXN_TYPE\") " +
+      "VALUES(?,%s,%s,?,?,?)";
+
+  private final OpenTxnRequest rqst;
+  private final long openTxnTimeOutMillis;
+  private final List transactionalListeners;
+
+  public OpenTxnsFunction(OpenTxnRequest rqst, long openTxnTimeOutMillis, 
+                          List transactionalListeners) {
+    this.rqst = rqst;
+    this.openTxnTimeOutMillis = openTxnTimeOutMillis;
+    this.transactionalListeners = transactionalListeners;
+  }
+
+  @Override
+  public List execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    DatabaseProduct dbProduct = jdbcResource.getDatabaseProduct();
+    int numTxns = rqst.getNum_txns();
+    // Make sure the user has not requested an insane amount of txns.
+    int maxTxns = MetastoreConf.getIntVar(jdbcResource.getConf(), MetastoreConf.ConfVars.TXN_MAX_OPEN_BATCH);
+    if (numTxns > maxTxns) {
+      numTxns = maxTxns;
+    }
+    List insertPreparedStmts = null;
+    TxnType txnType = rqst.isSetTxn_type() ? rqst.getTxn_type() : TxnType.DEFAULT;
+    boolean isReplayedReplTxn = txnType == TxnType.REPL_CREATED;
+    boolean isHiveReplTxn = rqst.isSetReplPolicy() && txnType == TxnType.DEFAULT;
+    if (isReplayedReplTxn) {
+      assert rqst.isSetReplPolicy();
+      List targetTxnIdList = jdbcResource.execute(new TargetTxnIdListHandler(rqst.getReplPolicy(), rqst.getReplSrcTxnIds()));
+
+      if (!targetTxnIdList.isEmpty()) {
+        if (targetTxnIdList.size() != rqst.getReplSrcTxnIds().size()) {
+          LOG.warn("target txn id number {} is not matching with source txn id number {}",
+              targetTxnIdList, rqst.getReplSrcTxnIds());
+        }
+        LOG.info("Target transactions {} are present for repl policy : {} and Source transaction id : {}",
+            targetTxnIdList, rqst.getReplPolicy(), rqst.getReplSrcTxnIds().toString());
+        return targetTxnIdList;
+      }
+    }
+
+    long minOpenTxnId = 0;
+    if (TxnHandler.ConfVars.useMinHistoryLevel()) {
+      minOpenTxnId = new MinOpenTxnIdWaterMarkFunction(openTxnTimeOutMillis).execute(jdbcResource);
+    }
+
+    List txnIds = new ArrayList<>(numTxns);
+    /*
+     * The getGeneratedKeys are not supported in every dbms, after executing a multi line insert.
+     * But it is supported in every used dbms for single line insert, even if the metadata says otherwise.
+     * If the getGeneratedKeys are not supported first we insert a random batchId in the TXN_META_INFO field,
+     * then the keys are selected beck with that batchid.
+     */
+    boolean genKeySupport = dbProduct.supportsGetGeneratedKeys();
+    genKeySupport = genKeySupport || (numTxns == 1);
+
+    String insertQuery = String.format(TXNS_INSERT_QRY, getEpochFn(dbProduct), getEpochFn(dbProduct));
+    LOG.debug("Going to execute insert <{}>", insertQuery);
+
+    Connection dbConn = jdbcResource.getConnection();
+    NamedParameterJdbcTemplate namedParameterJdbcTemplate = jdbcResource.getJdbcTemplate();
+    int maxBatchSize = MetastoreConf.getIntVar(jdbcResource.getConf(), MetastoreConf.ConfVars.JDBC_MAX_BATCH_SIZE);
+    try (PreparedStatement ps = dbConn.prepareStatement(insertQuery, new String[]{ "TXN_ID" })) {
+      String state = genKeySupport ? TxnStatus.OPEN.getSqlConst() : TXN_TMP_STATE;
+      if (numTxns == 1) {
+        ps.setString(1, state);
+        ps.setString(2, rqst.getUser());
+        ps.setString(3, rqst.getHostname());
+        ps.setInt(4, txnType.getValue());
+        txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(namedParameterJdbcTemplate, true, ps, false));
+      } else {
+        for (int i = 0; i < numTxns; ++i) {
+          ps.setString(1, state);
+          ps.setString(2, rqst.getUser());
+          ps.setString(3, rqst.getHostname());
+          ps.setInt(4, txnType.getValue());
+          ps.addBatch();
+
+          if ((i + 1) % maxBatchSize == 0) {
+            txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(namedParameterJdbcTemplate, genKeySupport, ps, true));
+          }
+        }
+        if (numTxns % maxBatchSize != 0) {
+          txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(namedParameterJdbcTemplate, genKeySupport, ps, true));
+        }
+      }
+    } catch (SQLException e) {
+      throw new UncategorizedSQLException(null, null, e);
+    }
+
+    assert txnIds.size() == numTxns;
+
+    addTxnToMinHistoryLevel(jdbcResource.getJdbcTemplate().getJdbcTemplate(), maxBatchSize, txnIds, minOpenTxnId);
+
+    if (isReplayedReplTxn) {
+      List rowsRepl = new ArrayList<>(numTxns);
+      List params = Collections.singletonList(rqst.getReplPolicy());
+      List> paramsList = new ArrayList<>(numTxns);
+      for (int i = 0; i < numTxns; i++) {
+        rowsRepl.add("?," + rqst.getReplSrcTxnIds().get(i) + "," + txnIds.get(i));
+        paramsList.add(params);
+      }
+
+      try {
+        insertPreparedStmts = jdbcResource.getSqlGenerator().createInsertValuesPreparedStmt(dbConn,
+            "\"REPL_TXN_MAP\" (\"RTM_REPL_POLICY\", \"RTM_SRC_TXN_ID\", \"RTM_TARGET_TXN_ID\")", rowsRepl,
+            paramsList);
+        for (PreparedStatement pst : insertPreparedStmts) {
+          try (PreparedStatement ppst = pst) {
+            ppst.execute();
+          }
+        }
+      } catch (SQLException e) {
+        throw new UncategorizedSQLException(null, null, e);
+      }
+    }
+
+    if (transactionalListeners != null && !isHiveReplTxn) {
+      MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+          EventMessage.EventType.OPEN_TXN, new OpenTxnEvent(txnIds, txnType), dbConn, jdbcResource.getSqlGenerator());
+    }
+    return txnIds;
+  }
+
+  /**
+   * Add min history level entry for each generated txn record
+   * @param jdbcTemplate {@link NamedParameterJdbcTemplate} to use for command execution
+   * @param txnIds new transaction ids
+   * @deprecated Remove this method when min_history_level table is dropped
+   * @throws SQLException ex
+   */
+  @Deprecated
+  private void addTxnToMinHistoryLevel(JdbcTemplate jdbcTemplate, int batchSize, List txnIds, long minOpenTxnId) {
+    if (!TxnHandler.ConfVars.useMinHistoryLevel()) {
+      return;
+    }
+    String sql = "INSERT INTO \"MIN_HISTORY_LEVEL\" (\"MHL_TXNID\", \"MHL_MIN_OPEN_TXNID\") VALUES(?, ?)";
+    LOG.debug("Going to execute insert batch: <{}>", sql);
+
+    jdbcTemplate.batchUpdate(sql, txnIds, batchSize, (ps, argument) -> {
+      ps.setLong(1, argument);
+      ps.setLong(2, minOpenTxnId);
+    });
+    
+    LOG.info("Added entries to MIN_HISTORY_LEVEL for current txns: ({}) with min_open_txn: {}", txnIds, minOpenTxnId);
+  }
+
+  private List executeTxnInsertBatchAndExtractGeneratedKeys(NamedParameterJdbcTemplate jdbcTemplate, boolean genKeySupport,
+                                                                  PreparedStatement ps, boolean batch) throws SQLException {
+    
+    List txnIds = new ArrayList<>();
+    if (batch) {
+      ps.executeBatch();
+    } else {
+      // For slight performance advantage we do not use the executeBatch, when we only have one row
+      ps.execute();
+    }
+    if (genKeySupport) {
+      try (ResultSet generatedKeys = ps.getGeneratedKeys()) {
+        while (generatedKeys.next()) {
+          txnIds.add(generatedKeys.getLong(1));
+        }
+      }
+    } else {
+      txnIds = jdbcTemplate.query("SELECT \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = :tmpState",
+          new MapSqlParameterSource().addValue("tmpState", TXN_TMP_STATE), (rs, rowNum) -> rs.getLong(1));
+
+      jdbcTemplate.update("UPDATE \"TXNS\" SET \"TXN_STATE\" = :newState WHERE \"TXN_STATE\" = :tmpState",
+          new MapSqlParameterSource()
+              .addValue("newState", TxnStatus.OPEN.getSqlConst())
+              .addValue("tmpState", TXN_TMP_STATE));
+    }
+    return txnIds;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PerformTimeoutsFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PerformTimeoutsFunction.java
new file mode 100644
index 000000000000..8a7e9555fb1a
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PerformTimeoutsFunction.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
+import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetTxnDbsUpdatedHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class PerformTimeoutsFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(PerformTimeoutsFunction.class);
+
+  private static final String SELECT_TIMED_OUT_LOCKS_QUERY = "SELECT DISTINCT \"HL_LOCK_EXT_ID\" FROM \"HIVE_LOCKS\" " +
+      "WHERE \"HL_LAST_HEARTBEAT\" < %s - :timeout AND \"HL_TXNID\" = 0";
+
+  public static int TIMED_OUT_TXN_ABORT_BATCH_SIZE = 50000;
+
+  private final long timeout;
+  private final long replicationTxnTimeout;
+  private final List transactionalListeners;  
+
+  public PerformTimeoutsFunction(long timeout, long replicationTxnTimeout, List transactionalListeners) {
+    this.timeout = timeout;
+    this.replicationTxnTimeout = replicationTxnTimeout;
+    this.transactionalListeners = transactionalListeners;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) {
+    DatabaseProduct dbProduct = jdbcResource.getDatabaseProduct();
+    try {
+      //We currently commit after selecting the TXNS to abort.  So whether SERIALIZABLE
+      //READ_COMMITTED, the effect is the same.  We could use FOR UPDATE on Select from TXNS
+      //and do the whole performTimeOuts() in a single huge transaction, but the only benefit
+      //would be to make sure someone cannot heartbeat one of these txns at the same time.
+      //The attempt to heartbeat would block and fail immediately after it's unblocked.
+      //With current (RC + multiple txns) implementation it is possible for someone to send
+      //heartbeat at the very end of the expiry interval, and just after the Select from TXNS
+      //is made, in which case heartbeat will succeed but txn will still be Aborted.
+      //Solving this corner case is not worth the perf penalty.  The client should heartbeat in a
+      //timely way.
+      timeOutLocks(jdbcResource, dbProduct);
+      while (true) {
+        String s = " \"TXN_ID\", \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN +
+            " AND (" +
+            "\"TXN_TYPE\" != " + TxnType.REPL_CREATED.getValue() +
+            " AND \"TXN_LAST_HEARTBEAT\" <  " + getEpochFn(dbProduct) + "-" + timeout +
+            " OR " +
+            " \"TXN_TYPE\" = " + TxnType.REPL_CREATED.getValue() +
+            " AND \"TXN_LAST_HEARTBEAT\" <  " + getEpochFn(dbProduct) + "-" + replicationTxnTimeout +
+            ")";
+        //safety valve for extreme cases
+        s = jdbcResource.getSqlGenerator().addLimitClause(10 * TIMED_OUT_TXN_ABORT_BATCH_SIZE, s);
+
+        LOG.debug("Going to execute query <{}>", s);
+        List> timedOutTxns = Objects.requireNonNull(jdbcResource.getJdbcTemplate().query(s, rs -> {
+          List> txnbatch = new ArrayList<>();
+          Map currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE);
+          while (rs.next()) {
+            currentBatch.put(rs.getLong(1),TxnType.findByValue(rs.getInt(2)));
+            if (currentBatch.size() == TIMED_OUT_TXN_ABORT_BATCH_SIZE) {
+              txnbatch.add(currentBatch);
+              currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE);
+            }
+          }
+          if (!currentBatch.isEmpty()) {
+            txnbatch.add(currentBatch);
+          }
+          return txnbatch;
+        }), "This never should be null, it's just to suppress warnings");
+        if (timedOutTxns.isEmpty()) {
+          return null;
+        }
+
+        TransactionContext context = jdbcResource.getTransactionManager().getActiveTransaction();
+        Object savePoint = context.createSavepoint();
+
+        int numTxnsAborted = 0;
+        for (Map batchToAbort : timedOutTxns) {
+          context.releaseSavepoint(savePoint);
+          savePoint = context.createSavepoint();
+          int abortedTxns = new AbortTxnsFunction(new ArrayList<>(batchToAbort.keySet()), true, false, false, 
+              TxnErrorMsg.ABORT_TIMEOUT).execute(jdbcResource);
+          
+          if (abortedTxns == batchToAbort.size()) {
+            numTxnsAborted += batchToAbort.size();
+            //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout'
+            LOG.info("Aborted the following transactions due to timeout: {}", batchToAbort);
+            if (transactionalListeners != null) {
+              for (Map.Entry txnEntry : batchToAbort.entrySet()) {
+                List dbsUpdated = jdbcResource.execute(new GetTxnDbsUpdatedHandler(txnEntry.getKey()));
+                MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                    EventMessage.EventType.ABORT_TXN,
+                    new AbortTxnEvent(txnEntry.getKey(), txnEntry.getValue(), null, dbsUpdated),
+                    jdbcResource.getConnection(), jdbcResource.getSqlGenerator());
+              }
+              LOG.debug("Added Notifications for the transactions that are aborted due to timeout: {}", batchToAbort);
+            }
+          } else {
+            //could not abort all txns in this batch - this may happen because in parallel with this
+            //operation there was activity on one of the txns in this batch (commit/abort/heartbeat)
+            //This is not likely but may happen if client experiences long pause between heartbeats or
+            //unusually long/extreme pauses between heartbeat() calls and other logic in checkLock(),
+            //lock(), etc.
+            context.rollbackToSavepoint(savePoint);
+          }
+        }
+        LOG.info("Aborted {} transaction(s) due to timeout", numTxnsAborted);
+        if (MetastoreConf.getBoolVar(jdbcResource.getConf(), MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
+          Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_TIMED_OUT_TXNS).inc(numTxnsAborted);
+        }
+      }
+    } catch (Exception e) {
+      LOG.warn("Aborting timed out transactions failed due to " + e.getMessage(), e);
+    }
+    return null;
+  }
+
+  // Clean time out locks from the database not associated with a transactions, i.e. locks
+  // for read-only autoCommit=true statements.  This does a commit,
+  // and thus should be done before any calls to heartbeat that will leave
+  // open transactions.
+  private void timeOutLocks(MultiDataSourceJdbcResource jdbcResource, DatabaseProduct dbProduct) {
+    //doing a SELECT first is less efficient but makes it easier to debug things
+    //when txnid is <> 0, the lock is associated with a txn and is handled by performTimeOuts()
+    //want to avoid expiring locks for a txn w/o expiring the txn itself
+    try {
+      Set timedOutLockIds = new TreeSet<>(
+          jdbcResource.getJdbcTemplate().query(String.format(SELECT_TIMED_OUT_LOCKS_QUERY, getEpochFn(dbProduct)),
+              new MapSqlParameterSource().addValue("timeout", timeout),
+              (rs, rowNum) -> rs.getLong(1)));
+      if (timedOutLockIds.isEmpty()) {
+        LOG.debug("Did not find any timed-out locks, therefore retuning.");
+        return;
+      }
+
+      List queries = new ArrayList<>();
+      StringBuilder prefix = new StringBuilder();
+      StringBuilder suffix = new StringBuilder();
+
+      //include same hl_last_heartbeat condition in case someone heartbeated since the select
+      prefix.append("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LAST_HEARTBEAT\" < ");
+      prefix.append(getEpochFn(dbProduct)).append("-").append(timeout);
+      prefix.append(" AND \"HL_TXNID\" = 0 AND ");
+
+      TxnUtils.buildQueryWithINClause(jdbcResource.getConf(), queries, prefix, suffix, timedOutLockIds,
+          "\"HL_LOCK_EXT_ID\"", true, false);
+
+      int deletedLocks = 0;
+      for (String query : queries) {
+        LOG.debug("Going to execute update: <{}>", query);
+        deletedLocks += jdbcResource.getJdbcTemplate().update(query, new MapSqlParameterSource());
+      }
+      if (deletedLocks > 0) {
+        LOG.info("Deleted {} locks due to timed-out. Lock ids: {}", deletedLocks, timedOutLockIds);
+      }
+    } catch (Exception ex) {
+      LOG.error("Failed to purge timed-out locks: " + ex.getMessage(), ex);
+    }
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/PurgeCompactionHistoryFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PurgeCompactionHistoryFunction.java
similarity index 93%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/PurgeCompactionHistoryFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PurgeCompactionHistoryFunction.java
index f1402ed899cf..1a5888499643 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/PurgeCompactionHistoryFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/PurgeCompactionHistoryFunction.java
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.InClauseBatchCommand;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
 import org.slf4j.Logger;
@@ -43,20 +44,15 @@ public class PurgeCompactionHistoryFunction implements TransactionalFunction deleteSet = new ArrayList<>();
     long timeoutThreshold = System.currentTimeMillis() -
         MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_TIMEOUT, TimeUnit.MILLISECONDS);
     int didNotInitiateRetention = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_DID_NOT_INITIATE);
-    int failedRetention = getFailedCompactionRetention();
+    int failedRetention = getFailedCompactionRetention(conf);
     int succeededRetention = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_SUCCEEDED);
     int refusedRetention = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_REFUSED);
         /* cc_id is monotonically increasing so for any entity sorts in order of compaction history,
@@ -90,13 +86,12 @@ public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaExcepti
       return null;
     }
 
-    int totalCount = TxnUtils.executeStatementWithInClause(conf, jdbcTemplate,
+    int totalCount = jdbcResource.execute(new InClauseBatchCommand<>(
         "DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_ID\" in (:ids)",
-        new MapSqlParameterSource(), "ids", deleteSet, Long::compareTo);
+        new MapSqlParameterSource().addValue("ids", deleteSet), "ids", Long::compareTo));
     LOG.debug("Removed {} records from COMPLETED_COMPACTIONS", totalCount);
     return null;
-  }
-  
+  }  
 
   private void checkForDeletion(List deleteSet, CompactionInfo ci, RetentionCounters rc, long timeoutThreshold) {
     switch (ci.state) {
@@ -139,7 +134,7 @@ private static boolean timedOut(CompactionInfo ci, RetentionCounters rc, long pa
    * this ensures that the number of failed compaction entries retained is > than number of failed
    * compaction threshold which prevents new compactions from being scheduled.
    */
-  private int getFailedCompactionRetention() {
+  private int getFailedCompactionRetention(Configuration conf) {
     int failedThreshold = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_FAILED_THRESHOLD);
     int failedRetention = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED);
     if(failedRetention < failedThreshold) {
@@ -167,5 +162,5 @@ private static class RetentionCounters {
       this.refusedRetention = refusedRetention;
     }
   }
-
+  
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java
new file mode 100644
index 000000000000..0ddebae5c588
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ReleaseMaterializationRebuildLocks implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReleaseMaterializationRebuildLocks.class);
+
+  private final ValidTxnList validTxnList;
+  private final long timeout;
+
+  public ReleaseMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) {
+    this.validTxnList = validTxnList;
+    this.timeout = timeout;
+  }
+
+  @Override
+  public Long execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    // Aux values
+    long cnt = 0L;
+    List txnIds = new ArrayList<>();
+    long timeoutTime = Instant.now().toEpochMilli() - timeout;
+
+
+    String selectQ = "SELECT \"MRL_TXN_ID\", \"MRL_LAST_HEARTBEAT\" FROM \"MATERIALIZATION_REBUILD_LOCKS\"";
+    LOG.debug("Going to execute query <{}>", selectQ);
+
+    jdbcResource.getJdbcTemplate().query(selectQ, rs -> {
+      long lastHeartbeat = rs.getLong(2);
+      if (lastHeartbeat < timeoutTime) {
+        // The heartbeat has timeout, double check whether we can remove it
+        long txnId = rs.getLong(1);
+        if (validTxnList.isTxnValid(txnId) || validTxnList.isTxnAborted(txnId)) {
+          // Txn was committed (but notification was not received) or it was aborted.
+          // Either case, we can clean it up
+          txnIds.add(txnId);
+        }
+      }
+      return null;
+    });
+
+    if (!txnIds.isEmpty()) {
+      String deleteQ = "DELETE FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE \"MRL_TXN_ID\" IN(:txnIds)";
+      LOG.debug("Going to execute update <{}>", deleteQ);
+      cnt = jdbcResource.getJdbcTemplate().update(deleteQ, new MapSqlParameterSource().addValue("txnIds", txnIds));
+    }
+    LOG.debug("Going to commit");
+    return cnt;
+  }
+
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReplTableWriteIdStateFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReplTableWriteIdStateFunction.java
new file mode 100644
index 000000000000..f369bb35f12f
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReplTableWriteIdStateFunction.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
+import org.apache.hadoop.hive.metastore.api.ReplTblWriteIdStateRequest;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class ReplTableWriteIdStateFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReplTableWriteIdStateFunction.class);
+  
+  private final ReplTblWriteIdStateRequest rqst;
+  private final TxnStore.MutexAPI mutexAPI;
+  private final List transactionalListeners;
+
+  public ReplTableWriteIdStateFunction(ReplTblWriteIdStateRequest rqst, TxnStore.MutexAPI mutexAPI, List transactionalListeners) {
+    this.rqst = rqst;
+    this.mutexAPI = mutexAPI;
+    this.transactionalListeners = transactionalListeners;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    long openTxnTimeOutMillis = MetastoreConf.getTimeVar(jdbcResource.getConf(), MetastoreConf.ConfVars.TXN_OPENTXN_TIMEOUT, TimeUnit.MILLISECONDS);
+
+    String dbName = rqst.getDbName().toLowerCase();
+    String tblName = rqst.getTableName().toLowerCase();
+    ValidWriteIdList validWriteIdList = new ValidReaderWriteIdList(rqst.getValidWriteIdlist());
+
+    NamedParameterJdbcTemplate npjdbcTemplate = jdbcResource.getJdbcTemplate();
+    // Check if this txn state is already replicated for this given table. If yes, then it is
+    // idempotent case and just return.
+    boolean found = Boolean.TRUE.equals(npjdbcTemplate.query(
+        "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = :dbName AND \"NWI_TABLE\" = :tableName", 
+        new MapSqlParameterSource()
+            .addValue("dbName", dbName)
+            .addValue("tableName", tblName),
+        ResultSet::next
+    ));
+
+    if (found) {
+      LOG.info("Idempotent flow: WriteId state <{}> is already applied for the table: {}.{}",
+          validWriteIdList, dbName, tblName);
+      return null;
+    }
+
+    // Get the abortedWriteIds which are already sorted in ascending order.
+    List abortedWriteIds = getAbortedWriteIds(validWriteIdList);
+    int numAbortedWrites = abortedWriteIds.size();
+    if (numAbortedWrites > 0) {
+      // Allocate/Map one txn per aborted writeId and abort the txn to mark writeid as aborted.
+      // We don't use the txnLock, all of these transactions will be aborted in this one rdbm transaction
+      // So they will not effect the commitTxn in any way
+
+      List txnIds = new OpenTxnsFunction(
+          new OpenTxnRequest(numAbortedWrites, rqst.getUser(), rqst.getHostName()),
+          openTxnTimeOutMillis, transactionalListeners).execute(jdbcResource);
+      assert (numAbortedWrites == txnIds.size());
+
+      // Map each aborted write id with each allocated txn.
+      List params = new ArrayList<>(txnIds.size());
+      for (int i = 0; i < txnIds.size(); i++) {
+        params.add(new Object[] {txnIds.get(i), dbName, tblName, abortedWriteIds.get(i)});
+        LOG.info("Allocated writeID: {} for txnId: {}", abortedWriteIds.get(i), txnIds.get(i));
+      }
+      
+      int maxBatchSize = MetastoreConf.getIntVar(jdbcResource.getConf(), MetastoreConf.ConfVars.JDBC_MAX_BATCH_SIZE);
+      jdbcResource.getJdbcTemplate().getJdbcTemplate().batchUpdate(
+          "INSERT INTO \"TXN_TO_WRITE_ID\" (\"T2W_TXNID\", \"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\") VALUES (?, ?, ?, ?)",
+          params, maxBatchSize, (PreparedStatement ps, Object[] statementParams) -> {
+            ps.setLong(1, (Long)statementParams[0]);
+            ps.setString(2, statementParams[1].toString());
+            ps.setString(3, statementParams[2].toString());
+            ps.setLong(4, (Long)statementParams[3]);
+          });
+
+      // Abort all the allocated txns so that the mapped write ids are referred as aborted ones.
+      int numAborts = new AbortTxnsFunction(txnIds, false, false,false,
+          TxnErrorMsg.ABORT_REPL_WRITEID_TXN).execute(jdbcResource);
+      assert (numAborts == numAbortedWrites);
+    }
+
+    // There are some txns in the list which has no write id allocated and hence go ahead and do it.
+    // Get the next write id for the given table and update it with new next write id.
+    // It is expected NEXT_WRITE_ID doesn't have entry for this table and hence directly insert it.
+    long nextWriteId = validWriteIdList.getHighWatermark() + 1;
+
+    // First allocation of write id (hwm+1) should add the table to the next_write_id meta table.
+    npjdbcTemplate.update(
+        "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (:dbName, :tableName, :nextWriteId)",
+        new MapSqlParameterSource()
+            .addValue("dbName", dbName)
+            .addValue("tableName", tblName)
+            .addValue("nextWriteId", nextWriteId));
+    LOG.info("WriteId state <{}> is applied for the table: {}.{}", validWriteIdList, dbName, tblName);
+
+    // Schedule Major compaction on all the partitions/table to clean aborted data
+    if (numAbortedWrites > 0) {
+      CompactionRequest compactRqst = new CompactionRequest(rqst.getDbName(), rqst.getTableName(),
+          CompactionType.MAJOR);
+      if (rqst.isSetPartNames()) {
+        for (String partName : rqst.getPartNames()) {
+          compactRqst.setPartitionname(partName);
+          new CompactFunction(compactRqst, openTxnTimeOutMillis, mutexAPI).execute(jdbcResource);
+        }
+      } else {
+        new CompactFunction(compactRqst, openTxnTimeOutMillis, mutexAPI).execute(jdbcResource);
+      }
+    }
+    return null;
+  }
+
+  private List getAbortedWriteIds(ValidWriteIdList validWriteIdList) {
+    return Arrays.stream(validWriteIdList.getInvalidWriteIds())
+        .filter(validWriteIdList::isWriteIdAborted)
+        .boxed()
+        .collect(Collectors.toList());
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/TopCompactionMetricsDataPerTypeFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/TopCompactionMetricsDataPerTypeFunction.java
similarity index 87%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/TopCompactionMetricsDataPerTypeFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/TopCompactionMetricsDataPerTypeFunction.java
index 409212f8edb6..16cfcfecc679 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/TopCompactionMetricsDataPerTypeFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/TopCompactionMetricsDataPerTypeFunction.java
@@ -15,11 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
-import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
 import org.springframework.jdbc.core.RowMapper;
@@ -37,11 +36,9 @@ public class TopCompactionMetricsDataPerTypeFunction implements TransactionalFun
           "\"COMPACTION_METRICS_CACHE\" WHERE \"CMC_METRIC_TYPE\" = :type ORDER BY \"CMC_METRIC_VALUE\" DESC";
   
   private final int limit;
-  private final SQLGenerator sqlGenerator;
 
-  public TopCompactionMetricsDataPerTypeFunction(int limit, SQLGenerator sqlGenerator) {
+  public TopCompactionMetricsDataPerTypeFunction(int limit) {
     this.limit = limit;
-    this.sqlGenerator = sqlGenerator;
   }
 
   @Override
@@ -50,7 +47,7 @@ public List execute(MultiDataSourceJdbcResource jdbcResou
     List metricsDataList = new ArrayList<>();
     for (CompactionMetricsData.MetricType type : CompactionMetricsData.MetricType.values()) {
       metricsDataList.addAll(jdbcResource.getJdbcTemplate().query(
-          sqlGenerator.addLimitClause(limit, NO_SELECT_COMPACTION_METRICS_CACHE_FOR_TYPE_QUERY),
+          jdbcResource.getSqlGenerator().addLimitClause(limit, NO_SELECT_COMPACTION_METRICS_CACHE_FOR_TYPE_QUERY),
           new MapSqlParameterSource().addValue("type", type.toString()),
           new CompactionMetricsDataMapper(type)));
     }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdataDatabasePropFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdataDatabasePropFunction.java
new file mode 100644
index 000000000000..23e9f7715bfa
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdataDatabasePropFunction.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate;
+
+public class UpdataDatabasePropFunction implements TransactionalFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(UpdataDatabasePropFunction.class);
+
+  private final String database;
+  private final long dbId;
+  private final String prop;
+  private final String propValue;
+
+  public UpdataDatabasePropFunction(String database, long dbId, String prop, String propValue) {
+    this.database = database;
+    this.dbId = dbId;
+    this.prop = prop;
+    this.propValue = propValue;
+  }
+
+  @Override
+  public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaException {
+    MapSqlParameterSource params = new MapSqlParameterSource()
+        .addValue("dbId", dbId)
+        .addValue("key", prop)
+        .addValue("value", propValue);
+    NamedParameterJdbcTemplate jdbcTemplate = jdbcResource.getJdbcTemplate();
+
+    String value = jdbcTemplate.query("SELECT \"PARAM_VALUE\" FROM \"DATABASE_PARAMS\" " +
+        "WHERE \"PARAM_KEY\" = :key AND \"DB_ID\" = :dbId", params, rs -> rs.next() ? rs.getString("PARAM_VALUE") : null);
+
+    int count = 1;
+    if (value != null) {
+      if (value.equals(propValue)) {
+        LOG.info("Database property: {} with value: {} already updated for db: {}", prop, propValue, database);
+      } else {
+        count = jdbcTemplate.update("UPDATE \"DATABASE_PARAMS\" SET \"PARAM_VALUE\" = :value WHERE \"DB_ID\" = :dbId AND " +
+            "\"PARAM_KEY\" = :key", params);
+      }
+    } else {
+      count = jdbcTemplate.update("INSERT INTO \"DATABASE_PARAMS\" VALUES (:dbId, :key, :value)", params);
+    }
+    if (count != 1) {
+      //only one row insert or update should happen
+      throw new RuntimeException("DATABASE_PARAMS is corrupted for database: " + database);
+    }
+    return null;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/UpdateCompactionMetricsDataFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdateCompactionMetricsDataFunction.java
similarity index 92%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/UpdateCompactionMetricsDataFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdateCompactionMetricsDataFunction.java
index e372041922cd..1f6ac60a6eb9 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/UpdateCompactionMetricsDataFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/UpdateCompactionMetricsDataFunction.java
@@ -15,10 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.functions;
 
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveCompactionMetricsDataCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.CompactionMetricsDataHandler;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionalFunction;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortTxnInfoHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortTxnInfoHandler.java
new file mode 100644
index 000000000000..cf7f50956d49
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortTxnInfoHandler.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.READY_FOR_CLEANING;
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class AbortTxnInfoHandler implements QueryHandler> {
+
+  // Three inner sub-queries which are under left-join to fetch the required data for aborted txns.
+  //language=SQL
+  private static final String SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY =
+      " \"res1\".\"TC_DATABASE\" AS \"DB\", \"res1\".\"TC_TABLE\" AS \"TBL\", \"res1\".\"TC_PARTITION\" AS \"PART\", " +
+          " \"res1\".\"MIN_TXN_START_TIME\" AS \"MIN_TXN_START_TIME\", \"res1\".\"ABORTED_TXN_COUNT\" AS \"ABORTED_TXN_COUNT\", " +
+          " \"res2\".\"MIN_OPEN_WRITE_TXNID\" AS \"MIN_OPEN_WRITE_TXNID\", \"res3\".\"RETRY_RETENTION\" AS \"RETRY_RETENTION\", " +
+          " \"res3\".\"ID\" AS \"RETRY_CQ_ID\" " +
+          " FROM " +
+          // First sub-query - Gets the aborted txns with min txn start time, number of aborted txns
+          // for corresponding db, table, partition.
+          " ( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", " +
+          " COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " +
+          " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = :abortedState" +
+          " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" %s ) \"res1\" " +
+          " LEFT JOIN" +
+          // Second sub-query - Gets the min open txn id for corresponding db, table, partition.
+          "( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\" " +
+          " FROM \"TXNS\", \"TXN_COMPONENTS\" " +
+          " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = :openState" +
+          " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" ) \"res2\"" +
+          " ON \"res1\".\"TC_DATABASE\" = \"res2\".\"TC_DATABASE\"" +
+          " AND \"res1\".\"TC_TABLE\" = \"res2\".\"TC_TABLE\"" +
+          " AND (\"res1\".\"TC_PARTITION\" = \"res2\".\"TC_PARTITION\" " +
+          " OR (\"res1\".\"TC_PARTITION\" IS NULL AND \"res2\".\"TC_PARTITION\" IS NULL)) " +
+          " LEFT JOIN " +
+          // Third sub-query - Gets the retry entries for corresponding db, table, partition.
+          "( SELECT \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", MAX(\"CQ_ID\") AS \"ID\", " +
+          " MAX(\"CQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\", " +
+          " MIN(\"CQ_COMMIT_TIME\") - %s + MAX(\"CQ_RETRY_RETENTION\") AS \"RETRY_RECORD_CHECK\" FROM \"COMPACTION_QUEUE\" " +
+          " WHERE \"CQ_TYPE\" = :type" +
+          " GROUP BY \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\") \"res3\" " +
+          " ON \"res1\".\"TC_DATABASE\" = \"res3\".\"CQ_DATABASE\" " +
+          " AND \"res1\".\"TC_TABLE\" = \"res3\".\"CQ_TABLE\" " +
+          " AND (\"res1\".\"TC_PARTITION\" = \"res3\".\"CQ_PARTITION\" " +
+          " OR (\"res1\".\"TC_PARTITION\" IS NULL AND \"res3\".\"CQ_PARTITION\" IS NULL))" +
+          " WHERE \"res3\".\"RETRY_RECORD_CHECK\" <= 0 OR \"res3\".\"RETRY_RECORD_CHECK\" IS NULL";
+
+  private final long abortedTimeThreshold;
+  private final int abortedThreshold;
+  private final int fetchSize;
+  
+  public String getParameterizedQueryString(DatabaseProduct dbProduct) throws MetaException {
+    return dbProduct.addLimitClause(
+        fetchSize,
+        String.format(AbortTxnInfoHandler.SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY,
+            abortedTimeThreshold >= 0 ? "" : " HAVING COUNT(*) > " + abortedThreshold, getEpochFn(dbProduct)));
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("abortedState", TxnStatus.ABORTED.getSqlConst(), Types.CHAR)
+        .addValue("openState", TxnStatus.OPEN.getSqlConst(), Types.CHAR)
+        .addValue("type", Character.toString(TxnStore.ABORT_TXN_CLEANUP_TYPE), Types.CHAR);
+  }
+
+  @Override
+  public List extractData(ResultSet rs) throws DataAccessException, SQLException {
+    List readyToCleanAborts = new ArrayList<>();
+    long systemTime = System.currentTimeMillis();
+    boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0;
+    while (rs.next()) {
+      boolean pastTimeThreshold =
+          checkAbortedTimeThreshold && rs.getLong("MIN_TXN_START_TIME") + abortedTimeThreshold < systemTime;
+      int numAbortedTxns = rs.getInt("ABORTED_TXN_COUNT");
+      if (numAbortedTxns > abortedThreshold || pastTimeThreshold) {
+        CompactionInfo info = new CompactionInfo();
+        info.dbname = rs.getString("DB");
+        info.tableName = rs.getString("TBL");
+        info.partName = rs.getString("PART");
+        // In this case, this field contains min open write txn ID.
+        long value = rs.getLong("MIN_OPEN_WRITE_TXNID");
+        info.minOpenWriteTxnId = value > 0 ? value : Long.MAX_VALUE;
+        // The specific type, state assigned to abort cleanup.
+        info.type = CompactionType.ABORT_TXN_CLEANUP;
+        info.state = READY_FOR_CLEANING;
+        info.retryRetention = rs.getLong("RETRY_RETENTION");
+        info.id = rs.getLong("RETRY_CQ_ID");
+        readyToCleanAborts.add(info);
+      }
+    }
+    return readyToCleanAborts;
+  }
+
+  public AbortTxnInfoHandler(long abortedTimeThreshold, int abortedThreshold, int fetchSize) {
+    this.abortedTimeThreshold = abortedTimeThreshold;
+    this.abortedThreshold = abortedThreshold;
+    this.fetchSize = fetchSize;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/AbortedTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortedTxnHandler.java
similarity index 94%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/AbortedTxnHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortedTxnHandler.java
index 10c27971daa1..f31b2d19d3a6 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/AbortedTxnHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/AbortedTxnHandler.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
-import org.apache.hadoop.hive.metastore.txn.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CheckFailedCompactionsHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CheckFailedCompactionsHandler.java
similarity index 96%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CheckFailedCompactionsHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CheckFailedCompactionsHandler.java
index 0041d4f734ba..b6a16cd09d19 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CheckFailedCompactionsHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CheckFailedCompactionsHandler.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CompactionCandidateHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CompactionCandidateHandler.java
similarity index 96%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CompactionCandidateHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CompactionCandidateHandler.java
index c5d056c78611..98158783277e 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CompactionCandidateHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CompactionCandidateHandler.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CompactionMetricsDataHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CompactionMetricsDataHandler.java
similarity index 95%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CompactionMetricsDataHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CompactionMetricsDataHandler.java
index 8186cda0d395..1e112f4db1e6 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/CompactionMetricsDataHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CompactionMetricsDataHandler.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CountOpenTxnsHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CountOpenTxnsHandler.java
new file mode 100644
index 000000000000..9f9458f2f1bd
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/CountOpenTxnsHandler.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class CountOpenTxnsHandler implements QueryHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CountOpenTxnsHandler.class);
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT COUNT(*) FROM \"TXNS\" WHERE \"TXN_STATE\" = :state";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource().addValue("state", TxnStatus.OPEN.getSqlConst());
+  }
+
+  @Override
+  public Integer extractData(ResultSet rs) throws SQLException, DataAccessException {
+    if (!rs.next()) {
+      LOG.error("Transaction database not properly configured, can't find txn_state from TXNS.");
+      return -1;
+    } else {
+      Long numOpen = rs.getLong(1);
+      if (numOpen > Integer.MAX_VALUE) {
+        LOG.error("Open transaction count above {}, can't count that high!", Integer.MAX_VALUE);
+        return -1;
+      } else {
+        return numOpen.intValue();
+      }
+    }
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/DbTimeHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/DbTimeHandler.java
new file mode 100644
index 000000000000..12f17b2c67dc
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/DbTimeHandler.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.dao.EmptyResultDataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+public class DbTimeHandler implements QueryHandler {
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return databaseProduct.getDBTime();
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource();
+  }
+
+  @Override
+  public Timestamp extractData(ResultSet rs) throws SQLException, DataAccessException {
+    if (rs.next()) {
+      return rs.getTimestamp(1);
+    } else {
+      throw new EmptyResultDataAccessException("Could not obtain DB time", 1);
+    }
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/FindColumnsWithStatsHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindColumnsWithStatsHandler.java
similarity index 95%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/FindColumnsWithStatsHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindColumnsWithStatsHandler.java
index 82a16e3e69e7..e20d429e25f2 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/FindColumnsWithStatsHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindColumnsWithStatsHandler.java
@@ -15,10 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindTxnStateHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindTxnStateHandler.java
new file mode 100644
index 000000000000..a5490af57c1a
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/FindTxnStateHandler.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class FindTxnStateHandler implements QueryHandler {
+  
+  private final long txnId;
+
+  public FindTxnStateHandler(long txnId) {
+    this.txnId = txnId;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return 
+        "SELECT s.STATE from (" +
+          "SELECT '1_txns' AS SOURCE, \"TXN_STATE\" AS STATE FROM \"TXNS\" WHERE \"TXN_ID\" = :txnId "+
+          "UNION " +
+          "SELECT '2_txn_compontents' AS SOURCE, 'c' AS STATE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_TXNID\" = :txnId) s " +
+        "ORDER BY s.SOURCE";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource().addValue("txnId", txnId);
+  }
+
+  @Override
+  public TxnStatus extractData(ResultSet rs) throws SQLException, DataAccessException {
+    if (rs.next()) {
+        return TxnStatus.fromString(rs.getString("STATE")); 
+    } else {
+      return TxnStatus.UNKNOWN;
+    }
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/GetCompactionInfoHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetCompactionInfoHandler.java
similarity index 96%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/GetCompactionInfoHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetCompactionInfoHandler.java
index 916f1c877ffb..7da6d9d377e0 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/GetCompactionInfoHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetCompactionInfoHandler.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
@@ -34,7 +34,7 @@ public class GetCompactionInfoHandler implements QueryHandler {
   private final long id;
   private final boolean isTransactionId;
 
-  // language=PostgreSQL
+  // language=SQL
   public static final String SELECT_BY_ID = 
       "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", "
       + "\"CQ_STATE\", \"CQ_TYPE\", \"CQ_TBLPROPERTIES\", \"CQ_WORKER_ID\", \"CQ_START\", \"CQ_RUN_AS\", "
@@ -43,7 +43,7 @@ public class GetCompactionInfoHandler implements QueryHandler {
       + "\"CQ_RETRY_RETENTION\", \"CQ_NEXT_TXN_ID\", \"CQ_TXN_ID\", \"CQ_COMMIT_TIME\", \"CQ_POOL_NAME\", "
       + "\"CQ_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = :id";
 
-  // language=PostgreSQL
+  // language=SQL
   public static final String SELECT_BY_TXN_ID =
       "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", "
           + "\"CQ_STATE\", \"CQ_TYPE\", \"CQ_TBLPROPERTIES\", \"CQ_WORKER_ID\", \"CQ_START\", \"CQ_RUN_AS\", "
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetDatabaseIdHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetDatabaseIdHandler.java
new file mode 100644
index 000000000000..d6c06017fa9b
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetDatabaseIdHandler.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class GetDatabaseIdHandler implements QueryHandler {
+
+  private final String database;
+  private final String catalog;
+
+  public GetDatabaseIdHandler(String database, String catalog) {
+    this.database = database;
+    this.catalog = catalog;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT \"DB_ID\" FROM \"DBS\" WHERE \"NAME\" = :database AND \"CTLG_NAME\" = :catalog";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("database", database)
+        .addValue("catalog", catalog);
+  }
+
+  @Override
+  public Long extractData(ResultSet rs) throws SQLException {
+    if (!rs.next()) {
+      throw new MetaWrapperException(new MetaException("DB with name " + database + " does not exist in catalog " + catalog));
+    }
+    return rs.getLong(1);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetHighWaterMarkHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetHighWaterMarkHandler.java
new file mode 100644
index 000000000000..c6cec1504d96
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetHighWaterMarkHandler.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.dao.EmptyResultDataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class GetHighWaterMarkHandler implements QueryHandler {
+  
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT MAX(\"TXN_ID\") FROM \"TXNS\"";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource();
+  }
+
+  @Override
+  public Long extractData(ResultSet rs) throws SQLException, DataAccessException {
+    if(rs.next()) {
+      long max = rs.getLong(1);
+      if (rs.wasNull()) {
+        throw new EmptyResultDataAccessException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)", 1);
+      } else {
+        return max;
+      }
+    }
+    throw new EmptyResultDataAccessException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)", 1);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLatestCommittedCompactionInfoHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLatestCommittedCompactionInfoHandler.java
new file mode 100644
index 000000000000..b22fb0f0a894
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLatestCommittedCompactionInfoHandler.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.CompactionInfoStruct;
+import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoRequest;
+import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
+
+public class GetLatestCommittedCompactionInfoHandler implements QueryHandler {
+  
+  private final GetLatestCommittedCompactionInfoRequest rqst;
+
+  public GetLatestCommittedCompactionInfoHandler(GetLatestCommittedCompactionInfoRequest rqst) {
+    this.rqst = rqst;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return 
+        "SELECT * FROM ( " +
+          "SELECT \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_TYPE\" FROM \"COMPLETED_COMPACTIONS\" " +
+          "   WHERE \"CC_STATE\" = :succeeded " + 
+          "UNION ALL " +
+          "SELECT \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", " +
+          "\"CQ_PARTITION\" AS \"CC_PARTITION\", \"CQ_TYPE\" AS \"CC_TYPE\" FROM \"COMPACTION_QUEUE\" " +
+          "   WHERE \"CQ_STATE\" = :readyForCleaning) AS compactions " +
+        "WHERE " +
+            "\"CC_DATABASE\" = :dbName AND \"CC_TABLE\" = :tableName AND " +
+            "(\"CC_PARTITION\" IN (:partitionNames) OR :emptyPartitionNames = TRUE) AND " +
+            "(\"CC_ID\" > :id OR :id IS NULL) " +
+        "ORDER BY \"CC_ID\" DESC";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("succeeded", CompactionState.SUCCEEDED.getSqlConst(), Types.CHAR)
+        .addValue("readyForCleaning", CompactionState.READY_FOR_CLEANING.getSqlConst(), Types.CHAR)
+        .addValue("dbName", rqst.getDbname())
+        .addValue("tableName", rqst.getTablename())
+        .addValue("emptyPartitionNames", CollectionUtils.isEmpty(rqst.getPartitionnames()), Types.BOOLEAN)
+        .addValue("partitionNames", CollectionUtils.isEmpty(rqst.getPartitionnames()) ? null : rqst.getPartitionnames(), Types.VARCHAR)
+        .addValue("id", rqst.isSetLastCompactionId() ? rqst.getLastCompactionId() : null, Types.BIGINT);
+  }
+
+  @Override
+  public GetLatestCommittedCompactionInfoResponse extractData(ResultSet rs) throws SQLException, DataAccessException {
+    GetLatestCommittedCompactionInfoResponse response = new GetLatestCommittedCompactionInfoResponse(new ArrayList<>());
+    Set partitionSet = new HashSet<>();
+    while (rs.next()) {
+      CompactionInfoStruct lci = new CompactionInfoStruct();
+      lci.setId(rs.getLong(1));
+      lci.setDbname(rs.getString(2));
+      lci.setTablename(rs.getString(3));
+      String partition = rs.getString(4);
+      if (!rs.wasNull()) {
+        lci.setPartitionname(partition);
+      }
+      lci.setType(TxnUtils.dbCompactionType2ThriftType(rs.getString(5).charAt(0)));
+      // Only put the latest record of each partition into response
+      if (!partitionSet.contains(partition)) {
+        response.addToCompactions(lci);
+        partitionSet.add(partition);
+      }
+    }
+    return response;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLocksByLockId.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLocksByLockId.java
new file mode 100644
index 000000000000..9981d19d2708
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetLocksByLockId.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.txn.entities.LockInfo;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * NEVER call this function without first calling heartbeat(long, long)
+ */
+public class GetLocksByLockId implements QueryHandler> {
+
+  private static final String noSelectQuery = " \"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", " +
+      "\"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_TXNID\" " +
+      "FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = :extLockId";
+  
+  private final long extLockId;
+  private final int limit;
+  private final SQLGenerator sqlGenerator;
+
+  public GetLocksByLockId(long extLockId, int limit, SQLGenerator sqlGenerator) {
+    this.extLockId = extLockId;
+    this.limit = limit;
+    this.sqlGenerator = sqlGenerator;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    if (limit > 0) {
+      return sqlGenerator.addLimitClause(limit, noSelectQuery);
+    } else {
+      return "SELECT " + noSelectQuery;      
+    }
+ }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource().addValue("extLockId", extLockId);
+  }
+
+  @Override
+  public List extractData(ResultSet rs) throws SQLException, DataAccessException {
+    List result = new ArrayList<>();
+    while (rs.next()) {
+      try {
+        result.add(new LockInfo(rs));
+      } catch (MetaException e) {
+        throw new MetaWrapperException(e);
+      }
+    }
+    return result;
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetMaxAllocatedTableWriteIdHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetMaxAllocatedTableWriteIdHandler.java
new file mode 100644
index 000000000000..5fcfd8d974a5
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetMaxAllocatedTableWriteIdHandler.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdRequest;
+import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class GetMaxAllocatedTableWriteIdHandler implements QueryHandler {
+
+  private final MaxAllocatedTableWriteIdRequest rqst;
+
+  public GetMaxAllocatedTableWriteIdHandler(MaxAllocatedTableWriteIdRequest rqst) {
+    this.rqst = rqst;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = :dbName AND \"NWI_TABLE\" = :tableName";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("dbName", rqst.getDbName())
+        .addValue("tableName", rqst.getTableName());
+  }
+
+  @Override
+  public MaxAllocatedTableWriteIdResponse extractData(ResultSet rs) throws SQLException, DataAccessException {
+    long maxWriteId = 0l;
+    if (rs.next()) {
+      // The row contains the nextId not the previously allocated
+      maxWriteId = rs.getLong(1) - 1;
+    }
+    return new MaxAllocatedTableWriteIdResponse(maxWriteId);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnTypeAndLockHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnTypeAndLockHandler.java
new file mode 100644
index 000000000000..889f3cbe1751
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnTypeAndLockHandler.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Optional;
+
+public class GetOpenTxnTypeAndLockHandler implements QueryHandler {
+  
+  private final SQLGenerator sqlGenerator;
+  private final long txnId;
+
+  public GetOpenTxnTypeAndLockHandler(SQLGenerator sqlGenerator, long txnId) {
+    this.sqlGenerator = sqlGenerator;
+    this.txnId = txnId;
+  }
+
+  /**
+   * Note that by definition select for update is divorced from update, i.e. you executeQuery() to read
+   * and then executeUpdate().  One other alternative would be to actually update the row in TXNS but
+   * to the same value as before thus forcing db to acquire write lock for duration of the transaction.
+   * SELECT ... FOR UPDATE locks the row until the transaction commits or rolls back.
+   * Second connection using `SELECT ... FOR UPDATE` will suspend until the lock is released.
+   * @return the txnType wrapped in an {@link Optional}
+   */
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return sqlGenerator.addForUpdateClause(
+        "SELECT \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_ID\" = :id AND \"TXN_STATE\" = :state");
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("id", txnId)
+        .addValue("state", TxnStatus.OPEN.getSqlConst(), Types.CHAR);
+  }
+
+  @Override
+  public TxnType extractData(ResultSet rs) throws SQLException, DataAccessException {
+    return rs.next() ? TxnType.findByValue(rs.getInt(1)) : null;
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnsListHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnsListHandler.java
new file mode 100644
index 000000000000..464af14328d7
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetOpenTxnsListHandler.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.txn.entities.OpenTxn;
+import org.apache.hadoop.hive.metastore.txn.entities.OpenTxnList;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class GetOpenTxnsListHandler implements QueryHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GetOpenTxnsListHandler.class);
+
+  //language=SQL
+  private static final String OPEN_TXNS_QUERY = "SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\", "
+      + "(%s - \"TXN_STARTED\") FROM \"TXNS\" ORDER BY \"TXN_ID\"";
+  //language=SQL
+  private static final String OPEN_TXNS_INFO_QUERY = "SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\", "
+      + "(%s - \"TXN_STARTED\"), \"TXN_USER\", \"TXN_HOST\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\" "
+      + "FROM \"TXNS\" ORDER BY \"TXN_ID\"";
+  
+  private final boolean infoFields;
+  private final long openTxnTimeOutMillis;
+
+  public GetOpenTxnsListHandler(boolean infoFields, long openTxnTimeOutMillis) {
+    this.infoFields = infoFields;
+    this.openTxnTimeOutMillis = openTxnTimeOutMillis;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return String.format(infoFields ? OPEN_TXNS_INFO_QUERY : OPEN_TXNS_QUERY, TxnUtils.getEpochFn(databaseProduct));  
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return null;
+  }
+
+  // We need to figure out the HighWaterMark and the list of open transactions.
+  /*
+   * This method need guarantees from
+   * {@link #openTxns(OpenTxnRequest)} and  {@link #commitTxn(CommitTxnRequest)}.
+   * It will look at the TXNS table and find each transaction between the max(txn_id) as HighWaterMark
+   * and the max(txn_id) before the TXN_OPENTXN_TIMEOUT period as LowWaterMark.
+   * Every transaction that is not found between these will be considered as open, since it may appear later.
+   * openTxns must ensure, that no new transaction will be opened with txn_id below LWM and
+   * commitTxn must ensure, that no committed transaction will be removed before the time period expires.
+   */
+  @Override
+  public OpenTxnList extractData(ResultSet rs) throws SQLException, DataAccessException {
+    /*
+     * We can use the maximum txn_id from the TXNS table as high water mark, since the commitTxn and the Initiator
+     * guarantees, that the transaction with the highest txn_id will never be removed from the TXNS table.
+     * If there is a pending openTxns, that is already acquired it's sequenceId but not yet committed the insert
+     * into the TXNS table, will have either a lower txn_id than HWM and will be listed in the openTxn list,
+     * or will have a higher txn_id and don't effect this getOpenTxns() call.
+     */
+    long hwm = 0;
+    long openTxnLowBoundary = 0;
+    List txnInfos = new ArrayList<>();
+
+    while (rs.next()) {
+      long txnId = rs.getLong(1);
+      long age = rs.getLong(4);
+      hwm = txnId;
+      if (age < openTxnTimeOutMillis) {
+        // We will consider every gap as an open transaction from the previous txnId
+        openTxnLowBoundary++;
+        while (txnId > openTxnLowBoundary) {
+          // Add an empty open transaction for every missing value
+          txnInfos.add(new OpenTxn(openTxnLowBoundary, TxnStatus.OPEN, TxnType.DEFAULT));
+          LOG.debug("Open transaction added for missing value in TXNS {}",
+              JavaUtils.txnIdToString(openTxnLowBoundary));
+          openTxnLowBoundary++;
+        }
+      } else {
+        openTxnLowBoundary = txnId;
+      }
+      TxnStatus state = TxnStatus.fromString(rs.getString(2));
+      if (state == TxnStatus.COMMITTED) {
+        // This is only here, to avoid adding this txnId as possible gap
+        continue;
+      }
+      OpenTxn txnInfo = new OpenTxn(txnId, state, TxnType.findByValue(rs.getInt(3)));
+      if (infoFields) {
+        txnInfo.setUser(rs.getString(5));
+        txnInfo.setHost(rs.getString(6));
+        txnInfo.setStartedTime(rs.getLong(7));
+        txnInfo.setLastHeartBeatTime(rs.getLong(8));
+      }
+      txnInfos.add(txnInfo);
+    }
+    LOG.debug("Got OpenTxnList with hwm: {} and openTxnList size {}.", hwm, txnInfos.size());
+    return new OpenTxnList(hwm, txnInfos);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetTxnDbsUpdatedHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetTxnDbsUpdatedHandler.java
new file mode 100644
index 000000000000..4600064afc3c
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetTxnDbsUpdatedHandler.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Returns the databases updated by txnId.
+ * Queries TXN_TO_WRITE_ID using txnId.
+ */
+public class GetTxnDbsUpdatedHandler implements QueryHandler> {
+  
+  private final long txnId;
+
+  public GetTxnDbsUpdatedHandler(long txnId) {
+    this.txnId = txnId;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT DISTINCT \"T2W_DATABASE\" FROM \"TXN_TO_WRITE_ID\" \"COMMITTED\" WHERE \"T2W_TXNID\" = :txnId";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource().addValue("txnId", txnId);
+  }
+
+  @Override
+  public List extractData(ResultSet rs) throws SQLException, DataAccessException {
+    List dbsUpdated = new ArrayList<>();
+    while (rs.next()) {
+      dbsUpdated.add(rs.getString(1));
+    }
+    return dbsUpdated;
+  }
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetWriteIdsHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetWriteIdsHandler.java
new file mode 100644
index 000000000000..8ae2023a89a3
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/GetWriteIdsHandler.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class GetWriteIdsHandler implements QueryHandler, Long>> {
+
+
+  //language=SQL
+  private static final String SELECT_WRITE_ID_QUERY = "SELECT \"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\" " +
+      "FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_TXNID\" = :txnId ";
+
+  private final LockRequest lockRequest;
+
+  public GetWriteIdsHandler(LockRequest lockRequest) {
+    this.lockRequest = lockRequest;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    StringBuilder sb = new StringBuilder(SELECT_WRITE_ID_QUERY);
+    sb.append(" AND (");
+    for(int i = 0; i< lockRequest.getComponentSize(); i++) {
+      sb.append("(\"T2W_DATABASE\" = ").append(":db").append(i)
+          .append(" AND \"T2W_TABLE\" = :table").append(i).append(")");
+      if(i < lockRequest.getComponentSize() - 1) {
+        sb.append(" OR ");
+      }
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    MapSqlParameterSource params = new MapSqlParameterSource()
+        .addValue("txnId", lockRequest.getTxnid());
+    for(int i = 0; i< lockRequest.getComponentSize(); i++) {
+      params.addValue("db" + i, lockRequest.getComponent().get(i).getDbname());
+      params.addValue("table" + i, lockRequest.getComponent().get(i).getTablename());
+    }
+    return params;
+  }
+
+  @Override
+  public Map, Long> extractData(ResultSet rs) throws SQLException, DataAccessException {
+    Map, Long> writeIds = new HashMap<>();
+    while (rs.next()) {
+      writeIds.put(Pair.of(rs.getString("T2W_DATABASE"), rs.getString("T2W_TABLE")), rs.getLong("T2W_WRITEID"));
+    }
+    return writeIds;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java
new file mode 100644
index 000000000000..33c7719cc711
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import com.sun.tools.javac.util.List;
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.OperationType;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+
+public class LatestTxnIdInConflictHandler implements QueryHandler {
+  
+  private final long txnId;
+
+  public LatestTxnIdInConflictHandler(long txnId) {
+    this.txnId = txnId;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return 
+        " SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" +
+        " FROM \"WRITE_SET\" \"COMMITTED\"" +
+        " INNER JOIN (" +
+        "   SELECT DISTINCT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_TXNID\"" +
+        "   FROM \"TXN_COMPONENTS\"" +
+        "   WHERE \"TC_TXNID\" = :txnId" +
+        "     AND \"TC_OPERATION_TYPE\" IN (:types)" +
+        " ) \"CUR\"" +
+        " ON \"COMMITTED\".\"WS_DATABASE\" = \"CUR\".\"TC_DATABASE\"" +
+        "   AND \"COMMITTED\".\"WS_TABLE\" = \"CUR\".\"TC_TABLE\"" +
+        (TxnHandler.ConfVars.useMinHistoryLevel() ? "" :
+        "   AND \"COMMITTED\".\"WS_OPERATION_TYPE\" != :wsType") + 
+        // For partitioned table we always track writes at partition level (never at table)
+        // and for non partitioned - always at table level, thus the same table should never
+        // have entries with partition key and w/o
+        "   AND (\"COMMITTED\".\"WS_PARTITION\" = \"CUR\".\"TC_PARTITION\" OR" +
+        "     \"CUR\".\"TC_PARTITION\" IS NULL) " +
+        // txns overlap
+        " WHERE \"CUR\".\"TC_TXNID\" <= \"COMMITTED\".\"WS_COMMIT_ID\"";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("txnId", txnId)
+        .addValue("types", List.of(OperationType.UPDATE.getSqlConst(), OperationType.DELETE.getSqlConst()), Types.CHAR)
+        .addValue("wsType", OperationType.INSERT.getSqlConst(), Types.CHAR);        
+  }
+
+  @Override
+  public Long extractData(ResultSet rs) throws SQLException, DataAccessException {
+    return rs.next() ? rs.getLong(1) : -1;
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MetricsInfoHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MetricsInfoHandler.java
new file mode 100644
index 000000000000..d967e222f0fc
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MetricsInfoHandler.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.text.MessageFormat;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.READY_FOR_CLEANING;
+
+public class MetricsInfoHandler implements QueryHandler {
+  
+  public static final MetricsInfoHandler INSTANCE = new MetricsInfoHandler();
+  
+  //language=SQL
+  private static final String SELECT_METRICS_INFO_QUERY =
+      "SELECT * FROM (SELECT COUNT(*) FROM \"TXN_TO_WRITE_ID\") \"TTWID\" CROSS JOIN (" +
+          "SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\") \"CTC\" CROSS JOIN (" +
+          "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" " +
+          "   WHERE \"TXN_STATE\"= :openStatus AND \"TXN_TYPE\" = :replCreatedType) \"TR\" CROSS JOIN (" +
+          "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" " +
+          "   WHERE \"TXN_STATE\"= :openStatus AND \"TXN_TYPE\" != :replCreatedType) \"T\" CROSS JOIN (" +
+          "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" " +
+          "   WHERE \"TXN_STATE\"= :abortedStatus) \"A\" CROSS JOIN (" +
+          "SELECT COUNT(*), ({0} - MIN(\"HL_ACQUIRED_AT\"))/1000 FROM \"HIVE_LOCKS\") \"HL\" CROSS JOIN (" +
+          "SELECT ({0} - MIN(\"CQ_COMMIT_TIME\"))/1000 from \"COMPACTION_QUEUE\" " +
+          "   WHERE \"CQ_STATE\"= :readyForCleaningState) OLDEST_CLEAN";
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return MessageFormat.format(SELECT_METRICS_INFO_QUERY, TxnUtils.getEpochFn(databaseProduct));
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("openStatus", TxnStatus.OPEN.getSqlConst(), Types.CHAR)
+        .addValue("abortedStatus", TxnStatus.ABORTED.getSqlConst(), Types.CHAR)
+        .addValue("replCreatedType", TxnType.REPL_CREATED.getValue())
+        .addValue("readyForCleaningState", Character.toString(READY_FOR_CLEANING), Types.CHAR);
+  }
+
+  @Override
+  public MetricsInfo extractData(ResultSet rs) throws SQLException, DataAccessException {
+    MetricsInfo metrics = new MetricsInfo();
+    if (rs.next()) {
+      metrics.setTxnToWriteIdCount(rs.getInt(1));
+      metrics.setCompletedTxnsCount(rs.getInt(2));
+      metrics.setOpenReplTxnsCount(rs.getInt(3));
+      metrics.setOldestOpenReplTxnId(rs.getInt(4));
+      metrics.setOldestOpenReplTxnAge(rs.getInt(5));
+      metrics.setOpenNonReplTxnsCount(rs.getInt(6));
+      metrics.setOldestOpenNonReplTxnId(rs.getInt(7));
+      metrics.setOldestOpenNonReplTxnAge(rs.getInt(8));
+      metrics.setAbortedTxnsCount(rs.getInt(9));
+      metrics.setOldestAbortedTxnId(rs.getInt(10));
+      metrics.setOldestAbortedTxnAge(rs.getInt(11));
+      metrics.setLocksCount(rs.getInt(12));
+      metrics.setOldestLockAge(rs.getInt(13));
+      metrics.setOldestReadyForCleaningAge(rs.getInt(14));
+    }
+    return metrics;
+  }
+  
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/MinUncommittedTxnIdHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MinUncommittedTxnIdHandler.java
similarity index 96%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/MinUncommittedTxnIdHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MinUncommittedTxnIdHandler.java
index ba527fb4d39b..327963a5a8d5 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/MinUncommittedTxnIdHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/MinUncommittedTxnIdHandler.java
@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/OpenTxnTimeoutLowBoundaryTxnIdHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/OpenTxnTimeoutLowBoundaryTxnIdHandler.java
new file mode 100644
index 000000000000..6fb65b75716f
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/OpenTxnTimeoutLowBoundaryTxnIdHandler.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
+
+public class OpenTxnTimeoutLowBoundaryTxnIdHandler implements QueryHandler {
+  
+  private final long openTxnTimeOutMillis;
+
+  public OpenTxnTimeoutLowBoundaryTxnIdHandler(long openTxnTimeOutMillis) {
+    this.openTxnTimeOutMillis = openTxnTimeOutMillis;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT MAX(\"TXN_ID\") FROM \"TXNS\" WHERE \"TXN_STARTED\" < (" + getEpochFn(databaseProduct) + " - "
+        + openTxnTimeOutMillis + ")";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource();
+  }
+
+  @Override
+  public Long extractData(ResultSet rs) throws SQLException, DataAccessException {
+    rs.next();
+    long result = rs.getLong(1);
+    if (rs.wasNull()) {
+      /*
+       * TXNS always contains at least one transaction,
+       * the row where txnid = (select max(txnid) where txn_started < epoch - TXN_OPENTXN_TIMEOUT) is never deleted
+       */
+      throw new SQLException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)");
+    }
+    return result;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/ReadyToCleanAbortHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java
similarity index 97%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/ReadyToCleanAbortHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java
index 4965bdd7baf3..4940d384095b 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/ReadyToCleanAbortHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
-import org.apache.hadoop.hive.metastore.txn.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/ReadyToCleanHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanHandler.java
similarity index 92%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/ReadyToCleanHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanHandler.java
index 66a1481c2366..0f22b00e1976 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/impl/ReadyToCleanHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanHandler.java
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.impl;
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
 import org.springframework.dao.DataAccessException;
@@ -38,13 +39,11 @@
 
 public class ReadyToCleanHandler implements QueryHandler> {
   
-  private final boolean useMinHistoryWriteId;
   private final long minOpenTxnWaterMark;
   private final long retentionTime;
   private final int fetchSize;
 
-  public ReadyToCleanHandler(Configuration conf, boolean useMinHistoryWriteId, long minOpenTxnWaterMark, long retentionTime) {
-    this.useMinHistoryWriteId = useMinHistoryWriteId;
+  public ReadyToCleanHandler(Configuration conf, long minOpenTxnWaterMark, long retentionTime) {
     this.minOpenTxnWaterMark = minOpenTxnWaterMark;
     this.retentionTime = retentionTime;
     this.fetchSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.COMPACTOR_FETCH_SIZE);
@@ -64,7 +63,7 @@ public String getParameterizedQueryString(DatabaseProduct databaseProduct) throw
         " \"CQ_ID\", \"cq1\".\"CQ_DATABASE\", \"cq1\".\"CQ_TABLE\", \"cq1\".\"CQ_PARTITION\"," +
             "  \"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TBLPROPERTIES\", \"CQ_RETRY_RETENTION\", " +
             "  \"CQ_NEXT_TXN_ID\"";
-    if (useMinHistoryWriteId) {
+    if (TxnHandler.ConfVars.useMinHistoryWriteId()) {
       queryStr += ", \"MIN_OPEN_WRITE_ID\"";
     }
     queryStr +=
@@ -80,7 +79,7 @@ public String getParameterizedQueryString(DatabaseProduct databaseProduct) throw
             "    OR \"cq1\".\"CQ_PARTITION\" IS NULL AND \"cq2\".\"CQ_PARTITION\" IS NULL)" +
             "  AND \"CQ_HIGHEST_WRITE_ID\" = \"MIN_WRITE_ID_HWM\" ";
 
-    if (useMinHistoryWriteId) {
+    if (TxnHandler.ConfVars.useMinHistoryWriteId()) {
       queryStr +=
           "LEFT JOIN (" +
               "  SELECT MIN(\"MH_WRITEID\") \"MIN_OPEN_WRITE_ID\", \"MH_DATABASE\", \"MH_TABLE\"" +
@@ -120,7 +119,7 @@ public List extractData(ResultSet rs) throws SQLException, DataA
       info.properties = rs.getString(8);
       info.retryRetention = rs.getInt(9);
       info.nextTxnId = rs.getLong(10);
-      if (useMinHistoryWriteId) {
+      if (TxnHandler.ConfVars.useMinHistoryWriteId()) {
         info.minOpenWriteId = rs.getLong(11);
       }
       infos.add(info);
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowCompactHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowCompactHandler.java
new file mode 100644
index 000000000000..d8d2c8da473b
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowCompactHandler.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+
+import static org.apache.commons.lang3.StringUtils.isBlank;
+import static org.apache.commons.lang3.StringUtils.isNotBlank;
+
+public class ShowCompactHandler implements QueryHandler {
+  
+  private static final String DEFAULT_POOL_NAME = "default";
+
+  //language=SQL
+  private static final String SHOW_COMPACTION_QUERY =
+      " XX.* FROM ( SELECT " +
+          "  \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\", " +
+          "  \"CQ_STATE\" AS \"CC_STATE\", \"CQ_TYPE\" AS \"CC_TYPE\", \"CQ_WORKER_ID\" AS \"CC_WORKER_ID\", " +
+          "  \"CQ_START\" AS \"CC_START\", -1 \"CC_END\", \"CQ_RUN_AS\" AS \"CC_RUN_AS\", " +
+          "  \"CQ_HADOOP_JOB_ID\" AS \"CC_HADOOP_JOB_ID\", \"CQ_ID\" AS \"CC_ID\", \"CQ_ERROR_MESSAGE\" AS \"CC_ERROR_MESSAGE\", " +
+          "  \"CQ_ENQUEUE_TIME\" AS \"CC_ENQUEUE_TIME\", \"CQ_WORKER_VERSION\" AS \"CC_WORKER_VERSION\", " +
+          "  \"CQ_INITIATOR_ID\" AS \"CC_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" AS \"CC_INITIATOR_VERSION\", " +
+          "  \"CQ_CLEANER_START\" AS \"CC_CLEANER_START\", \"CQ_POOL_NAME\" AS \"CC_POOL_NAME\", \"CQ_TXN_ID\" AS \"CC_TXN_ID\", " +
+          "  \"CQ_NEXT_TXN_ID\" AS \"CC_NEXT_TXN_ID\", \"CQ_COMMIT_TIME\" AS \"CC_COMMIT_TIME\", " +
+          "  \"CQ_HIGHEST_WRITE_ID\" AS \"CC_HIGHEST_WRITE_ID\" " +
+          "FROM " +
+          "  \"COMPACTION_QUEUE\" " +
+          "UNION ALL " +
+          "SELECT " +
+          "  \"CC_DATABASE\" , \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", \"CC_WORKER_ID\", " +
+          "  \"CC_START\", \"CC_END\", \"CC_RUN_AS\", \"CC_HADOOP_JOB_ID\", \"CC_ID\", \"CC_ERROR_MESSAGE\", " +
+          "  \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\", " +
+          "   -1 , \"CC_POOL_NAME\", \"CC_TXN_ID\", \"CC_NEXT_TXN_ID\", \"CC_COMMIT_TIME\", " +
+          "  \"CC_HIGHEST_WRITE_ID\"" +
+          "FROM " +
+          "  \"COMPLETED_COMPACTIONS\" ) XX " +
+          "WHERE " +
+          "  (\"CC_ID\" = :id OR :id IS NULL) AND " +
+          "  (\"CC_DATABASE\" = :dbName OR :dbName IS NULL) AND " +
+          "  (\"CC_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+          "  (\"CC_PARTITION\" = :partition OR :partition IS NULL) AND " +
+          "  (\"CC_STATE\" = :state OR :state IS NULL) AND " +
+          "  (\"CC_TYPE\" = :type OR :type IS NULL) AND " +
+          "  (\"CC_POOL_NAME\" = :poolName OR :poolName IS NULL)";
+
+  //language=SQL
+  private static final String SHOW_COMPACTION_ORDERBY_CLAUSE =
+      " ORDER BY CASE " +
+          "   WHEN \"CC_END\" > \"CC_START\" and \"CC_END\" > \"CC_COMMIT_TIME\" " +
+          "     THEN \"CC_END\" " +
+          "   WHEN \"CC_START\" > \"CC_COMMIT_TIME\" " +
+          "     THEN \"CC_START\" " +
+          "   ELSE \"CC_COMMIT_TIME\" " +
+          " END desc ," +
+          " \"CC_ENQUEUE_TIME\" asc";
+
+
+  private final ShowCompactRequest request;
+  private final SQLGenerator sqlGenerator;
+
+
+  public ShowCompactHandler(ShowCompactRequest request, SQLGenerator sqlGenerator) {
+    this.request = request;
+    this.sqlGenerator = sqlGenerator;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    String noSelectQuery = SHOW_COMPACTION_QUERY + getShowCompactSortingOrderClause(request);
+    int rowLimit = (int) request.getLimit();    
+    if (rowLimit > 0) {
+      return sqlGenerator.addLimitClause(rowLimit, noSelectQuery);
+    } else {
+    }
+    return "SELECT " + noSelectQuery;
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    Long id = request.getId() > 0 ? request.getId() : null;
+    try {
+      return new MapSqlParameterSource()
+          .addValue("id", id, Types.BIGINT)
+          .addValue("dbName", request.getDbName(), Types.VARCHAR)
+          .addValue("tableName", request.getTbName(), Types.VARCHAR)
+          .addValue("partition", request.getPartName(), Types.VARCHAR)
+          .addValue("state", request.getState(), Types.CHAR)
+          .addValue("type", request.getType() == null ? null : Character.toString(TxnUtils.thriftCompactionType2DbType(request.getType())), Types.CHAR)
+          .addValue("poolName", request.getPoolName(), Types.VARCHAR);
+    } catch (MetaException e) {
+      throw new MetaWrapperException(e);
+    }
+  }
+
+  @Override
+  public ShowCompactResponse extractData(ResultSet rs) throws SQLException, DataAccessException {
+    ShowCompactResponse response = new ShowCompactResponse(new ArrayList<>());
+    while (rs.next()) {
+      ShowCompactResponseElement e = new ShowCompactResponseElement();
+      e.setDbname(rs.getString(1));
+      e.setTablename(rs.getString(2));
+      e.setPartitionname(rs.getString(3));
+      e.setState(CompactionState.fromSqlConst(rs.getString(4)).toString());
+      try {
+        e.setType(TxnUtils.dbCompactionType2ThriftType(rs.getString(5).charAt(0)));
+      } catch (SQLException ex) {
+        //do nothing to handle RU/D if we add another status
+      }
+      e.setWorkerid(rs.getString(6));
+      long start = rs.getLong(7);
+      if (!rs.wasNull()) {
+        e.setStart(start);
+      }
+      long endTime = rs.getLong(8);
+      if (endTime != -1) {
+        e.setEndTime(endTime);
+      }
+      e.setRunAs(rs.getString(9));
+      e.setHadoopJobId(rs.getString(10));
+      e.setId(rs.getLong(11));
+      e.setErrorMessage(rs.getString(12));
+      long enqueueTime = rs.getLong(13);
+      if (!rs.wasNull()) {
+        e.setEnqueueTime(enqueueTime);
+      }
+      e.setWorkerVersion(rs.getString(14));
+      e.setInitiatorId(rs.getString(15));
+      e.setInitiatorVersion(rs.getString(16));
+      long cleanerStart = rs.getLong(17);
+      if (!rs.wasNull() && (cleanerStart != -1)) {
+        e.setCleanerStart(cleanerStart);
+      }
+      String poolName = rs.getString(18);
+      if (isBlank(poolName)) {
+        e.setPoolName(DEFAULT_POOL_NAME);
+      } else {
+        e.setPoolName(poolName);
+      }
+      e.setTxnId(rs.getLong(19));
+      e.setNextTxnId(rs.getLong(20));
+      e.setCommitTime(rs.getLong(21));
+      e.setHightestTxnId(rs.getLong(22));
+      response.addToCompacts(e);      
+    }
+    return response;
+  }
+
+  private String getShowCompactSortingOrderClause(ShowCompactRequest request) {
+    String sortingOrder = request.getOrder();
+    return isNotBlank(sortingOrder) ? "  ORDER BY  " + sortingOrder : SHOW_COMPACTION_ORDERBY_CLAUSE;
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowLocksHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowLocksHandler.java
new file mode 100644
index 000000000000..eefacad35252
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ShowLocksHandler.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.LockTypeComparator;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.hadoop.hive.metastore.txn.entities.LockInfo;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.apache.hadoop.hive.metastore.utils.LockTypeUtil;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.io.Serializable;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.txn.entities.LockInfo.LOCK_ACQUIRED;
+import static org.apache.hadoop.hive.metastore.txn.entities.LockInfo.LOCK_WAITING;
+
+public class ShowLocksHandler implements QueryHandler {
+
+  private final ShowLocksRequest request;
+
+  public ShowLocksHandler(ShowLocksRequest request) {
+    this.request = request;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return 
+        "SELECT \"HL_LOCK_EXT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", " +
+        "\"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", \"HL_ACQUIRED_AT\", \"HL_USER\", \"HL_HOST\", \"HL_LOCK_INT_ID\"," +
+        "\"HL_BLOCKEDBY_EXT_ID\", \"HL_BLOCKEDBY_INT_ID\", \"HL_AGENT_INFO\" FROM \"HIVE_LOCKS\"" +
+        "WHERE " +
+            "(\"HL_DB\" = :dbName OR :dbName IS NULL) AND " +
+            "(\"HL_TABLE\" = :tableName OR :tableName IS NULL) AND " +
+            "(\"HL_PARTITION\" = :partition OR :partition IS NULL) AND " +
+            "(\"HL_TXNID\" = :txnId OR :txnId IS NULL)";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("dbName", request.getDbname(), Types.VARCHAR)
+        .addValue("tableName", request.getTablename(), Types.VARCHAR)
+        .addValue("partition", request.getPartname(), Types.VARCHAR)
+        .addValue("txnId", request.isSetTxnid() ? request.getTxnid() : null, Types.BIGINT);
+  }
+
+  @Override
+  public ShowLocksResponse extractData(ResultSet rs) throws SQLException, DataAccessException {
+    ShowLocksResponse rsp = new ShowLocksResponse();
+    List elems = new ArrayList<>();
+    List sortedList = new ArrayList<>();
+    while (rs.next()) {
+      ShowLocksResponseElement e = new ShowLocksResponseElement();
+      e.setLockid(rs.getLong(1));
+      long txnid = rs.getLong(2);
+      if (!rs.wasNull()) e.setTxnid(txnid);
+      e.setDbname(rs.getString(3));
+      e.setTablename(rs.getString(4));
+      String partition = rs.getString(5);
+      if (partition != null) e.setPartname(partition);
+      switch (rs.getString(6).charAt(0)) {
+        case LOCK_ACQUIRED:
+          e.setState(LockState.ACQUIRED);
+          break;
+        case LOCK_WAITING:
+          e.setState(LockState.WAITING);
+          break;
+        default:
+          throw new SQLException("Unknown lock state " + rs.getString(6).charAt(0));
+      }
+
+      char lockChar = rs.getString(7).charAt(0);
+      LockType lockType = LockTypeUtil.getLockTypeFromEncoding(lockChar)
+          .orElseThrow(() -> new SQLException("Unknown lock type: " + lockChar));
+      e.setType(lockType);
+
+      e.setLastheartbeat(rs.getLong(8));
+      long acquiredAt = rs.getLong(9);
+      if (!rs.wasNull()) e.setAcquiredat(acquiredAt);
+      e.setUser(rs.getString(10));
+      e.setHostname(rs.getString(11));
+      e.setLockIdInternal(rs.getLong(12));
+      long id = rs.getLong(13);
+      if (!rs.wasNull()) {
+        e.setBlockedByExtId(id);
+      }
+      id = rs.getLong(14);
+      if (!rs.wasNull()) {
+        e.setBlockedByIntId(id);
+      }
+      e.setAgentInfo(rs.getString(15));
+      sortedList.add(new LockInfoExt(e));
+    }
+    //this ensures that "SHOW LOCKS" prints the locks in the same order as they are examined
+    //by checkLock() - makes diagnostics easier.
+    Collections.sort(sortedList, new LockInfoComparator());
+    for(LockInfoExt lockInfoExt : sortedList) {
+      elems.add(lockInfoExt.e);
+    }
+    rsp.setLocks(elems);
+    return rsp;    
+  }
+
+  /**
+   * used to sort entries in {@link org.apache.hadoop.hive.metastore.api.ShowLocksResponse}
+   */
+  private static class LockInfoExt extends LockInfo {
+    private final ShowLocksResponseElement e;
+    LockInfoExt(ShowLocksResponseElement e) {
+      super(e);
+      this.e = e;
+    }
+  }
+
+  private static class LockInfoComparator implements Comparator, Serializable {
+    private final LockTypeComparator lockTypeComparator = new LockTypeComparator();
+
+    public int compare(LockInfo info1, LockInfo info2) {
+      // We sort by state (acquired vs waiting) and then by LockType, then by id
+      if (info1.getState() == LockState.ACQUIRED &&
+          info2.getState() != LockState .ACQUIRED) {
+        return -1;
+      }
+      if (info1.getState() != LockState.ACQUIRED &&
+          info2.getState() == LockState .ACQUIRED) {
+        return 1;
+      }
+
+      int sortByType = lockTypeComparator.compare(info1.getType(), info2.getType());
+      if(sortByType != 0) {
+        return sortByType;
+      }
+      if (info1.getExtLockId() < info2.getExtLockId()) {
+        return -1;
+      } else if (info1.getExtLockId() > info2.getExtLockId()) {
+        return 1;
+      } else {
+        if (info1.getIntLockId() < info2.getIntLockId()) {
+          return -1;
+        } else if (info1.getTxnId() > info2.getIntLockId()) {
+          return 1;
+        } else {
+          return 0;
+        }
+      }
+    }
+  }  
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TablesWithAbortedTxnsHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TablesWithAbortedTxnsHandler.java
new file mode 100644
index 000000000000..16b768dc34bb
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TablesWithAbortedTxnsHandler.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Set;
+import java.util.TreeSet;
+
+public class TablesWithAbortedTxnsHandler implements QueryHandler> {
+
+  //language=SQL
+  private static final String SELECT_TABLES_WITH_X_ABORTED_TXNS =
+      "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXN_COMPONENTS\" " +
+          "INNER JOIN \"TXNS\" ON \"TC_TXNID\" = \"TXN_ID\" WHERE \"TXN_STATE\" = :abortedState " +
+          "GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" HAVING COUNT(\"TXN_ID\") > :txnThreshold";
+  
+  private final int txnThreshold;
+
+  public TablesWithAbortedTxnsHandler(int txnThreshold) {
+    this.txnThreshold = txnThreshold;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return SELECT_TABLES_WITH_X_ABORTED_TXNS;
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("abortedState", TxnStatus.ABORTED.getSqlConst(), Types.CHAR)
+        .addValue("txnThreshold", txnThreshold);
+  }
+
+  @Override
+  public Set extractData(ResultSet rs) throws SQLException, DataAccessException {
+    Set resourceNames = new TreeSet<>();
+    while (rs.next()) {
+      String resourceName = rs.getString(1) + "." + rs.getString(2);
+      String partName = rs.getString(3);
+      resourceName = partName != null ? resourceName + "#" + partName : resourceName;
+      resourceNames.add(resourceName);
+    }
+    return resourceNames;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TargetTxnIdListHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TargetTxnIdListHandler.java
new file mode 100644
index 000000000000..70d5ed7ecfbf
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TargetTxnIdListHandler.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TargetTxnIdListHandler implements QueryHandler> {
+  
+  private final String replPolicy;
+  private final List sourceTxnsIds;
+
+  public TargetTxnIdListHandler(String replPolicy, List sourceTxnsIds) {
+    this.replPolicy = replPolicy;
+    this.sourceTxnsIds = sourceTxnsIds;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT \"RTM_TARGET_TXN_ID\" FROM \"REPL_TXN_MAP\" " +
+        "WHERE \"RTM_SRC_TXN_ID\" IN (:txnIds) AND \"RTM_REPL_POLICY\" = :policy";
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("txnIds", sourceTxnsIds, Types.BIGINT)
+        .addValue("policy", replPolicy);
+  }
+
+  @Override
+  public List extractData(ResultSet rs) throws SQLException, DataAccessException {
+    List targetTxnIdList = new ArrayList<>();
+    while (rs.next()) {
+      targetTxnIdList.add(rs.getLong(1));
+    }
+    return targetTxnIdList;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TxnIdForWriteIdHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TxnIdForWriteIdHandler.java
new file mode 100644
index 000000000000..9fdb3465d724
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/TxnIdForWriteIdHandler.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.jdbc.queries;
+
+import org.apache.hadoop.hive.metastore.DatabaseProduct;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+import org.springframework.jdbc.core.namedparam.SqlParameterSource;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class TxnIdForWriteIdHandler implements QueryHandler {
+  
+  private final long writeId;
+  private final String dbName;
+  private final String tableName;
+
+  public TxnIdForWriteIdHandler(long writeId, String dbName, String tableName) {
+    this.writeId = writeId;
+    this.dbName = dbName;
+    this.tableName = tableName;
+  }
+
+  @Override
+  public String getParameterizedQueryString(DatabaseProduct databaseProduct) throws MetaException {
+    return "SELECT \"T2W_TXNID\" FROM \"TXN_TO_WRITE_ID\" WHERE"
+        + " \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND \"T2W_WRITEID\" = " + writeId;
+  }
+
+  @Override
+  public SqlParameterSource getQueryParameters() {
+    return new MapSqlParameterSource()
+        .addValue("writeId", writeId)
+        .addValue("dbName", dbName)
+        .addValue("tableName", tableName);
+  }
+
+  @Override
+  public Long extractData(ResultSet rs) throws SQLException, DataAccessException {
+    long txnId = -1;
+    if (rs.next()) {
+      txnId = rs.getLong(1);
+    }
+    return txnId;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/RetryPropagation.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/RetryPropagation.java
similarity index 97%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/RetryPropagation.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/RetryPropagation.java
index 9171985b2121..90721440d645 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/RetryPropagation.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/RetryPropagation.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.retryhandling;
+package org.apache.hadoop.hive.metastore.txn.retry;
 
 /**
  * Specifies how the Retry context propagation is done in various situations.
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetry.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetry.java
similarity index 96%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetry.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetry.java
index 6c4b5493cc82..e1427a3d250e 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetry.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetry.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.retryhandling;
+package org.apache.hadoop.hive.metastore.txn.retry;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryCallProperties.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryCallProperties.java
similarity index 98%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryCallProperties.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryCallProperties.java
index b3614211c9c6..213040a60997 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryCallProperties.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryCallProperties.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.retryhandling;
+package org.apache.hadoop.hive.metastore.txn.retry;
 
 /**
  * Contains all the properties which can alter the behavior of a retry-call in 
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ContextNode.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryException.java
similarity index 58%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ContextNode.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryException.java
index f983430f4a1f..ae4634bd2d11 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ContextNode.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryException.java
@@ -15,28 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn;
+package org.apache.hadoop.hive.metastore.txn.retry;
+
+import org.springframework.dao.DataAccessException;
+
+import java.sql.SQLException;
+
+import static org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler.MANUAL_RETRY;
 
 /**
- * Used for storing values in {@link ThreadLocal}. By using the {@link #getParent()} property, it is possible to have a 
- * stack-like set of values to support embedded/nested contexts.
- * @param  The type of the value to store
+ * This exception can be used to trigger a manual retry in {@link SqlRetryHandler}. Since it is extending
+ * {@link RuntimeException} there is no need to mark it in the throws section of the methods.
  */
-public class ContextNode {
-  
-  private final ContextNode parent;
-  private final T value;
+public class SqlRetryException extends DataAccessException {
 
-  public ContextNode(ContextNode parent, T value) {
-    this.parent = parent;
-    this.value = value;
+  public SqlRetryException(String message) {
+    super(message, new SQLException(message, MANUAL_RETRY));
   }
 
-  public ContextNode getParent() {
-    return parent;
-  }
-
-  public T getValue() {
-    return value;
-  }
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryFunction.java
similarity index 89%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryFunction.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryFunction.java
index 59a25bf39841..8c7445e0ad37 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryFunction.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryFunction.java
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.retryhandling;
+package org.apache.hadoop.hive.metastore.txn.retry;
 
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.thrift.TException;
 import org.springframework.dao.DataAccessException;
 
 import java.sql.SQLException;
@@ -31,6 +32,6 @@
 @FunctionalInterface
 public interface SqlRetryFunction {
   
-  T execute() throws SQLException, DataAccessException, MetaException;
+  T execute() throws SQLException, DataAccessException, TException;
   
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryHandler.java
similarity index 91%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryHandler.java
index 780a3186e01a..0b7127b6826e 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retryhandling/SqlRetryHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/retry/SqlRetryHandler.java
@@ -15,15 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn.retryhandling;
+package org.apache.hadoop.hive.metastore.txn.retry;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.ContextNode;
 import org.apache.hadoop.hive.metastore.txn.MetaWrapperException;
+import org.apache.hadoop.hive.metastore.utils.StackThreadLocal;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.dao.DataAccessException;
@@ -43,9 +44,9 @@ public class SqlRetryHandler {
   private static final Logger LOG = LoggerFactory.getLogger(SqlRetryHandler.class);
 
   private static final int ALLOWED_REPEATED_DEADLOCKS = 10;
-  private static final String MANUAL_RETRY = "ManualRetry";
+  static final String MANUAL_RETRY = "ManualRetry";
 
-  private final ThreadLocal> threadLocal = new ThreadLocal<>();
+  private final StackThreadLocal threadLocal = new StackThreadLocal<>();
 
   /**
    * Derby specific concurrency control
@@ -89,11 +90,11 @@ public SqlRetryHandler(Configuration conf, DatabaseProduct databaseProduct){
    * @param  Type of the result
    * @throws MetaException Thrown in case of execution error.
    */
-  public  Result executeWithRetry(SqlRetryCallProperties properties, SqlRetryFunction function) throws MetaException {
+  public  Result executeWithRetry(SqlRetryCallProperties properties, SqlRetryFunction function) throws TException {
     Objects.requireNonNull(function, "RetryFunction cannot be null!");
     Objects.requireNonNull(properties, "RetryCallProperties cannot be null!");
 
-    if (threadLocal.get() != null && properties.getRetryPropagation().canJoinContext()) {
+    if (threadLocal.isSet() && properties.getRetryPropagation().canJoinContext()) {
       /*
         If there is a context in the ThreadLocal and we are allowed to join it, we can skip establishing a nested retry-call.
       */
@@ -123,15 +124,10 @@ public  Result executeWithRetry(SqlRetryCallProperties properties, SqlRe
       if (properties.isLockInternally()) {
         lockInternal();
       }
-      threadLocal.set(new ContextNode<>(threadLocal.get(), new Object()));
+      threadLocal.set(new Object());
       return executeWithRetryInternal(properties, function);
     } finally {
-      ContextNode node = threadLocal.get();
-      if (node != null && node.getParent() != null) {
-        threadLocal.set(node.getParent());
-      } else {
-        threadLocal.remove();
-      }
+      threadLocal.unset();
       if (properties.isLockInternally()) {
         unlockInternal();
       }
@@ -139,16 +135,18 @@ public  Result executeWithRetry(SqlRetryCallProperties properties, SqlRe
   }
 
   private  Result executeWithRetryInternal(SqlRetryCallProperties properties, SqlRetryFunction function) 
-      throws MetaException {
+      throws TException {
     LOG.debug("Running retry function:" + properties);
 
     try {
       return function.execute();
-    } catch (DataAccessException e) {
+    } catch (DataAccessException | SQLException e) {
       SQLException sqlEx = null;
       if (e.getCause() instanceof SQLException) {
         sqlEx = (SQLException) e.getCause();
-      }
+      } else if (e instanceof SQLException) {
+        sqlEx = (SQLException) e;
+      }      
       if (sqlEx != null) {
         if (checkDeadlock(sqlEx, properties)) {
           properties.setDeadlockCount(properties.getDeadlockCount() - 1); 
@@ -165,9 +163,6 @@ private  Result executeWithRetryInternal(SqlRetryCallProperties properti
       //unwrap and re-throw
       LOG.error("Execution failed for caller {}", properties, e.getCause());
       throw (MetaException) e.getCause();
-    } catch (Exception e) {
-      LOG.error("Execution failed for caller {}", properties, e);
-      throw new MetaException("Failed to execute function: " + properties.getCaller() + ", details:" + e.getMessage());
     }
   }
   
@@ -181,6 +176,7 @@ private boolean checkDeadlock(SQLException e, SqlRetryCallProperties properties)
         try {
           Thread.sleep(waitInterval);
         } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
           // NOP
         }
         return true;
@@ -231,7 +227,7 @@ private boolean waitForRetry(String caller, String errMsg, int retryCount) {
       try {
         Thread.sleep(retryInterval);
       } catch (InterruptedException ex) {
-        //
+        Thread.currentThread().interrupt();
       }
       return true;
     } else {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/StackThreadLocal.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/StackThreadLocal.java
new file mode 100644
index 000000000000..63a4062460ac
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/StackThreadLocal.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.utils;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+public class StackThreadLocal {
+
+  private final ThreadLocal> threadLocal = new ThreadLocal<>();
+
+  public void set(T value) {
+    Deque stack = threadLocal.get();
+    if (stack == null) {
+      stack = new ArrayDeque<>();
+    }
+    stack.push(value);
+    threadLocal.set(stack);
+  }
+
+  public void unset() {
+    Deque stack = threadLocal.get();
+    stack.pop();
+    if (stack.isEmpty()) {
+      threadLocal.remove();
+    }
+  }
+  
+  public T get() {
+    Deque stack = threadLocal.get();
+    if (stack != null) {
+      return stack.peek();
+    } else {
+      throw new IllegalStateException("There is no context to return!");
+    }
+  }
+  
+  public boolean isSet() {
+    return threadLocal.get() != null;
+  }  
+
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyCustomRDBMS.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyCustomRDBMS.java
index 053fbe4b2ec0..444edd8812ff 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyCustomRDBMS.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyCustomRDBMS.java
@@ -98,7 +98,7 @@ public boolean supportsGetGeneratedKeys() {
     return true;
   }
   @Override
-  public boolean isDuplicateKeyError(SQLException ex) {
+  public boolean isDuplicateKeyError(Throwable t) {
    return true;
   }
   @Override
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/retry/TestSqlRetryHandler.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/retry/TestSqlRetryHandler.java
new file mode 100644
index 000000000000..3948faaaadc4
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/retry/TestSqlRetryHandler.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.txn.retry;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.sql.SQLException;
+
+public class TestSqlRetryHandler {
+
+  @Test
+  public void testRetryableRegex() {
+    HiveConf conf = new HiveConf();
+    SQLException sqlException = new SQLException("ORA-08177: can't serialize access for this transaction", "72000");
+    // Note that we have 3 regex'es below
+    conf.setVar(HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX, "^Deadlock detected, roll back,.*08177.*,.*08178.*");
+    boolean result = SqlRetryHandler.isRetryable(conf, sqlException);
+    Assert.assertTrue("regex should be retryable", result);
+
+    sqlException = new SQLException("This error message, has comma in it");
+    conf.setVar(HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX, ".*comma.*");
+    result = SqlRetryHandler.isRetryable(conf, sqlException);
+    Assert.assertTrue("regex should be retryable", result);
+  }
+  
+}
diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
index f68c396de7e7..16c002cd572e 100644
--- a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
+++ b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
@@ -1366,12 +1366,15 @@ public void testTimeOutReaper() throws Exception {
       .withRecordWriter(writer)
       .withHiveConf(conf)
       .connect();
-
-    connection.beginTransaction();
+    
+    HiveConf houseKeeperConf = new HiveConf(conf);
     //ensure txn timesout
-    conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 2, TimeUnit.MILLISECONDS);
+    houseKeeperConf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 100, TimeUnit.MILLISECONDS);
     AcidHouseKeeperService houseKeeperService = new AcidHouseKeeperService();
-    houseKeeperService.setConf(conf);
+    houseKeeperService.setConf(houseKeeperConf);
+
+    connection.beginTransaction();
+    Thread.sleep(150);
     houseKeeperService.run();
     try {
       //should fail because the TransactionBatch timed out
@@ -1390,6 +1393,7 @@ public void testTimeOutReaper() throws Exception {
     connection.beginTransaction();
     connection.commitTransaction();
     connection.beginTransaction();
+    Thread.sleep(150);
     houseKeeperService.run();
     try {
       //should fail because the TransactionBatch timed out

From f9497749d1f3868464d01e0356a27abe0612a089 Mon Sep 17 00:00:00 2001
From: Sourabh Badhya 
Date: Mon, 18 Dec 2023 17:35:21 +0530
Subject: [PATCH 103/179] HIVE-27749: SchemaTool initSchema fails on Mariadb
 10.2 (Sourabh Badhya, reviewed by Denys Kuzmenko, Zsolt Miskolczi)

Closes #4949
---
 .../src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql    | 2 +-
 .../src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql     | 2 +-
 .../mysql/upgrade-4.0.0-alpha-1-to-4.0.0-alpha-2.mysql.sql    | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql
index 7aa3d46f2f41..3016f8021f61 100644
--- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql
+++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql
@@ -892,7 +892,7 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE`
     PRIMARY KEY (`NNI_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-ALTER TABLE `NOTIFICATION_SEQUENCE` MODIFY COLUMN `NNI_ID` BIGINT(20) GENERATED ALWAYS AS (1) STORED NOT NULL;
+ALTER TABLE `NOTIFICATION_SEQUENCE` ADD CONSTRAINT `ONE_ROW_CONSTRAINT` CHECK (`NNI_ID` = 1);
 
 INSERT INTO `NOTIFICATION_SEQUENCE` (`NEXT_EVENT_ID`) SELECT * from (select 1 as `NOTIFICATION_SEQUENCE`) a WHERE (SELECT COUNT(*) FROM `NOTIFICATION_SEQUENCE`) = 0;
 
diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql
index 04a99721fc64..e9d37d357748 100644
--- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql
+++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql
@@ -894,7 +894,7 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE`
     PRIMARY KEY (`NNI_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-ALTER TABLE `NOTIFICATION_SEQUENCE` MODIFY COLUMN `NNI_ID` BIGINT(20) GENERATED ALWAYS AS (1) STORED NOT NULL;
+ALTER TABLE `NOTIFICATION_SEQUENCE` ADD CONSTRAINT `ONE_ROW_CONSTRAINT` CHECK (`NNI_ID` = 1);
 
 INSERT INTO `NOTIFICATION_SEQUENCE` (`NEXT_EVENT_ID`) SELECT * from (select 1 as `NOTIFICATION_SEQUENCE`) a WHERE (SELECT COUNT(*) FROM `NOTIFICATION_SEQUENCE`) = 0;
 
diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-1-to-4.0.0-alpha-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-1-to-4.0.0-alpha-2.mysql.sql
index e7a9a6f21163..951ac24507a1 100644
--- a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-1-to-4.0.0-alpha-2.mysql.sql
+++ b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-1-to-4.0.0-alpha-2.mysql.sql
@@ -5,8 +5,8 @@ ALTER TABLE `COMPLETED_COMPACTIONS` ADD COLUMN `CC_NEXT_TXN_ID` bigint;
 ALTER TABLE `COMPLETED_COMPACTIONS` ADD COLUMN `CC_TXN_ID` bigint;
 ALTER TABLE `COMPLETED_COMPACTIONS` ADD COLUMN `CC_COMMIT_TIME` bigint;
 
--- HIVE-26324
-ALTER TABLE `NOTIFICATION_SEQUENCE` MODIFY COLUMN `NNI_ID` INT GENERATED ALWAYS AS (1) STORED NOT NULL;
+-- HIVE-27749
+ALTER TABLE `NOTIFICATION_SEQUENCE` ADD CONSTRAINT `ONE_ROW_CONSTRAINT` CHECK (`NNI_ID` = 1);
 
 -- HIVE-26443
 ALTER TABLE `COMPACTION_QUEUE` ADD COLUMN `CQ_POOL_NAME` VARCHAR(128);

From 4a057a7cea11029dbda7adaec09117d46e56d925 Mon Sep 17 00:00:00 2001
From: seonggon 
Date: Tue, 19 Dec 2023 02:30:09 +0900
Subject: [PATCH 104/179] HIVE-27952: Use SslContextFactory.Server() instead of
 SslContextFactory (#4947)

---
 common/src/java/org/apache/hive/http/HttpServer.java            | 2 +-
 .../src/main/java/org/apache/hive/hcatalog/templeton/Main.java  | 2 +-
 .../apache/hive/service/cli/thrift/ThriftHttpCLIService.java    | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/common/src/java/org/apache/hive/http/HttpServer.java b/common/src/java/org/apache/hive/http/HttpServer.java
index 2d73a920d47f..83555fc5c73e 100644
--- a/common/src/java/org/apache/hive/http/HttpServer.java
+++ b/common/src/java/org/apache/hive/http/HttpServer.java
@@ -536,7 +536,7 @@ Connector createChannelConnector(int queueSize, Builder b) {
     if (!b.useSSL) {
       connector = new ServerConnector(webServer, http);
     } else {
-      SslContextFactory sslContextFactory = new SslContextFactory();
+      SslContextFactory sslContextFactory = new SslContextFactory.Server();
       sslContextFactory.setKeyStorePath(b.keyStorePath);
       sslContextFactory.setKeyStoreType(b.keyStoreType == null || b.keyStoreType.isEmpty() ?
           KeyStore.getDefaultType(): b.keyStoreType);
diff --git a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java
index 66fa5eb4ae8a..04d99254e4ed 100644
--- a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java
+++ b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Main.java
@@ -286,7 +286,7 @@ private Connector createChannelConnector(Server server) {
 
     if (conf.getBoolean(AppConfig.USE_SSL, false)) {
       LOG.info("Using SSL for templeton.");
-      SslContextFactory sslContextFactory = new SslContextFactory();
+      SslContextFactory sslContextFactory = new SslContextFactory.Server();
       sslContextFactory.setKeyStorePath(conf.get(AppConfig.KEY_STORE_PATH, DEFAULT_KEY_STORE_PATH));
       sslContextFactory.setKeyStorePassword(conf.get(AppConfig.KEY_STORE_PASSWORD, DEFAULT_KEY_STORE_PASSWORD));
       Set excludedSSLProtocols = Sets.newHashSet(Splitter.on(",").trimResults().omitEmptyStrings()
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
index bc20ea7797eb..63d23c11d38a 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
@@ -160,7 +160,7 @@ public void onClosed(Connection connection) {
         if (keyStoreAlgorithm.isEmpty()) {
           keyStoreAlgorithm = KeyManagerFactory.getDefaultAlgorithm();
         }
-        SslContextFactory sslContextFactory = new SslContextFactory();
+        SslContextFactory sslContextFactory = new SslContextFactory.Server();
         String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",");
         LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols));
         sslContextFactory.addExcludeProtocols(excludedProtocols);

From f265cc25905d0bdbdc65a16720e33fb21ee79da9 Mon Sep 17 00:00:00 2001
From: Ramesh Kumar 
Date: Wed, 20 Dec 2023 01:02:36 -0800
Subject: [PATCH 105/179] HIVE-27876 Incorrect query results on tables with
 ClusterBy & SortBy (Ramesh Kumar Thangarajan, reviewed by Krisztian Kasa,
 Attila Turoczy)

---
 .../org/apache/hadoop/hive/conf/HiveConf.java |   6 +-
 .../clientpositive/groupby_sort_2_23.q        |  10 +
 .../clientpositive/cbo_rp_auto_join1.q.out    | 457 ++++++++++++------
 .../llap/auto_sortmerge_join_10.q.out         | 295 +++++++----
 .../clientpositive/llap/bucket_groupby.q.out  |  89 +++-
 .../llap/groupby_sort_2_23.q.out              | 180 +++++++
 6 files changed, 779 insertions(+), 258 deletions(-)
 create mode 100644 ql/src/test/queries/clientpositive/groupby_sort_2_23.q
 create mode 100644 ql/src/test/results/clientpositive/llap/groupby_sort_2_23.q.out

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 714df4c22a96..1fa63ae3821a 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2019,10 +2019,10 @@ public static enum ConfVars {
     HIVEMULTIGROUPBYSINGLEREDUCER("hive.multigroupby.singlereducer", true,
         "Whether to optimize multi group by query to generate single M/R  job plan. If the multi group by query has \n" +
         "common group by keys, it will be optimized to generate single M/R job."),
-    HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", true,
+    HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", false,
         "If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform \n" +
-        "the group by in the mapper by using BucketizedHiveInputFormat. The only downside to this\n" +
-        "is that it limits the number of mappers to the number of files."),
+        "the group by in the mapper by using BucketizedHiveInputFormat. This can only work if the number of files to be\n" +
+        "processed is exactly 1. The downside to this is that it limits the number of mappers to the number of files."),
     HIVE_DEFAULT_NULLS_LAST("hive.default.nulls.last", true,
         "Whether to set NULLS LAST as the default null ordering for ASC order and " +
             "NULLS FIRST for DESC order."),
diff --git a/ql/src/test/queries/clientpositive/groupby_sort_2_23.q b/ql/src/test/queries/clientpositive/groupby_sort_2_23.q
new file mode 100644
index 000000000000..b241bee68550
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/groupby_sort_2_23.q
@@ -0,0 +1,10 @@
+set hive.mapred.mode=nonstrict;
+set hive.map.aggr=true;
+set hive.explain.user=false;
+
+create table test_bucket(age int, name string, dept string) clustered by (age, name) sorted by (age asc, name asc) into 2 buckets stored as ORC;
+insert into test_bucket values (1, 'user1', 'dept1'), ( 2, 'user2' , 'dept2');
+insert into test_bucket values (1, 'user1', 'dept1'), ( 2, 'user2' , 'dept2');
+
+explain vectorization detail select age, name, count(*) from test_bucket group by  age, name having count(*) > 1;
+select age, name, count(*) from test_bucket group by  age, name having count(*) > 1;
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index 8f3788d40fab..5bdf0edc2b44 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -92,8 +92,10 @@ POSTHOOK: Input: default@tbl2_n12
 #### A masked pattern was here ####
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-2 depends on stages: Stage-1, Stage-4
+  Stage-3 depends on stages: Stage-2
+  Stage-4 is a root stage
+  Stage-0 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-1
@@ -112,49 +114,53 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
+                  bucketGroup: true
                   keys: key (type: int)
-                  mode: final
+                  minReductionHashAggr: 0.99
+                  mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col1 (type: bigint)
-                    outputColumnNames: key, $f1
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: $f1 (type: bigint)
+                    value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
           TableScan
-            alias: subq1:b
-            filterExpr: key is not null (type: boolean)
-            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: key (type: int)
-                outputColumnNames: key
-                Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: count()
-                  keys: key (type: int)
-                  mode: final
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col1 (type: bigint)
-                    outputColumnNames: key, $f1
-                    Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: $f1 (type: bigint)
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
+          TableScan
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -181,7 +187,7 @@ STAGE PLANS:
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-2
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -205,6 +211,50 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: subq1:b
+            filterExpr: key is not null (type: boolean)
+            Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int)
+                outputColumnNames: key
+                Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  bucketGroup: true
+                  keys: key (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
   Stage: Stage-0
     Fetch Operator
       limit: -1
@@ -255,7 +305,8 @@ POSTHOOK: Input: default@tbl2_n12
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-1
@@ -273,20 +324,42 @@ STAGE PLANS:
                 outputColumnNames: key
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
+                  bucketGroup: true
                   keys: key (type: int)
-                  mode: final
+                  minReductionHashAggr: 0.99
+                  mode: hash
                   outputColumnNames: _col0
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int)
-                    outputColumnNames: key
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key
+          Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: subq2:subq1:b
             filterExpr: key is not null (type: boolean)
@@ -325,7 +398,7 @@ STAGE PLANS:
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-2
+  Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -421,16 +494,20 @@ POSTHOOK: Input: default@tbl1_n13
 POSTHOOK: Input: default@tbl2_n12
 #### A masked pattern was here ####
 STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-3, Stage-5
   Stage-3 is a root stage
-  Stage-1 depends on stages: Stage-3
-  Stage-0 depends on stages: Stage-1
+  Stage-4 is a root stage
+  Stage-5 depends on stages: Stage-4, Stage-6
+  Stage-6 is a root stage
+  Stage-0 depends on stages: Stage-2
 
 STAGE PLANS:
-  Stage: Stage-3
+  Stage: Stage-1
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src1:subq1:a
+            alias: src2:subq2:a
             filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
@@ -442,23 +519,89 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
+                  bucketGroup: true
                   keys: key (type: int)
-                  mode: final
+                  minReductionHashAggr: 0.99
+                  mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col1 (type: bigint)
-                    outputColumnNames: key, $f1
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: $f1 (type: bigint)
+                    value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
           TableScan
-            alias: src1:subq1:b
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
+          TableScan
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
+          TableScan
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint), $f10 (type: bigint)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+               Inner Join 0 to 2
+          keys:
+            0 key (type: int)
+            1 key (type: int)
+            2 key (type: int)
+          outputColumnNames: key, $f1, $f10, $f11, $f100
+          Statistics: Num rows: 6 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: key (type: int), ($f11 * $f100) (type: bigint), ($f1 * $f10) (type: bigint)
+            outputColumnNames: key, cnt1, cnt11
+            Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src2:subq2:b
             filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
@@ -470,30 +613,27 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
+                  bucketGroup: true
                   keys: key (type: int)
-                  mode: final
+                  minReductionHashAggr: 0.99
+                  mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col1 (type: bigint)
-                    outputColumnNames: key, $f1
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: $f1 (type: bigint)
+                    value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
       Reduce Operator Tree:
-        Join Operator
-          condition map:
-               Inner Join 0 to 1
-          keys:
-            0 key (type: int)
-            1 key (type: int)
-          outputColumnNames: key, $f1, $f10
-          Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -501,11 +641,11 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-1
+  Stage: Stage-4
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src2:subq2:a
+            alias: src1:subq1:a
             filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
@@ -517,23 +657,74 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
+                  bucketGroup: true
                   keys: key (type: int)
-                  mode: final
+                  minReductionHashAggr: 0.99
+                  mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col1 (type: bigint)
-                    outputColumnNames: key, $f1
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: $f1 (type: bigint)
+                    value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
           TableScan
-            alias: src2:subq2:b
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
+          TableScan
+            Reduce Output Operator
+              key expressions: key (type: int)
+              null sort order: z
+              sort order: +
+              Map-reduce partition columns: key (type: int)
+              Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: $f1 (type: bigint)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 key (type: int)
+            1 key (type: int)
+          outputColumnNames: key, $f1, $f10
+          Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-6
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1:subq1:b
             filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
@@ -545,51 +736,33 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
+                  bucketGroup: true
                   keys: key (type: int)
-                  mode: final
+                  minReductionHashAggr: 0.99
+                  mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), _col1 (type: bigint)
-                    outputColumnNames: key, $f1
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: int)
-                      Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: $f1 (type: bigint)
-          TableScan
-            Reduce Output Operator
-              key expressions: key (type: int)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: int)
-              Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: $f1 (type: bigint), $f10 (type: bigint)
+                    value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
       Reduce Operator Tree:
-        Join Operator
-          condition map:
-               Inner Join 0 to 1
-               Inner Join 0 to 2
-          keys:
-            0 key (type: int)
-            1 key (type: int)
-            2 key (type: int)
-          outputColumnNames: key, $f1, $f10, $f11, $f100
-          Statistics: Num rows: 6 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: key (type: int), ($f11 * $f100) (type: bigint), ($f1 * $f10) (type: bigint)
-            outputColumnNames: key, cnt1, cnt11
-            Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 6 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: int)
+          mode: mergepartial
+          outputColumnNames: key, $f1
+          Statistics: Num rows: 6 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out
index 7b93c1d6a54f..35538b911745 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_10.q.out
@@ -372,7 +372,9 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -384,11 +386,22 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
                     Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
+                    Group By Operator
+                      bucketGroup: true
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                      Dummy Store
+                      Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -397,31 +410,54 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
                     Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      mode: final
+                    Select Operator
+                      expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                      Merge Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
-                        Group By Operator
-                          aggregations: count()
-                          minReductionHashAggr: 0.85714287
-                          mode: hash
-                          outputColumnNames: _col0
-                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                          Reduce Output Operator
-                            null sort order: 
-                            sort order: 
-                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                            value expressions: _col0 (type: bigint)
-            Execution mode: llap
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
         Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.85714287
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -492,6 +528,8 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -503,11 +541,22 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
                     Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: key (type: int)
+                    Group By Operator
+                      bucketGroup: true
+                      keys: key (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                      Dummy Store
+                      Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -516,30 +565,53 @@ STAGE PLANS:
                   Filter Operator
                     predicate: (key < 6) (type: boolean)
                     Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      mode: final
+                    Select Operator
+                      expressions: key (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                      Merge Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col0 (type: int)
-                          1 _col0 (type: int)
-                        outputColumnNames: _col1
+                      Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                        Select Operator
-                          expressions: _col1 (type: int)
-                          outputColumnNames: _col0
-                          Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-                          Reduce Output Operator
-                            key expressions: _col0 (type: int)
-                            null sort order: z
-                            sort order: +
-                            Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
         Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col1
+                Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
@@ -717,10 +789,29 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: t1
+                  Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t2
@@ -730,51 +821,50 @@ STAGE PLANS:
                     predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
+                      bucketGroup: true
                       keys: key (type: int)
-                      mode: final
+                      minReductionHashAggr: 0.4
+                      mode: hash
                       outputColumnNames: _col0
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                      Select Operator
-                        expressions: true (type: boolean), _col0 (type: int)
-                        outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
-                        Dummy Store
-            Map Operator Tree:
-                TableScan
-                  alias: t1
-                  Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col1 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: key (type: int)
+                    expressions: _col0 (type: int)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Merge Join Operator
-                      condition map:
-                           Left Outer Join 0 to 1
-                      keys:
-                        0 _col0 (type: int)
-                        1 _col1 (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
-                      Filter Operator
-                        predicate: _col1 is null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(_col0)
+                      minReductionHashAggr: 0.4
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        Select Operator
-                          expressions: _col0 (type: int)
-                          outputColumnNames: _col0
-                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                          Group By Operator
-                            aggregations: count(_col0)
-                            minReductionHashAggr: 0.4
-                            mode: hash
-                            outputColumnNames: _col0
-                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                            Reduce Output Operator
-                              null sort order: 
-                              sort order: 
-                              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                              value expressions: _col0 (type: bigint)
-            Execution mode: llap
-        Reducer 2 
+                        value expressions: _col0 (type: bigint)
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -789,6 +879,25 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: true (type: boolean), _col0 (type: int)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: int)
+                    Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out b/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
index e0a6a183faa6..f85c3bbf22c9 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
@@ -1512,30 +1512,79 @@ POSTHOOK: Input: default@clustergroupby
 POSTHOOK: Input: default@clustergroupby@ds=102
 #### A masked pattern was here ####
 STAGE DEPENDENCIES:
-  Stage-0 is a root stage
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: clustergroupby
+                  filterExpr: (ds = '102') (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Top N Key Operator
+                    sort order: ++
+                    keys: key (type: string), value (type: string)
+                    null sort order: zz
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    top n: 10
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: key, value
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        bucketGroup: true
+                        keys: key (type: string), value (type: string)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1, _col2
+                        Statistics: Num rows: 316 Data size: 58776 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string), _col1 (type: string)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                          Statistics: Num rows: 316 Data size: 58776 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col2 (type: bigint)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 316 Data size: 58776 Basic stats: COMPLETE Column stats: COMPLETE
+                Limit
+                  Number of rows: 10
+                  Statistics: Num rows: 10 Data size: 1860 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: string), _col2 (type: bigint)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 10 Data size: 950 Basic stats: COMPLETE Column stats: COMPLETE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 10 Data size: 950 Basic stats: COMPLETE Column stats: COMPLETE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
   Stage: Stage-0
     Fetch Operator
       limit: 10
       Processor Tree:
-        TableScan
-          alias: clustergroupby
-          filterExpr: (ds = '102') (type: boolean)
-          Select Operator
-            expressions: key (type: string), value (type: string)
-            outputColumnNames: key, value
-            Group By Operator
-              aggregations: count()
-              keys: key (type: string), value (type: string)
-              mode: final
-              outputColumnNames: _col0, _col1, _col2
-              Limit
-                Number of rows: 10
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: bigint)
-                  outputColumnNames: _col0, _col1
-                  ListSink
+        ListSink
 
 PREHOOK: query: select key, count(1) from clustergroupby  where ds='102'  group by key, value limit 10
 PREHOOK: type: QUERY
@@ -1547,7 +1596,6 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@clustergroupby
 POSTHOOK: Input: default@clustergroupby@ds=102
 #### A masked pattern was here ####
-0	3
 10	1
 100	2
 103	2
@@ -1555,8 +1603,9 @@ POSTHOOK: Input: default@clustergroupby@ds=102
 105	1
 11	1
 111	1
-113	2
 114	1
+0	3
+113	2
 PREHOOK: query: drop table clustergroupby
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@clustergroupby
diff --git a/ql/src/test/results/clientpositive/llap/groupby_sort_2_23.q.out b/ql/src/test/results/clientpositive/llap/groupby_sort_2_23.q.out
new file mode 100644
index 000000000000..780cb6ccd27b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/groupby_sort_2_23.q.out
@@ -0,0 +1,180 @@
+PREHOOK: query: create table test_bucket(age int, name string, dept string) clustered by (age, name) sorted by (age asc, name asc) into 2 buckets stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_bucket
+POSTHOOK: query: create table test_bucket(age int, name string, dept string) clustered by (age, name) sorted by (age asc, name asc) into 2 buckets stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_bucket
+PREHOOK: query: insert into test_bucket values (1, 'user1', 'dept1'), ( 2, 'user2' , 'dept2')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@test_bucket
+POSTHOOK: query: insert into test_bucket values (1, 'user1', 'dept1'), ( 2, 'user2' , 'dept2')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@test_bucket
+POSTHOOK: Lineage: test_bucket.age SCRIPT []
+POSTHOOK: Lineage: test_bucket.dept SCRIPT []
+POSTHOOK: Lineage: test_bucket.name SCRIPT []
+PREHOOK: query: insert into test_bucket values (1, 'user1', 'dept1'), ( 2, 'user2' , 'dept2')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@test_bucket
+POSTHOOK: query: insert into test_bucket values (1, 'user1', 'dept1'), ( 2, 'user2' , 'dept2')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@test_bucket
+POSTHOOK: Lineage: test_bucket.age SCRIPT []
+POSTHOOK: Lineage: test_bucket.dept SCRIPT []
+POSTHOOK: Lineage: test_bucket.name SCRIPT []
+PREHOOK: query: explain vectorization detail select age, name, count(*) from test_bucket group by  age, name having count(*) > 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_bucket
+#### A masked pattern was here ####
+POSTHOOK: query: explain vectorization detail select age, name, count(*) from test_bucket group by  age, name having count(*) > 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_bucket
+#### A masked pattern was here ####
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: test_bucket
+                  Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                  TableScan Vectorization:
+                      native: true
+                      vectorizationSchemaColumns: [0:age:int, 1:name:string, 2:dept:string, 3:ROW__ID:struct, 4:ROW__IS__DELETED:boolean]
+                  Select Operator
+                    expressions: age (type: int), name (type: string)
+                    outputColumnNames: age, name
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [0, 1]
+                    Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count()
+                      bucketGroup: true
+                      Group By Vectorization:
+                          aggregators: VectorUDAFCountStar(*) -> bigint
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 0:int, col 1:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: [0]
+                      keys: age (type: int), name (type: string)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: zz
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkMultiKeyOperator
+                            keyColumns: 0:int, 1:string
+                            native: true
+                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                            valueColumns: 2:bigint
+                        Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col2 (type: bigint)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+            Map Vectorization:
+                enabled: true
+                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+                inputFormatFeatureSupport: [DECIMAL_64]
+                featureSupportInUse: [DECIMAL_64]
+                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 3
+                    includeColumns: [0, 1]
+                    dataColumns: age:int, name:string, dept:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true
+                reduceColumnNullOrder: zz
+                reduceColumnSortOrder: ++
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 3
+                    dataColumns: KEY._col0:int, KEY._col1:string, VALUE._col0:bigint
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                Group By Vectorization:
+                    aggregators: VectorUDAFCountMerge(col 2:bigint) -> bigint
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    keyExpressions: col 0:int, col 1:string
+                    native: false
+                    vectorProcessingMode: MERGE_PARTIAL
+                    projectedOutputColumnNums: [0]
+                keys: KEY._col0 (type: int), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 2 Data size: 202 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  Filter Vectorization:
+                      className: VectorFilterOperator
+                      native: true
+                      predicateExpression: FilterLongColGreaterLongScalar(col 2:bigint, val 1)
+                  predicate: (_col2 > 1L) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    File Sink Vectorization:
+                        className: VectorFileSinkOperator
+                        native: false
+                    Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select age, name, count(*) from test_bucket group by  age, name having count(*) > 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_bucket
+#### A masked pattern was here ####
+POSTHOOK: query: select age, name, count(*) from test_bucket group by  age, name having count(*) > 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_bucket
+#### A masked pattern was here ####
+1	user1	2
+2	user2	2

From 6671e1a91f288d8552840a2517912e5820d0204b Mon Sep 17 00:00:00 2001
From: akshat0395 
Date: Wed, 20 Dec 2023 14:01:53 +0530
Subject: [PATCH 106/179] HIVE-27963: Build failure when license-maven-plugin
 downloads bsd-license.php (Akshat Mathur reviewed by Stamatis Zampetakis,
 Ayush Saxena)

1. Add BSD-2-CLAUSE in licenseUrlFileNames patterns to unify downloaded files and avoid the build failure.
2. Upgrade plugin version from 2.1.0 to 2.3.0 (unrelated to the failure but still beneficial)

Close apache/hive#4963
---
 packaging/pom.xml | 4 ++++
 pom.xml           | 2 +-
 2 files changed, 5 insertions(+), 1 deletion(-)

diff --git a/packaging/pom.xml b/packaging/pom.xml
index 274dcc7d08b7..9a25658475d1 100644
--- a/packaging/pom.xml
+++ b/packaging/pom.xml
@@ -158,6 +158,10 @@
                   \Qhttps://opensource.org/licenses/Apache-2.0\E
                   \Qhttp://www.apache.org/licenses/\E
                 
+                
+                  https?://(www\.)?opensource.org/licenses/bsd-license.php
+                  https?://(www\.)?opensource.org/licenses/BSD-2-Clause
+                
                 
                   https?://(www\.)?opensource.org/licenses/BSD-3-Clause
                 
diff --git a/pom.xml b/pom.xml
index 62ba8c78d613..b6959e10ceef 100644
--- a/pom.xml
+++ b/pom.xml
@@ -100,7 +100,7 @@
     3.5.0
     3.0.0-M4
     2.7.10
-    2.1.0
+    2.3.0
     
     1.10.1
     1.10.13

From 36ce858163a19e29eafe4a8d3307191bc28fc175 Mon Sep 17 00:00:00 2001
From: Stamatis Zampetakis 
Date: Fri, 8 Dec 2023 13:22:56 +0100
Subject: [PATCH 107/179] HIVE-27919: Constant reduction in CBO does not work
 for FROM_UNIXTIME, DATE_ADD, DATE_SUB, TO_UNIX_TIMESTAMP (Stamatis Zampetakis
 reviewed by Akshat Mathur, Krisztian Kasa)

Constant reduction does not work because the functions are declared as
dynamic (isDynamicFunction returns true). However, the dynamic
declaration is wrong cause none of the above depends on context
variables; they all operate on concrete parameters and require one or
more inputs.

Moreover, DATE_ADD, DATE_SUB, and FROM_UNIXTIME are not time functions,
so it is wrong to extend the SqlAbsctractTimeFunction class. The
overrides in SqlAbsctractTimeFunction are not correct/relevant to these
functions so the changes here address this as well.

Overview of the changes:
1. Turn DATE_ADD, DATE_SUB, and FROM_UNIXTIME to regular SqlFunctions
and pass the correct return type inference strategy. The operand type
inference and type checker can remain null as they were before since
they are not used currently in Hive.
2. Change the type family for FROM_UNIXTIME to reflect that the
function returns a string (and not date or time).
3. Create and pass an appropriate operand checker for FROM_UNIXTIME
(minor since it's not used at the moment).
4. Remove isDynamicFunction override from TO_UNIX_TIMESTAMP
(which is wrong), to enable constant reduction.
5. Finalize classes and make them non-instantiable

Constant reduction in CBO allows some further optimizations to kick-in
such as the removal of the (always true) filter operator in
constant_prop_coalesce.q.out and the transformation to
dynamic partition hash join (DPHJ) in tez_dynpart_hashjoin_4.q.out.

Note, that without the changes here the DPHJ transformation for the
query in tez_dynpart_hashjoin_4.q fails due to inconsistencies on the
way constant folding is performed at the Operator (physical) layer
(HIVE-27658).

Close apache/hive#4932
---
 .../reloperators/HiveDateAddSqlOperator.java  |  15 +--
 .../reloperators/HiveDateSubSqlOperator.java  |  15 +--
 .../HiveFromUnixTimeSqlOperator.java          |  29 +++--
 .../HiveToUnixTimestampSqlOperator.java       |  12 +--
 .../clientpositive/cbo_constantfolding.q      |   5 +
 .../llap/cbo_constantfolding.q.out            |  60 +++++++++++
 .../llap/constant_prop_coalesce.q.out         |  10 +-
 .../llap/tez_dynpart_hashjoin_4.q.out         | 101 +++++++++---------
 8 files changed, 165 insertions(+), 82 deletions(-)
 create mode 100644 ql/src/test/queries/clientpositive/cbo_constantfolding.q
 create mode 100644 ql/src/test/results/clientpositive/llap/cbo_constantfolding.q.out

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateAddSqlOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateAddSqlOperator.java
index af9b12ee6f71..ef865e4d22c2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateAddSqlOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateAddSqlOperator.java
@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
 
-import org.apache.calcite.sql.fun.SqlAbstractTimeFunction;
-import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.ReturnTypes;
 
-public class HiveDateAddSqlOperator extends SqlAbstractTimeFunction {
-  public static final HiveDateAddSqlOperator INSTANCE = new HiveDateAddSqlOperator();
+public final class HiveDateAddSqlOperator {
+  public static final SqlFunction INSTANCE =
+      new SqlFunction("DATE_ADD", SqlKind.OTHER_FUNCTION, ReturnTypes.DATE_NULLABLE, null, null,
+          SqlFunctionCategory.TIMEDATE);
 
-  protected HiveDateAddSqlOperator() {
-    super("DATE_ADD", SqlTypeName.DATE);
+  private HiveDateAddSqlOperator() {
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateSubSqlOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateSubSqlOperator.java
index 4f737126f02a..d1c002111895 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateSubSqlOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateSubSqlOperator.java
@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
 
-import org.apache.calcite.sql.fun.SqlAbstractTimeFunction;
-import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.ReturnTypes;
 
-public class HiveDateSubSqlOperator extends SqlAbstractTimeFunction {
-  public static final HiveDateSubSqlOperator INSTANCE = new HiveDateSubSqlOperator();
+public final class HiveDateSubSqlOperator {
+  public static final SqlFunction INSTANCE =
+      new SqlFunction("DATE_SUB", SqlKind.OTHER_FUNCTION, ReturnTypes.DATE_NULLABLE, null, null,
+          SqlFunctionCategory.TIMEDATE);
 
-  protected HiveDateSubSqlOperator() {
-    super("DATE_SUB", SqlTypeName.DATE);
+  private HiveDateSubSqlOperator() {
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFromUnixTimeSqlOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFromUnixTimeSqlOperator.java
index 22d91b5a537a..54c68ec89be6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFromUnixTimeSqlOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFromUnixTimeSqlOperator.java
@@ -18,15 +18,32 @@
 
 package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
 
-import org.apache.calcite.sql.fun.SqlAbstractTimeFunction;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
+
+import java.util.Arrays;
 
 /**
- * Calcite SQL operator mapping to FROM_UNIXTIME Hive UDF
+ * Calcite SQL operator mapping to FROM_UNIXTIME Hive UDF.
+ * 

+ * The return type of the function is declared as {@code VARCHAR(100)} since it is highly unlikely that a user will + * request a timestamp format that requires more than 100 characters. + *

*/ -public class HiveFromUnixTimeSqlOperator extends SqlAbstractTimeFunction { - public static final HiveFromUnixTimeSqlOperator INSTANCE = new HiveFromUnixTimeSqlOperator(); - protected HiveFromUnixTimeSqlOperator() { - super("FROM_UNIXTIME", SqlTypeName.TIMESTAMP); +public final class HiveFromUnixTimeSqlOperator { + public static final SqlFunction INSTANCE = new SqlFunction("FROM_UNIXTIME", + SqlKind.OTHER_FUNCTION, + ReturnTypes.explicit(SqlTypeName.VARCHAR, 100).andThen(SqlTypeTransforms.TO_NULLABLE), + null, + OperandTypes.family(Arrays.asList(SqlTypeFamily.INTEGER, SqlTypeFamily.STRING), number -> number == 1), + SqlFunctionCategory.STRING); + + private HiveFromUnixTimeSqlOperator() { } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveToUnixTimestampSqlOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveToUnixTimestampSqlOperator.java index 372aa3002522..18751ce73929 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveToUnixTimestampSqlOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveToUnixTimestampSqlOperator.java @@ -27,15 +27,13 @@ /** * Sql UNIX_TIMESTAMP calcite operator. */ -public class HiveToUnixTimestampSqlOperator { +public final class HiveToUnixTimestampSqlOperator { public static final SqlFunction INSTANCE = new SqlFunction("UNIX_TIMESTAMP", SqlKind.OTHER_FUNCTION, ReturnTypes.BIGINT, null, OperandTypes.or(OperandTypes.NILADIC, OperandTypes.or(OperandTypes.STRING, OperandTypes.TIMESTAMP, OperandTypes.DATE), - OperandTypes.STRING_STRING), SqlFunctionCategory.NUMERIC) { - @Override - public boolean isDynamicFunction() { - return true; - } - }; + OperandTypes.STRING_STRING), SqlFunctionCategory.NUMERIC); + + private HiveToUnixTimestampSqlOperator() { + } } diff --git a/ql/src/test/queries/clientpositive/cbo_constantfolding.q b/ql/src/test/queries/clientpositive/cbo_constantfolding.q new file mode 100644 index 000000000000..25f6f1345b78 --- /dev/null +++ b/ql/src/test/queries/clientpositive/cbo_constantfolding.q @@ -0,0 +1,5 @@ +EXPLAIN CBO SELECT DATE_ADD('2023-01-01', 1); +EXPLAIN CBO SELECT DATE_SUB('2023-01-01', 1); +EXPLAIN CBO SELECT FROM_UNIXTIME(1672560000); +EXPLAIN CBO SELECT TO_UNIX_TIMESTAMP(DATE '2023-01-01'); +EXPLAIN CBO SELECT UNIX_TIMESTAMP(DATE '2023-01-01'); diff --git a/ql/src/test/results/clientpositive/llap/cbo_constantfolding.q.out b/ql/src/test/results/clientpositive/llap/cbo_constantfolding.q.out new file mode 100644 index 000000000000..5e316937c46a --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/cbo_constantfolding.q.out @@ -0,0 +1,60 @@ +PREHOOK: query: EXPLAIN CBO SELECT DATE_ADD('2023-01-01', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT DATE_ADD('2023-01-01', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(_o__c0=[CAST(2023-01-02:DATE):DATE]) + HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) + +PREHOOK: query: EXPLAIN CBO SELECT DATE_SUB('2023-01-01', 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT DATE_SUB('2023-01-01', 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(_o__c0=[CAST(2022-12-31:DATE):DATE]) + HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) + +PREHOOK: query: EXPLAIN CBO SELECT FROM_UNIXTIME(1672560000) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT FROM_UNIXTIME(1672560000) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(_o__c0=[CAST(_UTF-16LE'2023-01-01 00:00:00':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"]) + HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) + +PREHOOK: query: EXPLAIN CBO SELECT TO_UNIX_TIMESTAMP(DATE '2023-01-01') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT TO_UNIX_TIMESTAMP(DATE '2023-01-01') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(_o__c0=[CAST(1672560000:BIGINT):BIGINT]) + HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) + +PREHOOK: query: EXPLAIN CBO SELECT UNIX_TIMESTAMP(DATE '2023-01-01') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN CBO SELECT UNIX_TIMESTAMP(DATE '2023-01-01') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +CBO PLAN: +HiveProject(_o__c0=[CAST(1672560000:BIGINT):BIGINT]) + HiveTableScan(table=[[_dummy_database, _dummy_table]], table:alias=[_dummy_table]) + diff --git a/ql/src/test/results/clientpositive/llap/constant_prop_coalesce.q.out b/ql/src/test/results/clientpositive/llap/constant_prop_coalesce.q.out index cb0cd4de2c8d..43bb312ec5e2 100644 --- a/ql/src/test/results/clientpositive/llap/constant_prop_coalesce.q.out +++ b/ql/src/test/results/clientpositive/llap/constant_prop_coalesce.q.out @@ -50,12 +50,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2 UDTF Operator function name: stack - Filter Operator - predicate: if((col0 = 1), true, true) (type: boolean) - Select Operator - expressions: if((col0 = 1), 20210308L, 20210309L) (type: bigint) - outputColumnNames: _col0 - ListSink + Select Operator + expressions: if((col0 = 1), 20210308L, 20210309L) (type: bigint) + outputColumnNames: _col0 + ListSink PREHOOK: query: select * from ( select diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out index 5ca79e22b2b0..9c999893817e 100644 --- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out +++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_4.q.out @@ -70,6 +70,7 @@ POSTHOOK: query: ALTER TABLE table_b UPDATE STATISTICS FOR COLUMN product_sk SET POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS POSTHOOK: Input: default@table_b POSTHOOK: Output: default@table_b +Warning: Map Join MAPJOIN[31][bigTable=?] in task 'Map 1' is a cross product PREHOOK: query: EXPLAIN SELECT TC.CONST_DATE, TB.PRODUCT_SK FROM TABLE_A TA @@ -103,100 +104,98 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Map 1 <- Map 3 (BROADCAST_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Map 1 <- Map 2 (BROADCAST_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: ta + alias: tb filterExpr: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) - Statistics: Num rows: 200000000 Data size: 12000000000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 100000000 Data size: 15400000000 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) - Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 50000000 Data size: 7700000000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: product_id (type: int) - outputColumnNames: _col1 - Statistics: Num rows: 100000000 Data size: 400000000 Basic stats: COMPLETE Column stats: COMPLETE + expressions: product_id (type: int), product_sk (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE Map Join Operator condition map: Inner Join 0 to 1 keys: - 0 DATE'2023-11-27' (type: date) - 1 DATE'2023-11-27' (type: date) - outputColumnNames: _col1, _col2 + 0 + 1 + outputColumnNames: _col0, _col1 input vertices: - 1 Map 3 - Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE + 1 Map 2 + Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: int), _col2 (type: date) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col1 (type: int), _col2 (type: date) - Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string) Execution mode: vectorized, llap LLAP IO: all inputs - Map 3 + Map 2 Map Operator Tree: TableScan alias: _dummy_table Row Limit Per Split: 1 Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: DATE'2023-11-27' (type: date) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: DATE'2023-11-27' (type: date) - null sort order: z - sort order: + - Map-reduce partition columns: DATE'2023-11-27' (type: date) - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: date) + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: no inputs - Map 4 + Map 3 Map Operator Tree: TableScan - alias: tb + alias: ta filterExpr: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) - Statistics: Num rows: 100000000 Data size: 15400000000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 200000000 Data size: 12000000000 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((start_date = DATE'2023-11-27') and product_id is not null) (type: boolean) - Statistics: Num rows: 50000000 Data size: 7700000000 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 100000000 Data size: 6000000000 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: product_id (type: int), product_sk (type: string) - outputColumnNames: _col1, _col2 - Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE + expressions: product_id (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 100000000 Data size: 400000000 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col1 (type: int), DATE'2023-11-27' (type: date) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col1 (type: int), DATE'2023-11-27' (type: date) - Statistics: Num rows: 50000000 Data size: 4900000000 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: string) + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 100000000 Data size: 400000000 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: all inputs - Reducer 2 - Execution mode: llap + Reducer 4 + Execution mode: vectorized, llap Reduce Operator Tree: - Merge Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: int), _col2 (type: date) - 1 _col1 (type: int), DATE'2023-11-27' (type: date) - outputColumnNames: _col2, _col5 - Statistics: Num rows: 16666666666 Data size: 2499999999900 Basic stats: COMPLETE Column stats: COMPLETE + 0 KEY.reducesinkkey0 (type: int) + 1 KEY.reducesinkkey0 (type: int) + outputColumnNames: _col1 + input vertices: + 0 Map 1 + Statistics: Num rows: 16666666666 Data size: 1566666666604 Basic stats: COMPLETE Column stats: COMPLETE + DynamicPartitionHashJoin: true Select Operator - expressions: _col2 (type: date), _col5 (type: string) + expressions: DATE'2023-11-27' (type: date), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 16666666666 Data size: 1566666666604 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 16666666666 Data size: 2499999999900 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 16666666666 Data size: 1566666666604 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 16666666666 Data size: 2499999999900 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat From ac2019c2adcd7ca5da8cadc885b80bdc94fc225b Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Thu, 21 Dec 2023 13:40:46 +0100 Subject: [PATCH 108/179] HIVE-27161: MetaException when executing CTAS query in Druid storage handler (Krisztian Kasa, reviewed by Denys Kuzmenko) --- .../iceberg/mr/hive/HiveIcebergStorageHandler.java | 6 ++++++ .../hadoop/hive/ql/metadata/HiveStorageHandler.java | 9 ++++++++- .../apache/hadoop/hive/ql/parse/SemanticAnalyzer.java | 3 ++- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 6e9e9870cfe8..985e7d48f617 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -73,6 +73,7 @@ import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc; import org.apache.hadoop.hive.ql.ddl.table.AlterTableType; +import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc; import org.apache.hadoop.hive.ql.ddl.table.create.like.CreateTableLikeDesc; import org.apache.hadoop.hive.ql.ddl.table.misc.properties.AlterTableSetPropertiesDesc; import org.apache.hadoop.hive.ql.exec.ColumnInfo; @@ -1647,6 +1648,11 @@ public void setTableParametersForCTLT(org.apache.hadoop.hive.ql.metadata.Table t } } + @Override + public void setTableLocationForCTAS(CreateTableDesc desc, String location) { + desc.setLocation(location); + } + @Override public Map getNativeProperties(org.apache.hadoop.hive.ql.metadata.Table table) { Table origTable = IcebergTableUtil.getTable(conf, table.getTTable()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 9fe61c0bdc32..e9a0d139e90a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -24,7 +24,6 @@ import java.net.URISyntaxException; import java.util.Collections; import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.common.classification.InterfaceAudience; import org.apache.hadoop.hive.common.classification.InterfaceStability; import org.apache.hadoop.hive.common.type.SnapshotContext; @@ -43,6 +42,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc; import org.apache.hadoop.hive.ql.ddl.table.AlterTableType; +import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc; import org.apache.hadoop.hive.ql.ddl.table.create.like.CreateTableLikeDesc; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.hooks.WriteEntity; @@ -378,6 +378,13 @@ default void setTableParametersForCTLT(org.apache.hadoop.hive.ql.metadata.Table Map origParams) { } + /** + * Sets tables physical location at create table as select. + * Some storage handlers requires specifying the location of tables others generates it internally. + */ + default void setTableLocationForCTAS(CreateTableDesc desc, String location) { + } + /** * Extract the native properties of the table which aren't stored in the HMS * @param table the table diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 2f3869319573..c1c04526bcfe 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -7949,7 +7949,8 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) } } else { if (tblDesc.isCTAS() && tblDesc.getStorageHandler() != null) { - tblDesc.setLocation(getCtasOrCMVLocation(tblDesc, viewDesc, createTableUseSuffix).toString()); + tblDesc.toTable(conf).getStorageHandler().setTableLocationForCTAS( + tblDesc, getCtasOrCMVLocation(tblDesc, viewDesc, false).toString()); } tableDescriptor = PlanUtils.getTableDesc(tblDesc, cols, colTypes); } From cdfada59809f5ee129b692bd9a63b4f5a7212a05 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Fri, 22 Dec 2023 01:08:19 +0530 Subject: [PATCH 109/179] HIVE-25803: URL Mapping appends default Fs scheme even for LOCAL DIRECTORY ops. (#4957). (Ayush Saxena, reviewed by Denys Kuzmenko) --- .../org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java | 3 ++- ql/src/test/results/clientpositive/llap/ppd_transform.q.out | 6 ++---- .../test/results/clientpositive/llap/schemeAuthority.q.out | 2 -- .../test/results/clientpositive/llap/schemeAuthority2.q.out | 2 -- 4 files changed, 4 insertions(+), 9 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index c1c04526bcfe..769ab25e43e9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -8052,8 +8052,9 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) loadFileDesc.setMoveTaskId(moveTaskId); loadFileWork.add(loadFileDesc); try { + FileSystem fs = isDfsDir ? destinationPath.getFileSystem(conf) : FileSystem.getLocal(conf); Path qualifiedPath = conf.getBoolVar(ConfVars.HIVE_RANGER_USE_FULLY_QUALIFIED_URL) ? - destinationPath.getFileSystem(conf).makeQualified(destinationPath) : destinationPath; + fs.makeQualified(destinationPath) : destinationPath; if (!outputs.add(new WriteEntity(qualifiedPath, !isDfsDir, isDestTempFile))) { throw new SemanticException(ErrorMsg.OUTPUT_SPECIFIED_MULTIPLE_TIMES .getMsg(destinationPath.toUri().toString())); diff --git a/ql/src/test/results/clientpositive/llap/ppd_transform.q.out b/ql/src/test/results/clientpositive/llap/ppd_transform.q.out index 9a0f487d0f66..1505011626c8 100644 --- a/ql/src/test/results/clientpositive/llap/ppd_transform.q.out +++ b/ql/src/test/results/clientpositive/llap/ppd_transform.q.out @@ -384,8 +384,7 @@ FROM ( PREHOOK: type: QUERY PREHOOK: Input: cat PREHOOK: Input: default@src -PREHOOK: Output: hdfs://### HDFS PATH ### -PREHOOK: Output: hdfs://### HDFS PATH ### +#### A masked pattern was here #### POSTHOOK: query: EXPLAIN FROM ( FROM ( SELECT * FROM src ) mapout REDUCE * USING 'cat' AS x,y @@ -394,8 +393,7 @@ FROM ( POSTHOOK: type: QUERY POSTHOOK: Input: cat POSTHOOK: Input: default@src -POSTHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: Output: hdfs://### HDFS PATH ### +#### A masked pattern was here #### STAGE DEPENDENCIES: Stage-2 is a root stage Stage-3 depends on stages: Stage-2 diff --git a/ql/src/test/results/clientpositive/llap/schemeAuthority.q.out b/ql/src/test/results/clientpositive/llap/schemeAuthority.q.out index 009424ba8d23..47d458fefb7c 100644 --- a/ql/src/test/results/clientpositive/llap/schemeAuthority.q.out +++ b/ql/src/test/results/clientpositive/llap/schemeAuthority.q.out @@ -9,11 +9,9 @@ POSTHOOK: Output: default@dynPart #### A masked pattern was here #### PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: hdfs://### HDFS PATH ### #### A masked pattern was here #### POSTHOOK: type: QUERY POSTHOOK: Input: default@src -POSTHOOK: Output: hdfs://### HDFS PATH ### #### A masked pattern was here #### PREHOOK: type: QUERY PREHOOK: Input: default@src diff --git a/ql/src/test/results/clientpositive/llap/schemeAuthority2.q.out b/ql/src/test/results/clientpositive/llap/schemeAuthority2.q.out index d3731f70e670..9009cd296714 100644 --- a/ql/src/test/results/clientpositive/llap/schemeAuthority2.q.out +++ b/ql/src/test/results/clientpositive/llap/schemeAuthority2.q.out @@ -9,11 +9,9 @@ POSTHOOK: Output: default@dynPart_n0 #### A masked pattern was here #### PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: hdfs://### HDFS PATH ### #### A masked pattern was here #### POSTHOOK: type: QUERY POSTHOOK: Input: default@src -POSTHOOK: Output: hdfs://### HDFS PATH ### #### A masked pattern was here #### PREHOOK: type: QUERY PREHOOK: Input: default@src From 5022b85b5f50615f85da07bce42aebd414deb9b0 Mon Sep 17 00:00:00 2001 From: tarak271 Date: Fri, 22 Dec 2023 15:57:25 +0530 Subject: [PATCH 110/179] HIVE-27797: Addendum: Fix flaky test case (Taraka Rama Rao Lethavadla, reviewed by Denys Kuzmenko) Closes #4933 --- itests/hive-unit/pom.xml | 7 +++ .../TestTimedOutTxnNotificationLogging.java | 53 ++++++++++--------- 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index 7c3b2333b9eb..b89c3f05dd46 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -27,6 +27,7 @@ ../.. 1.15.2 2.70.0 + 5.6.2 @@ -494,6 +495,12 @@ postgresql test + + org.junit.jupiter + junit-jupiter + ${junit.jupiter.version} + test + diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java index acad300c6817..888b8ee0f40b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.MetastoreTaskThread; +import org.apache.hadoop.hive.metastore.ObjectStore; import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage; @@ -38,11 +39,8 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.events.EventUtils; import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.util.StringUtils; import org.apache.hive.hcatalog.listener.DbNotificationListener; import org.apache.thrift.TException; import org.junit.After; @@ -51,18 +49,23 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.IOException; import java.util.*; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import static org.junit.jupiter.api.Assertions.assertEquals; -import static junit.framework.Assert.assertEquals; @RunWith(Parameterized.class) public class TestTimedOutTxnNotificationLogging { private HiveConf hiveConf; + private ObjectStore objectStore; + + private MetastoreTaskThread acidTxnCleanerService; + + private MetastoreTaskThread acidHouseKeeperService; + private static IMetaStoreClient hive; @Parameterized.Parameter @@ -86,14 +89,19 @@ public void setUp() throws Exception { TestTxnDbUtil.prepDb(hiveConf); SessionState.start(new CliSessionState(hiveConf)); hive = new HiveMetaStoreClient(hiveConf); + objectStore = new ObjectStore(); + objectStore.setConf(hiveConf); + acidTxnCleanerService = new AcidTxnCleanerService(); + acidTxnCleanerService.setConf(hiveConf); + acidHouseKeeperService = new AcidHouseKeeperService(); + acidHouseKeeperService.setConf(hiveConf); } private void setConf() { hiveConf = new HiveConf(); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.HIVE_IN_TEST, true); MetastoreConf.setVar(hiveConf, MetastoreConf.ConfVars.WAREHOUSE, "/tmp"); - MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.TXN_TIMEOUT, 1000, TimeUnit.MILLISECONDS); - MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.EVENT_DB_LISTENER_TTL, 1, TimeUnit.SECONDS); + MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.TXN_TIMEOUT, 1, TimeUnit.SECONDS); HiveConf.setVar(hiveConf, HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, SQLStdHiveAuthorizerFactory.class.getName()); MetastoreConf.setVar(hiveConf, MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS, @@ -118,20 +126,20 @@ public void tearDown() throws Exception { public void testTxnNotificationLogging() throws Exception { try { List txnIds = openTxns(numberOfTxns, txnType); - assertEquals(txnIds.size(), getNumberOfTxns(txnIds, TxnState.OPEN)); - assertEquals(expectedNotifications, getNumNotifications(txnIds, MessageBuilder.OPEN_TXN_EVENT)); + assertEquals(txnIds.size(), getNumberOfTxnsWithTxnState(txnIds, TxnState.OPEN)); + assertEquals(expectedNotifications, getNumberOfNotificationsWithEventType(txnIds, MessageBuilder.OPEN_TXN_EVENT)); Thread.sleep(1000); - runHouseKeeperService(); + acidHouseKeeperService.run(); //this will abort timed-out txns if (txnType != TxnType.REPL_CREATED) { - assertEquals(txnIds.size(), getNumberOfTxns(txnIds, TxnState.ABORTED)); - assertEquals(expectedNotifications, getNumNotifications(txnIds, MessageBuilder.ABORT_TXN_EVENT)); + assertEquals(txnIds.size(), getNumberOfTxnsWithTxnState(txnIds, TxnState.ABORTED)); + assertEquals(expectedNotifications, getNumberOfNotificationsWithEventType(txnIds, MessageBuilder.ABORT_TXN_EVENT)); } } finally { - runTxnHouseKeeperService(); + runCleanerServices(); } } - private int getNumNotifications(List txnIds, String eventType) throws IOException, TException { + private int getNumberOfNotificationsWithEventType(List txnIds, String eventType) throws TException { int numNotifications = 0; IMetaStoreClient.NotificationFilter evFilter = new AndFilter(new ReplEventFilter(new ReplScope()), new CatalogFilter(MetaStoreUtils.getDefaultCatalog(hiveConf)), new EventBoundaryFilter(0, 100)); @@ -140,7 +148,7 @@ private int getNumNotifications(List txnIds, String eventType) throws IOEx return numNotifications; } Iterator eventIterator = rsp.getEvents().iterator(); - MessageDeserializer deserializer = null; + MessageDeserializer deserializer; while (eventIterator.hasNext()) { NotificationEvent ev = eventIterator.next(); if (eventType.equals(ev.getEventType())) { @@ -168,7 +176,7 @@ private List openTxns(int txnCounter, TxnType txnType) throws TException { for (; txnCounter > 0; txnCounter--) { if (txnType == TxnType.REPL_CREATED) { Long srcTxn = (long) (11 + txnCounter); - List srcTxns = Arrays.asList(new Long[] { srcTxn }); + List srcTxns = Collections.singletonList(srcTxn); txnIds.addAll(hive.replOpenTxn("testPolicy", srcTxns, "hive", txnType)); } else { txnIds.add(hive.openTxn("hive", txnType)); @@ -177,7 +185,7 @@ private List openTxns(int txnCounter, TxnType txnType) throws TException { return txnIds; } - private int getNumberOfTxns(List txnIds, TxnState txnState) throws TException { + private int getNumberOfTxnsWithTxnState(List txnIds, TxnState txnState) throws TException { AtomicInteger numTxns = new AtomicInteger(); hive.showTxns().getOpen_txns().forEach(txnInfo -> { if (txnInfo.getState() == txnState && txnIds.contains(txnInfo.getId())) { @@ -187,15 +195,8 @@ private int getNumberOfTxns(List txnIds, TxnState txnState) throws TExcept return numTxns.get(); } - private void runHouseKeeperService() { - MetastoreTaskThread acidHouseKeeperService = new AcidHouseKeeperService(); - acidHouseKeeperService.setConf(hiveConf); - acidHouseKeeperService.run(); //this will abort timedout txns - } - - private void runTxnHouseKeeperService() { - MetastoreTaskThread acidTxnCleanerService = new AcidTxnCleanerService(); - acidTxnCleanerService.setConf(hiveConf); + private void runCleanerServices() { + objectStore.cleanNotificationEvents(0); acidTxnCleanerService.run(); //this will remove empty aborted txns } } \ No newline at end of file From 2d6e4a7ef4d9ed6ce997520e5fe3b6841ab5ee31 Mon Sep 17 00:00:00 2001 From: Vikram Ahuja Date: Fri, 22 Dec 2023 21:25:09 +0530 Subject: [PATCH 111/179] HIVE-27804: Implement batching in getPartition calls which returns partition list along with auth info (Vikram Ahuja, Reviewed by Chinna Rao Lalam) --- .../apache/hadoop/hive/ql/metadata/Hive.java | 172 +- .../hive/ql/metadata/PartitionIterable.java | 25 +- .../hadoop/hive/ql/exec/PartitionUtil.java | 28 + .../exec/TestGetPartitionAuthWithBatches.java | 300 ++ .../ql/exec/TestGetPartitionInBatches.java | 53 +- .../thrift/gen-cpp/ThriftHiveMetastore.cpp | 2560 +++++++-------- .../thrift/gen-cpp/hive_metastore_types.cpp | 652 ++-- .../gen/thrift/gen-cpp/hive_metastore_types.h | 10 +- .../metastore/api/GetOpenTxnsRequest.java | 36 +- .../api/GetPartitionsPsWithAuthRequest.java | 204 +- .../api/GetPartitionsPsWithAuthResponse.java | 36 +- .../metastore/api/ReplicationMetricList.java | 36 +- .../metastore/api/ThriftHiveMetastore.java | 2780 ++++++++--------- .../gen-php/metastore/GetOpenTxnsRequest.php | 18 +- .../GetPartitionsPsWithAuthRequest.php | 52 +- .../GetPartitionsPsWithAuthResponse.php | 20 +- .../metastore/ReplicationMetricList.php | 20 +- ...hriftHiveMetastore_add_partitions_args.php | 20 +- ...iveMetastore_add_partitions_pspec_args.php | 20 +- ...tore_add_write_ids_to_min_history_args.php | 26 +- ...iftHiveMetastore_alter_partitions_args.php | 20 +- ...rtitions_with_environment_context_args.php | 20 +- ...iftHiveMetastore_append_partition_args.php | 18 +- ...artition_with_environment_context_args.php | 18 +- ...ore_create_table_with_constraints_args.php | 120 +- ...hriftHiveMetastore_drop_partition_args.php | 18 +- ...artition_with_environment_context_args.php | 18 +- ...tHiveMetastore_exchange_partition_args.php | 26 +- ...HiveMetastore_exchange_partitions_args.php | 26 +- ...veMetastore_exchange_partitions_result.php | 20 +- ...tastore_find_columns_with_stats_result.php | 18 +- ...HiveMetastore_get_all_databases_result.php | 18 +- ...ized_view_objects_for_rewriting_result.php | 20 +- ...tHiveMetastore_get_all_packages_result.php | 18 +- ...store_get_all_stored_procedures_result.php | 18 +- ...iftHiveMetastore_get_all_tables_result.php | 18 +- ...store_get_all_token_identifiers_result.php | 18 +- ...astore_get_all_write_event_info_result.php | 20 +- ...riftHiveMetastore_get_databases_result.php | 18 +- ...iveMetastore_get_dataconnectors_result.php | 18 +- .../ThriftHiveMetastore_get_fields_result.php | 20 +- ...fields_with_environment_context_result.php | 20 +- ...riftHiveMetastore_get_functions_result.php | 18 +- ...ftHiveMetastore_get_master_keys_result.php | 18 +- ...aterialized_views_for_rewriting_result.php | 18 +- ...astore_get_part_specs_by_filter_result.php | 20 +- ...ThriftHiveMetastore_get_partition_args.php | 18 +- ...eMetastore_get_partition_names_ps_args.php | 18 +- ...etastore_get_partition_names_ps_result.php | 18 +- ...tastore_get_partition_names_req_result.php | 18 +- ...veMetastore_get_partition_names_result.php | 18 +- ...Metastore_get_partition_with_auth_args.php | 36 +- ...re_get_partitions_by_filter_req_result.php | 20 +- ...astore_get_partitions_by_filter_result.php | 20 +- ...Metastore_get_partitions_by_names_args.php | 18 +- ...tastore_get_partitions_by_names_result.php | 20 +- ...ftHiveMetastore_get_partitions_ps_args.php | 18 +- ...HiveMetastore_get_partitions_ps_result.php | 20 +- ...store_get_partitions_ps_with_auth_args.php | 36 +- ...ore_get_partitions_ps_with_auth_result.php | 20 +- ...eMetastore_get_partitions_pspec_result.php | 20 +- ...iftHiveMetastore_get_partitions_result.php | 20 +- ...etastore_get_partitions_with_auth_args.php | 18 +- ...astore_get_partitions_with_auth_result.php | 20 +- ...ftHiveMetastore_get_privilege_set_args.php | 18 +- ...iftHiveMetastore_get_role_names_result.php | 18 +- ...HiveMetastore_get_runtime_stats_result.php | 20 +- ...tastore_get_schema_all_versions_result.php | 20 +- .../ThriftHiveMetastore_get_schema_result.php | 20 +- ...schema_with_environment_context_result.php | 20 +- ...hriftHiveMetastore_get_table_meta_args.php | 18 +- ...iftHiveMetastore_get_table_meta_result.php | 20 +- ...store_get_table_names_by_filter_result.php | 18 +- ...tastore_get_table_objects_by_name_args.php | 18 +- ...store_get_table_objects_by_name_result.php | 20 +- ...iveMetastore_get_tables_by_type_result.php | 18 +- ...iftHiveMetastore_get_tables_ext_result.php | 20 +- .../ThriftHiveMetastore_get_tables_result.php | 18 +- ...hriftHiveMetastore_get_type_all_result.php | 28 +- ...tastore_isPartitionMarkedForEvent_args.php | 26 +- ...ftHiveMetastore_list_privileges_result.php | 20 +- .../ThriftHiveMetastore_list_roles_result.php | 20 +- ...veMetastore_markPartitionForEvent_args.php | 26 +- ...rtition_name_has_valid_characters_args.php | 18 +- ...etastore_partition_name_to_spec_result.php | 26 +- ...etastore_partition_name_to_vals_result.php | 18 +- ...iftHiveMetastore_rename_partition_args.php | 18 +- .../ThriftHiveMetastore_set_ugi_args.php | 18 +- .../ThriftHiveMetastore_set_ugi_result.php | 18 +- ...hriftHiveMetastore_truncate_table_args.php | 18 +- .../hive_metastore/ThriftHiveMetastore.py | 1056 +++---- .../thrift/gen-py/hive_metastore/ttypes.py | 70 +- .../gen/thrift/gen-rb/hive_metastore_types.rb | 4 +- .../src/main/thrift/hive_metastore.thrift | 3 +- .../hadoop/hive/metastore/HMSHandler.java | 1 + .../hadoop/hive/metastore/ObjectStore.java | 21 +- 96 files changed, 5187 insertions(+), 4534 deletions(-) create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index cca06126369b..192fa13ffa1f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -4104,49 +4104,20 @@ public List getPartitionNames(Table tbl, ExprNodeGenericFuncDesc expr, S } /** - * get all the partitions that the table has + * get all the partitions that the table has along with auth info * * @param tbl * object for which partition is needed - * @return list of partition objects + * @return list of partition objects along with auth info */ public List getPartitions(Table tbl) throws HiveException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS); - try { - if (tbl.isPartitioned()) { - List tParts; - try { - GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest(); - req.setTblName(tbl.getTableName()); - req.setDbName(tbl.getDbName()); - req.setUserName(getUserName()); - req.setMaxParts((short) -1); - req.setGroupNames(getGroupNames()); - if (AcidUtils.isTransactionalTable(tbl)) { - ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); - req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); - req.setId(tbl.getTTable().getId()); - } - GetPartitionsPsWithAuthResponse res = getMSC().listPartitionsWithAuthInfoRequest(req); - tParts = res.getPartitions(); - - } catch (NoSuchObjectException nsoe) { - return Lists.newArrayList(); - } catch (Exception e) { - LOG.error("Failed getPartitions", e); - throw new HiveException(e); - } - List parts = new ArrayList<>(tParts.size()); - for (org.apache.hadoop.hive.metastore.api.Partition tpart : tParts) { - parts.add(new Partition(tbl, tpart)); - } - - return parts; - } else { - return Collections.singletonList(new Partition(tbl)); - } + int batchSize= MetastoreConf.getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX); + return new ArrayList<>(getAllPartitionsInBatches(tbl, batchSize, DEFAULT_BATCH_DECAYING_FACTOR, MetastoreConf + .getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES), + null, true, getUserName(), getGroupNames())); } finally { perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS, "HS2-cache"); } @@ -4186,14 +4157,33 @@ public Set getAllPartitionsOf(Table tbl) throws HiveException { Hive.get().getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX); if (batchSize > 0) { return getAllPartitionsInBatches(tbl, batchSize, DEFAULT_BATCH_DECAYING_FACTOR, MetastoreConf.getIntVar( - Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES)); + Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES), null, false); } else { return getAllPartitions(tbl); } } public Set getAllPartitionsInBatches(Table tbl, int batchSize, int decayingFactor, - int maxRetries) throws HiveException { + int maxRetries, Map partialPartitionSpec, boolean isAuthRequired) throws HiveException { + return getAllPartitionsInBatches(tbl, batchSize, decayingFactor, maxRetries, partialPartitionSpec, isAuthRequired, + null, null); + } + + /** + * Main method which fetches the partitions in batches + * @param tbl table for which partitions are needed + * @param batchSize Number of partitions to be fectehd in one batched call + * @param decayingFactor the value by which batchSize decays in the next retry in case it faces an exception + * @param maxRetries Number of retries allowed for this operation + * @param partialPartitionSpec partialPartitionSpec for the table + * @param isAuthRequired If auth information is required along with partitions + * @param userName name of the calling user + * @param groupNames groups the call + * @return list of partition objects + */ + public Set getAllPartitionsInBatches(Table tbl, int batchSize, int decayingFactor, + int maxRetries, Map partialPartitionSpec, boolean isAuthRequired, + String userName, List groupNames) throws HiveException { if (!tbl.isPartitioned()) { return Sets.newHashSet(new Partition(tbl)); } @@ -4202,13 +4192,11 @@ public Set getAllPartitionsInBatches(Table tbl, int batchSize, int de .ExponentiallyDecayingBatchWork(batchSize, decayingFactor, maxRetries) { @Override public Void execute(int size) throws HiveException { - try { - result.clear(); - new PartitionIterable(Hive.get(), tbl, null, size).forEach(result::add); - return null; - } catch (HiveException e) { - throw e; - } + result.clear(); + PartitionIterable partitionIterable = new PartitionIterable(Hive.get(), tbl, partialPartitionSpec, size, + isAuthRequired, userName, groupNames); + partitionIterable.forEach(result::add); + return null; } }; try { @@ -4219,6 +4207,25 @@ public Void execute(int size) throws HiveException { return result; } + public List getPartitions(Table tbl, Map partialPartSpec, + short limit) throws HiveException { + PerfLogger perfLogger = SessionState.getPerfLogger(); + perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2); + try { + // TODO: Implement Batching when limit is >=0 + if (limit >= 0) { + return getPartitionsWithAuth(tbl, partialPartSpec, limit); + } else { + int batchSize = MetastoreConf.getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX); + return new ArrayList<>(getAllPartitionsInBatches(tbl, batchSize, DEFAULT_BATCH_DECAYING_FACTOR, + MetastoreConf.getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES), + partialPartSpec, true, getUserName(), getGroupNames())); + } + } finally { + perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2, "HS2-cache"); + } + } + /** * get all the partitions of the table that matches the given partial * specification. partition columns whose value is can be anything should be @@ -4230,36 +4237,30 @@ public Void execute(int size) throws HiveException { * @return list of partition objects * @throws HiveException */ - public List getPartitions(Table tbl, Map partialPartSpec, - short limit) - throws HiveException { - PerfLogger perfLogger = SessionState.getPerfLogger(); - perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2); - try { - if (!tbl.isPartitioned()) { - throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName()); - } + private List getPartitionsWithAuth(Table tbl, Map partialPartSpec, + short limit) + throws HiveException { + if (!tbl.isPartitioned()) { + throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName()); + } - List partialPvals = MetaStoreUtils.getPvals(tbl.getPartCols(), partialPartSpec); + List partialPvals = MetaStoreUtils.getPvals(tbl.getPartCols(), partialPartSpec); - List partitions = null; - try { - String userName = getUserName(); - partitions = getMSC().listPartitionsWithAuthInfo(tbl.getDbName(), tbl.getTableName(), - partialPvals, limit, userName, getGroupNames()); - } catch (Exception e) { - throw new HiveException(e); - } - - List qlPartitions = new ArrayList(); - for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) { - qlPartitions.add(new Partition(tbl, p)); - } + List partitions = null; + try { + String userName = getUserName(); + partitions = getMSC().listPartitionsWithAuthInfo(tbl.getDbName(), tbl.getTableName(), + partialPvals, limit, userName, getGroupNames()); + } catch (Exception e) { + throw new HiveException(e); + } - return qlPartitions; - } finally { - perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2, "HS2-cache"); + List qlPartitions = new ArrayList(); + for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) { + qlPartitions.add(new Partition(tbl, p)); } + + return qlPartitions; } /** @@ -4385,6 +4386,37 @@ public List getPartitionsByNames(Table tbl, List partNames, b return partitions; } + public List getPartitionsAuthByNames(Table tbl, List partNames, String userName, + List groupNames) throws HiveException { + if (!tbl.isPartitioned()) { + throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName()); + } + GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest(); + req.setTblName(tbl.getTableName()); + req.setDbName(tbl.getDbName()); + req.setUserName(userName); + req.setGroupNames(groupNames); + req.setPartNames(partNames); + if (AcidUtils.isTransactionalTable(tbl)) { + ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); + req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); + req.setId(tbl.getTTable().getId()); + } + + List tParts; + try { + GetPartitionsPsWithAuthResponse res = getMSC().listPartitionsWithAuthInfoRequest(req); + tParts = res.getPartitions(); + } catch (Exception e) { + throw new HiveException(e); + } + List parts = new ArrayList<>(tParts.size()); + for (org.apache.hadoop.hive.metastore.api.Partition tpart : tParts) { + parts.add(new Partition(tbl, tpart)); + } + return parts; + } + /** * Get a list of Partitions by filter. * @param tbl The table containing the partitions. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java index 158f31bf7f41..dabfef014ce7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java @@ -99,7 +99,11 @@ private void getNextBatch() { batchCounter++; } try { - batchIter = db.getPartitionsByNames(table, nameBatch, getColStats).iterator(); + if (isAuthRequired) { + batchIter = db.getPartitionsAuthByNames(table, nameBatch, userName, groupNames).iterator(); + } else { + batchIter = db.getPartitionsByNames(table, nameBatch, getColStats).iterator(); + } } catch (HiveException e) { throw new RuntimeException(e); } @@ -130,6 +134,9 @@ enum Type { private List partitionNames = null; private int batchSize; private boolean getColStats = false; + private boolean isAuthRequired = false; + private String userName; + private List groupNames; /** * Dummy constructor, which simply acts as an iterator on an already-present @@ -150,12 +157,25 @@ public PartitionIterable(Hive db, Table table, Map partialPartit this(db, table, partialPartitionSpec, batchSize, false); } + public PartitionIterable(Hive db, Table table, Map partialPartitionSpec, + int batchSize, boolean isAuthRequired, String userName, + List groupNames) throws HiveException { + this(db, table, partialPartitionSpec, batchSize, false, isAuthRequired, userName, groupNames); + } + + /** * Primary constructor that fetches all partitions in a given table, given * a Hive object and a table object, and a partial partition spec. */ public PartitionIterable(Hive db, Table table, Map partialPartitionSpec, int batchSize, boolean getColStats) throws HiveException { + this(db, table, partialPartitionSpec, batchSize, getColStats, false, null, null); + } + + private PartitionIterable(Hive db, Table table, Map partialPartitionSpec, + int batchSize, boolean getColStats, boolean isAuthRequired, String userName, + List groupNames) throws HiveException { if (batchSize < 1) { throw new HiveException("Invalid batch size for partition iterable. Please use a batch size greater than 0"); } @@ -165,6 +185,9 @@ public PartitionIterable(Hive db, Table table, Map partialPartit this.partialPartitionSpec = partialPartitionSpec; this.batchSize = batchSize; this.getColStats = getColStats; + this.isAuthRequired = isAuthRequired; + this.userName = userName; + this.groupNames = groupNames; if (this.partialPartitionSpec == null){ partitionNames = db.getPartitionNames( diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java index f449c79760fd..59179ebb51c0 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java @@ -18,19 +18,25 @@ package org.apache.hadoop.hive.ql.exec; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.CheckResult; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.stats.StatsUtils; import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.util.StringUtils; import java.util.Arrays; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Set; import static org.junit.Assert.fail; @@ -82,4 +88,26 @@ public static Set createPartsNotInMs(int numOfParts } return partsNotInMs; } + + + public static void addPartitions(IMetaStoreClient db, String dbName, String tableName, String location, + HiveConf hiveConf, int numPartitions) throws Exception { + List partitions = new ArrayList<>(); + for (int i = 0; i < numPartitions; i++) { + partitions.add(buildPartition(dbName, tableName, String.valueOf(i), location + "/city=" + i, hiveConf)); + } + db.add_partitions(partitions, true, true); + } + + protected static Partition buildPartition(String dbName, String tableName, String value, + String location, HiveConf hiveConf) throws MetaException { + return new PartitionBuilder() + .setDbName(dbName) + .setTableName(tableName) + .addValue(value) + .addCol("test_id", "int", "test col id") + .addCol("test_value", "string", "test col value") + .setLocation(location) + .build(hiveConf); + } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java new file mode 100644 index 000000000000..191d211d4b72 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; + +import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthRequest; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.junit.Assert; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class TestGetPartitionAuthWithBatches { + + private final String catName = "hive"; + private final String dbName = "default"; + private final String tableName = "test_partition_batch_with_auth"; + private static HiveConf hiveConf; + private static HiveMetaStoreClient msc; + private static Hive hive; + private Table table; + private final static int NUM_PARTITIONS = 30; + private final static int DECAYING_FACTOR = 2; + private final static int MAX_RETRIES = 0; + private final static boolean IS_AUTH_REQUIRED = true; + private final static String USER_NAME = "username"; + private final static List GROUP_NAMES = Arrays.asList("Grp1", "Grp2"); + private final static Map PARTIAL_PARTITION_SPEC = null; + + @BeforeClass + public static void setupClass() throws HiveException { + hiveConf = new HiveConf(TestGetPartitionAuthWithBatches.class); + hiveConf.set("hive.security.authorization.enabled", "true"); + hiveConf.set("hive.security.authorization.manager","org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"); + hive = Hive.get(); + SessionState.start(hiveConf); + try { + msc = new HiveMetaStoreClient(hiveConf); + } catch (MetaException e) { + throw new HiveException(e); + } + } + + @Before + public void before() throws Exception { + PartitionUtil.createPartitionedTable(msc, catName, dbName, tableName); + table = msc.getTable(catName, dbName, tableName); + PartitionUtil.addPartitions(msc, dbName, tableName, table.getSd().getLocation(), hiveConf, NUM_PARTITIONS); + } + + @After + public void after() throws Exception { + PartitionUtil.cleanUpTableQuietly(msc, catName, dbName, tableName); + } + + @Test + public void testNumberOfPartitionsRetrieved() throws HiveException { + List numParts = hive.getPartitionNames(dbName, tableName, (short)-1); + Assert.assertEquals(numParts.size(), NUM_PARTITIONS); + List partitions = hive.getPartitionsAuthByNames(new org.apache.hadoop.hive.ql.metadata.Table(table), + numParts.subList(0,5), USER_NAME, GROUP_NAMES); + Assert.assertEquals(partitions.size(), 5); + } + + /** + * Tests the number of partitions recieved from the HMS + * + * @throws Exception + */ + @Test + public void testGetPartitionsAPI() throws Exception { + List part = hive.getPartitions(hive.getTable(dbName, tableName)); + Assert.assertEquals(part.size(), NUM_PARTITIONS); + } + + @Test + public void testGetPartitionsAPI2() throws Exception { + List part = hive.getPartitions(hive.getTable(dbName, tableName), + new HashMap() , (short) -1); + Assert.assertEquals(part.size(), NUM_PARTITIONS); + } + + @Test + public void testGetPartitionsAPI2limit() throws Exception { + List part = hive.getPartitions(hive.getTable(dbName, tableName), + new HashMap() , (short) 1); + Assert.assertEquals(part.size(), 1); + + List part1 = hive.getPartitions(hive.getTable(dbName, tableName), + new HashMap() , (short) 10); + Assert.assertEquals(part1.size(), 10); + } + + /** + * Tests the number of times Hive.getPartitions calls are executed with total number of + * partitions to be added are equally divisible by batch size + * + * @throws Exception + */ + @Test + public void testNumberOfGetPartitionCalls() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + // test with a batch size of 10 and decaying factor of 2 + int batchSize = 10; + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), batchSize, DECAYING_FACTOR, MAX_RETRIES, + PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // there should be 3 calls to get partitions + verify(spyMSC, times(3)).listPartitionsWithAuthInfoRequest(req.capture()); + req.getAllValues().forEach(part-> Assert.assertEquals(part.getPartNames().size(),10)); + } + + @Test + public void testNumberOfGetPartitionCalls2() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + // test with a batch size of 10 and decaying factor of 2 + int batchSize = 10; + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), batchSize, DECAYING_FACTOR, MAX_RETRIES, + new HashMap(), IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // there should be 3 calls to get partitions + verify(spyMSC, times(3)).listPartitionsWithAuthInfoRequest(req.capture()); + req.getAllValues().forEach(part-> Assert.assertEquals(part.getPartNames().size(), 10)); + } + + /** + * Tests the number of times Hive.getAllPartitionsOf calls are executed with total number of + * partitions to be added are not exactly divisible by batch size + * + * @throws Exception + */ + @Test + public void testUnevenNumberOfGetPartitionCalls() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + // there should be 2 calls to get partitions with batch sizes of 19, 11 + int batchSize = 19; + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),batchSize, DECAYING_FACTOR, MAX_RETRIES, + PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // there should be 2 calls to get partitions + verify(spyMSC, times(2)).listPartitionsWithAuthInfoRequest(req.capture()); + // confirm the batch sizes were 19, 11 in the two calls to get partitions + List apds = req.getAllValues(); + Assert.assertEquals(19, apds.get(0).getPartNames().size()); + Assert.assertEquals(11, apds.get(1).getPartNames().size()); + } + + /** + * Tests the number of times Hive.getAllPartitionsOf calls are executed with total number of + * partitions to is less than batch size + * + * @throws Exception + */ + @Test + public void testSmallNumberOfPartitions() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + int batchSize = 100; + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),batchSize, DECAYING_FACTOR, MAX_RETRIES, + PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // there should be 1 call to get partitions + verify(spyMSC, times(1)).listPartitionsWithAuthInfoRequest(req.capture()); + Assert.assertEquals(NUM_PARTITIONS, req.getValue().getPartNames().size()); + } + + /** + * Tests the retries exhausted case when getAllPartitionsOf method call always keep throwing + * HiveException. The batch sizes should exponentially decreased based on the decaying factor and + * ultimately give up when it reaches 0 + * + * @throws Exception + */ + @Test + public void testRetriesExhaustedBatchSize() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + doThrow(MetaException.class).when(spyMSC).listPartitionsWithAuthInfoRequest(any()); + try { + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), NUM_PARTITIONS, DECAYING_FACTOR, MAX_RETRIES, + PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + } catch (HiveException ignored) {} + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // there should be 5 call to get partitions with batch sizes as 30, 15, 7, 3, 1 + verify(spyMSC, times(5)).listPartitionsWithAuthInfoRequest(req.capture()); + List apds = req.getAllValues(); + Assert.assertEquals(5, apds.size()); + + Assert.assertEquals(30, apds.get(0).getPartNamesSize()); + Assert.assertEquals(15, apds.get(1).getPartNamesSize()); + Assert.assertEquals(7, apds.get(2).getPartNamesSize()); + Assert.assertEquals(3, apds.get(3).getPartNamesSize()); + Assert.assertEquals(1, apds.get(4).getPartNamesSize()); + } + + /** + * Tests the maximum retry attempts provided by configuration + * @throws Exception + */ + @Test + public void testMaxRetriesReached() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + doThrow(MetaException.class).when(spyMSC).listPartitionsWithAuthInfoRequest(any()); + int maxRetries = 2; + try { + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), NUM_PARTITIONS, DECAYING_FACTOR, maxRetries, + PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + } catch (HiveException ignored) {} + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // there should be 2 call to get partitions with batch sizes as 30, 15 + verify(spyMSC, times(2)).listPartitionsWithAuthInfoRequest(req.capture()); + List apds = req.getAllValues(); + Assert.assertEquals(2, apds.size()); + + Assert.assertEquals(30, apds.get(0).getPartNamesSize()); + Assert.assertEquals(15, apds.get(1).getPartNamesSize()); + } + + /** + * Tests the number of calls to getPartitions and the respective batch sizes when first call to + * getPartitions throws HiveException. The batch size should be reduced by the decayingFactor + * and the second call should fetch all the results + * + * @throws Exception + */ + @Test + public void testBatchingWhenException() throws Exception { + HiveMetaStoreClient spyMSC = spy(msc); + hive.setMSC(spyMSC); + // This will throw exception only the first time. + doThrow(new MetaException()).doCallRealMethod() + .when(spyMSC).listPartitionsWithAuthInfoRequest(any()); + + int maxRetries = 5; + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), NUM_PARTITIONS, DECAYING_FACTOR, maxRetries, + PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES); + ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class); + // The first call with batch size of 30 will fail, the rest two call will be of size 15 each. Total 3 calls + verify(spyMSC, times(3)).listPartitionsWithAuthInfoRequest(req.capture()); + List apds = req.getAllValues(); + Assert.assertEquals(3, apds.size()); + + Assert.assertEquals(30, apds.get(0).getPartNamesSize()); + Assert.assertEquals(15, apds.get(1).getPartNamesSize()); + Assert.assertEquals(15, apds.get(2).getPartNamesSize()); + + Set partNames = new HashSet<>(apds.get(1).getPartNames()); + partNames.addAll(apds.get(2).getPartNames()); + assert(partNames.size() == NUM_PARTITIONS); + + List partitionNames = hive.getPartitionNames(table.getDbName(),table.getTableName(), (short) -1); + assert(partitionNames.size() == NUM_PARTITIONS); + partitionNames.forEach(partNames::remove); + assert(partitionNames.size() == NUM_PARTITIONS); + // In case any duplicate/incomplete list is given by hive.getAllPartitionsInBatches, the below assertion will fail + assert(partNames.size() == 0); + } +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java index f11eee0500d3..d4afff716bf4 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java @@ -24,12 +24,11 @@ import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.MetastoreException; -import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.PartitionIterable; +import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Assert; import org.junit.After; @@ -38,10 +37,10 @@ import org.junit.Test; import org.mockito.ArgumentCaptor; - -import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import static org.mockito.ArgumentMatchers.any; @@ -59,6 +58,9 @@ public class TestGetPartitionInBatches { private static HiveMetaStoreClient msc; private static Hive hive; private Table table; + private final static int NUM_PARTITIONS = 30; + private final static boolean IS_AUTH_REQUIRED = false; + private final static Map PARTIAL_PARTITION_SPEC = null; @BeforeClass public static void setupClass() throws HiveException { @@ -76,7 +78,7 @@ public static void setupClass() throws HiveException { public void before() throws Exception { PartitionUtil.createPartitionedTable(msc, catName, dbName, tableName); table = msc.getTable(catName, dbName, tableName); - addPartitions(dbName, tableName); + PartitionUtil.addPartitions(msc, dbName, tableName, table.getSd().getLocation(), hiveConf, NUM_PARTITIONS); } @After @@ -84,24 +86,13 @@ public void after() throws Exception { PartitionUtil.cleanUpTableQuietly(msc, catName, dbName, tableName); } - private void addPartitions(String dbName, String tableName) throws Exception { - List partitions = new ArrayList<>(); - for (int i = 0; i < 30; i++) { - partitions.add(buildPartition(dbName, tableName, String.valueOf(i), table.getSd().getLocation() + "/city=" + i)); - } - msc.add_partitions(partitions, true, true); - } - - protected Partition buildPartition(String dbName, String tableName, String value, - String location) throws MetaException { - return new PartitionBuilder() - .setDbName(dbName) - .setTableName(tableName) - .addValue(value) - .addCol("test_id", "int", "test col id") - .addCol("test_value", "string", "test col value") - .setLocation(location) - .build(hiveConf); + @Test + public void TestNumberOfPartitionsRetrieved() throws HiveException { + List numParts = hive.getPartitionNames(dbName, tableName, (short)-1); + Assert.assertEquals(numParts.size(), NUM_PARTITIONS); + List partitions = hive.getPartitionsByNames(new org.apache.hadoop.hive.ql.metadata.Table(table), + numParts.subList(0,5), false); + Assert.assertEquals(partitions.size(), 5); } /** @@ -110,9 +101,9 @@ protected Partition buildPartition(String dbName, String tableName, String value * @throws Exception */ @Test - public void testgetAllPartitionsOf() throws Exception { + public void testGetAllPartitionsOf() throws Exception { Set part = hive.getAllPartitionsOf(hive.getTable(dbName, tableName)); - Assert.assertEquals(part.size(), 30); + Assert.assertEquals(part.size(), NUM_PARTITIONS); } /** @@ -126,7 +117,7 @@ public void testNumberOfGetPartitionCalls() throws Exception { HiveMetaStoreClient spyMSC = spy(msc); hive.setMSC(spyMSC); // test with a batch size of 10 and decaying factor of 2 - hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),10, 2, 0); + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),10, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED); ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class); // there should be 3 calls to get partitions verify(spyMSC, times(3)).getPartitionsByNames(req.capture()); @@ -144,7 +135,7 @@ public void testUnevenNumberOfGetPartitionCalls() throws Exception { HiveMetaStoreClient spyMSC = spy(msc); hive.setMSC(spyMSC); // there should be 2 calls to get partitions with batch sizes of 19, 11 - hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),19, 2, 0); + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),19, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED); ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class); // there should be 2 calls to get partitions verify(spyMSC, times(2)).getPartitionsByNames(req.capture()); @@ -164,7 +155,7 @@ public void testUnevenNumberOfGetPartitionCalls() throws Exception { public void testSmallNumberOfPartitions() throws Exception { HiveMetaStoreClient spyMSC = spy(msc); hive.setMSC(spyMSC); - hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),100, 2, 0); + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),100, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED); ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class); // there should be 1 call to get partitions verify(spyMSC, times(1)).getPartitionsByNames(req.capture()); @@ -184,7 +175,7 @@ public void testRetriesExhaustedBatchSize() throws Exception { hive.setMSC(spyMSC); doThrow(MetaException.class).when(spyMSC).getPartitionsByNames(any()); try { - hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 0); + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED); } catch (Exception ignored) {} ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class); // there should be 5 call to get partitions with batch sizes as 30, 15, 7, 3, 1 @@ -209,7 +200,7 @@ public void testMaxRetriesReached() throws Exception { hive.setMSC(spyMSC); doThrow(MetaException.class).when(spyMSC).getPartitionsByNames(any()); try { - hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 2); + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 2, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED); } catch (Exception ignored) {} ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class); // there should be 2 call to get partitions with batch sizes as 30, 15 @@ -236,7 +227,7 @@ public void testBatchingWhenException() throws Exception { doThrow(new MetaException()).doCallRealMethod() .when(spyMSC).getPartitionsByNames(any()); - hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 5); + hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 5, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED); ArgumentCaptor req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class); // The first call with batch size of 30 will fail, the rest two call will be of size 15 each. Total 3 calls verify(spyMSC, times(3)).getPartitionsByNames(req.capture()); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp index 9afe386894e1..5ab6a4c81734 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp @@ -2975,14 +2975,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1856; - ::apache::thrift::protocol::TType _etype1859; - xfer += iprot->readListBegin(_etype1859, _size1856); - this->success.resize(_size1856); - uint32_t _i1860; - for (_i1860 = 0; _i1860 < _size1856; ++_i1860) + uint32_t _size1862; + ::apache::thrift::protocol::TType _etype1865; + xfer += iprot->readListBegin(_etype1865, _size1862); + this->success.resize(_size1862); + uint32_t _i1866; + for (_i1866 = 0; _i1866 < _size1862; ++_i1866) { - xfer += iprot->readString(this->success[_i1860]); + xfer += iprot->readString(this->success[_i1866]); } xfer += iprot->readListEnd(); } @@ -3021,10 +3021,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1861; - for (_iter1861 = this->success.begin(); _iter1861 != this->success.end(); ++_iter1861) + std::vector ::const_iterator _iter1867; + for (_iter1867 = this->success.begin(); _iter1867 != this->success.end(); ++_iter1867) { - xfer += oprot->writeString((*_iter1861)); + xfer += oprot->writeString((*_iter1867)); } xfer += oprot->writeListEnd(); } @@ -3069,14 +3069,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1862; - ::apache::thrift::protocol::TType _etype1865; - xfer += iprot->readListBegin(_etype1865, _size1862); - (*(this->success)).resize(_size1862); - uint32_t _i1866; - for (_i1866 = 0; _i1866 < _size1862; ++_i1866) + uint32_t _size1868; + ::apache::thrift::protocol::TType _etype1871; + xfer += iprot->readListBegin(_etype1871, _size1868); + (*(this->success)).resize(_size1868); + uint32_t _i1872; + for (_i1872 = 0; _i1872 < _size1868; ++_i1872) { - xfer += iprot->readString((*(this->success))[_i1866]); + xfer += iprot->readString((*(this->success))[_i1872]); } xfer += iprot->readListEnd(); } @@ -3193,14 +3193,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1867; - ::apache::thrift::protocol::TType _etype1870; - xfer += iprot->readListBegin(_etype1870, _size1867); - this->success.resize(_size1867); - uint32_t _i1871; - for (_i1871 = 0; _i1871 < _size1867; ++_i1871) + uint32_t _size1873; + ::apache::thrift::protocol::TType _etype1876; + xfer += iprot->readListBegin(_etype1876, _size1873); + this->success.resize(_size1873); + uint32_t _i1877; + for (_i1877 = 0; _i1877 < _size1873; ++_i1877) { - xfer += iprot->readString(this->success[_i1871]); + xfer += iprot->readString(this->success[_i1877]); } xfer += iprot->readListEnd(); } @@ -3239,10 +3239,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1872; - for (_iter1872 = this->success.begin(); _iter1872 != this->success.end(); ++_iter1872) + std::vector ::const_iterator _iter1878; + for (_iter1878 = this->success.begin(); _iter1878 != this->success.end(); ++_iter1878) { - xfer += oprot->writeString((*_iter1872)); + xfer += oprot->writeString((*_iter1878)); } xfer += oprot->writeListEnd(); } @@ -3287,14 +3287,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1873; - ::apache::thrift::protocol::TType _etype1876; - xfer += iprot->readListBegin(_etype1876, _size1873); - (*(this->success)).resize(_size1873); - uint32_t _i1877; - for (_i1877 = 0; _i1877 < _size1873; ++_i1877) + uint32_t _size1879; + ::apache::thrift::protocol::TType _etype1882; + xfer += iprot->readListBegin(_etype1882, _size1879); + (*(this->success)).resize(_size1879); + uint32_t _i1883; + for (_i1883 = 0; _i1883 < _size1879; ++_i1883) { - xfer += iprot->readString((*(this->success))[_i1877]); + xfer += iprot->readString((*(this->success))[_i1883]); } xfer += iprot->readListEnd(); } @@ -4347,14 +4347,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1878; - ::apache::thrift::protocol::TType _etype1881; - xfer += iprot->readListBegin(_etype1881, _size1878); - this->success.resize(_size1878); - uint32_t _i1882; - for (_i1882 = 0; _i1882 < _size1878; ++_i1882) + uint32_t _size1884; + ::apache::thrift::protocol::TType _etype1887; + xfer += iprot->readListBegin(_etype1887, _size1884); + this->success.resize(_size1884); + uint32_t _i1888; + for (_i1888 = 0; _i1888 < _size1884; ++_i1888) { - xfer += iprot->readString(this->success[_i1882]); + xfer += iprot->readString(this->success[_i1888]); } xfer += iprot->readListEnd(); } @@ -4393,10 +4393,10 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::write(::apache::thrift:: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1883; - for (_iter1883 = this->success.begin(); _iter1883 != this->success.end(); ++_iter1883) + std::vector ::const_iterator _iter1889; + for (_iter1889 = this->success.begin(); _iter1889 != this->success.end(); ++_iter1889) { - xfer += oprot->writeString((*_iter1883)); + xfer += oprot->writeString((*_iter1889)); } xfer += oprot->writeListEnd(); } @@ -4441,14 +4441,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_presult::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1884; - ::apache::thrift::protocol::TType _etype1887; - xfer += iprot->readListBegin(_etype1887, _size1884); - (*(this->success)).resize(_size1884); - uint32_t _i1888; - for (_i1888 = 0; _i1888 < _size1884; ++_i1888) + uint32_t _size1890; + ::apache::thrift::protocol::TType _etype1893; + xfer += iprot->readListBegin(_etype1893, _size1890); + (*(this->success)).resize(_size1890); + uint32_t _i1894; + for (_i1894 = 0; _i1894 < _size1890; ++_i1894) { - xfer += iprot->readString((*(this->success))[_i1888]); + xfer += iprot->readString((*(this->success))[_i1894]); } xfer += iprot->readListEnd(); } @@ -5510,17 +5510,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_MAP) { { this->success.clear(); - uint32_t _size1889; - ::apache::thrift::protocol::TType _ktype1890; - ::apache::thrift::protocol::TType _vtype1891; - xfer += iprot->readMapBegin(_ktype1890, _vtype1891, _size1889); - uint32_t _i1893; - for (_i1893 = 0; _i1893 < _size1889; ++_i1893) + uint32_t _size1895; + ::apache::thrift::protocol::TType _ktype1896; + ::apache::thrift::protocol::TType _vtype1897; + xfer += iprot->readMapBegin(_ktype1896, _vtype1897, _size1895); + uint32_t _i1899; + for (_i1899 = 0; _i1899 < _size1895; ++_i1899) { - std::string _key1894; - xfer += iprot->readString(_key1894); - Type& _val1895 = this->success[_key1894]; - xfer += _val1895.read(iprot); + std::string _key1900; + xfer += iprot->readString(_key1900); + Type& _val1901 = this->success[_key1900]; + xfer += _val1901.read(iprot); } xfer += iprot->readMapEnd(); } @@ -5559,11 +5559,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::map ::const_iterator _iter1896; - for (_iter1896 = this->success.begin(); _iter1896 != this->success.end(); ++_iter1896) + std::map ::const_iterator _iter1902; + for (_iter1902 = this->success.begin(); _iter1902 != this->success.end(); ++_iter1902) { - xfer += oprot->writeString(_iter1896->first); - xfer += _iter1896->second.write(oprot); + xfer += oprot->writeString(_iter1902->first); + xfer += _iter1902->second.write(oprot); } xfer += oprot->writeMapEnd(); } @@ -5608,17 +5608,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_MAP) { { (*(this->success)).clear(); - uint32_t _size1897; - ::apache::thrift::protocol::TType _ktype1898; - ::apache::thrift::protocol::TType _vtype1899; - xfer += iprot->readMapBegin(_ktype1898, _vtype1899, _size1897); - uint32_t _i1901; - for (_i1901 = 0; _i1901 < _size1897; ++_i1901) + uint32_t _size1903; + ::apache::thrift::protocol::TType _ktype1904; + ::apache::thrift::protocol::TType _vtype1905; + xfer += iprot->readMapBegin(_ktype1904, _vtype1905, _size1903); + uint32_t _i1907; + for (_i1907 = 0; _i1907 < _size1903; ++_i1907) { - std::string _key1902; - xfer += iprot->readString(_key1902); - Type& _val1903 = (*(this->success))[_key1902]; - xfer += _val1903.read(iprot); + std::string _key1908; + xfer += iprot->readString(_key1908); + Type& _val1909 = (*(this->success))[_key1908]; + xfer += _val1909.read(iprot); } xfer += iprot->readMapEnd(); } @@ -5772,14 +5772,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1904; - ::apache::thrift::protocol::TType _etype1907; - xfer += iprot->readListBegin(_etype1907, _size1904); - this->success.resize(_size1904); - uint32_t _i1908; - for (_i1908 = 0; _i1908 < _size1904; ++_i1908) + uint32_t _size1910; + ::apache::thrift::protocol::TType _etype1913; + xfer += iprot->readListBegin(_etype1913, _size1910); + this->success.resize(_size1910); + uint32_t _i1914; + for (_i1914 = 0; _i1914 < _size1910; ++_i1914) { - xfer += this->success[_i1908].read(iprot); + xfer += this->success[_i1914].read(iprot); } xfer += iprot->readListEnd(); } @@ -5834,10 +5834,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1909; - for (_iter1909 = this->success.begin(); _iter1909 != this->success.end(); ++_iter1909) + std::vector ::const_iterator _iter1915; + for (_iter1915 = this->success.begin(); _iter1915 != this->success.end(); ++_iter1915) { - xfer += (*_iter1909).write(oprot); + xfer += (*_iter1915).write(oprot); } xfer += oprot->writeListEnd(); } @@ -5890,14 +5890,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1910; - ::apache::thrift::protocol::TType _etype1913; - xfer += iprot->readListBegin(_etype1913, _size1910); - (*(this->success)).resize(_size1910); - uint32_t _i1914; - for (_i1914 = 0; _i1914 < _size1910; ++_i1914) + uint32_t _size1916; + ::apache::thrift::protocol::TType _etype1919; + xfer += iprot->readListBegin(_etype1919, _size1916); + (*(this->success)).resize(_size1916); + uint32_t _i1920; + for (_i1920 = 0; _i1920 < _size1916; ++_i1920) { - xfer += (*(this->success))[_i1914].read(iprot); + xfer += (*(this->success))[_i1920].read(iprot); } xfer += iprot->readListEnd(); } @@ -6083,14 +6083,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1915; - ::apache::thrift::protocol::TType _etype1918; - xfer += iprot->readListBegin(_etype1918, _size1915); - this->success.resize(_size1915); - uint32_t _i1919; - for (_i1919 = 0; _i1919 < _size1915; ++_i1919) + uint32_t _size1921; + ::apache::thrift::protocol::TType _etype1924; + xfer += iprot->readListBegin(_etype1924, _size1921); + this->success.resize(_size1921); + uint32_t _i1925; + for (_i1925 = 0; _i1925 < _size1921; ++_i1925) { - xfer += this->success[_i1919].read(iprot); + xfer += this->success[_i1925].read(iprot); } xfer += iprot->readListEnd(); } @@ -6145,10 +6145,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1920; - for (_iter1920 = this->success.begin(); _iter1920 != this->success.end(); ++_iter1920) + std::vector ::const_iterator _iter1926; + for (_iter1926 = this->success.begin(); _iter1926 != this->success.end(); ++_iter1926) { - xfer += (*_iter1920).write(oprot); + xfer += (*_iter1926).write(oprot); } xfer += oprot->writeListEnd(); } @@ -6201,14 +6201,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1921; - ::apache::thrift::protocol::TType _etype1924; - xfer += iprot->readListBegin(_etype1924, _size1921); - (*(this->success)).resize(_size1921); - uint32_t _i1925; - for (_i1925 = 0; _i1925 < _size1921; ++_i1925) + uint32_t _size1927; + ::apache::thrift::protocol::TType _etype1930; + xfer += iprot->readListBegin(_etype1930, _size1927); + (*(this->success)).resize(_size1927); + uint32_t _i1931; + for (_i1931 = 0; _i1931 < _size1927; ++_i1931) { - xfer += (*(this->success))[_i1925].read(iprot); + xfer += (*(this->success))[_i1931].read(iprot); } xfer += iprot->readListEnd(); } @@ -6625,14 +6625,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1926; - ::apache::thrift::protocol::TType _etype1929; - xfer += iprot->readListBegin(_etype1929, _size1926); - this->success.resize(_size1926); - uint32_t _i1930; - for (_i1930 = 0; _i1930 < _size1926; ++_i1930) + uint32_t _size1932; + ::apache::thrift::protocol::TType _etype1935; + xfer += iprot->readListBegin(_etype1935, _size1932); + this->success.resize(_size1932); + uint32_t _i1936; + for (_i1936 = 0; _i1936 < _size1932; ++_i1936) { - xfer += this->success[_i1930].read(iprot); + xfer += this->success[_i1936].read(iprot); } xfer += iprot->readListEnd(); } @@ -6687,10 +6687,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1931; - for (_iter1931 = this->success.begin(); _iter1931 != this->success.end(); ++_iter1931) + std::vector ::const_iterator _iter1937; + for (_iter1937 = this->success.begin(); _iter1937 != this->success.end(); ++_iter1937) { - xfer += (*_iter1931).write(oprot); + xfer += (*_iter1937).write(oprot); } xfer += oprot->writeListEnd(); } @@ -6743,14 +6743,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1932; - ::apache::thrift::protocol::TType _etype1935; - xfer += iprot->readListBegin(_etype1935, _size1932); - (*(this->success)).resize(_size1932); - uint32_t _i1936; - for (_i1936 = 0; _i1936 < _size1932; ++_i1936) + uint32_t _size1938; + ::apache::thrift::protocol::TType _etype1941; + xfer += iprot->readListBegin(_etype1941, _size1938); + (*(this->success)).resize(_size1938); + uint32_t _i1942; + for (_i1942 = 0; _i1942 < _size1938; ++_i1942) { - xfer += (*(this->success))[_i1936].read(iprot); + xfer += (*(this->success))[_i1942].read(iprot); } xfer += iprot->readListEnd(); } @@ -6936,14 +6936,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1937; - ::apache::thrift::protocol::TType _etype1940; - xfer += iprot->readListBegin(_etype1940, _size1937); - this->success.resize(_size1937); - uint32_t _i1941; - for (_i1941 = 0; _i1941 < _size1937; ++_i1941) + uint32_t _size1943; + ::apache::thrift::protocol::TType _etype1946; + xfer += iprot->readListBegin(_etype1946, _size1943); + this->success.resize(_size1943); + uint32_t _i1947; + for (_i1947 = 0; _i1947 < _size1943; ++_i1947) { - xfer += this->success[_i1941].read(iprot); + xfer += this->success[_i1947].read(iprot); } xfer += iprot->readListEnd(); } @@ -6998,10 +6998,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1942; - for (_iter1942 = this->success.begin(); _iter1942 != this->success.end(); ++_iter1942) + std::vector ::const_iterator _iter1948; + for (_iter1948 = this->success.begin(); _iter1948 != this->success.end(); ++_iter1948) { - xfer += (*_iter1942).write(oprot); + xfer += (*_iter1948).write(oprot); } xfer += oprot->writeListEnd(); } @@ -7054,14 +7054,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size1943; - ::apache::thrift::protocol::TType _etype1946; - xfer += iprot->readListBegin(_etype1946, _size1943); - (*(this->success)).resize(_size1943); - uint32_t _i1947; - for (_i1947 = 0; _i1947 < _size1943; ++_i1947) + uint32_t _size1949; + ::apache::thrift::protocol::TType _etype1952; + xfer += iprot->readListBegin(_etype1952, _size1949); + (*(this->success)).resize(_size1949); + uint32_t _i1953; + for (_i1953 = 0; _i1953 < _size1949; ++_i1953) { - xfer += (*(this->success))[_i1947].read(iprot); + xfer += (*(this->success))[_i1953].read(iprot); } xfer += iprot->readListEnd(); } @@ -7901,14 +7901,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->primaryKeys.clear(); - uint32_t _size1948; - ::apache::thrift::protocol::TType _etype1951; - xfer += iprot->readListBegin(_etype1951, _size1948); - this->primaryKeys.resize(_size1948); - uint32_t _i1952; - for (_i1952 = 0; _i1952 < _size1948; ++_i1952) + uint32_t _size1954; + ::apache::thrift::protocol::TType _etype1957; + xfer += iprot->readListBegin(_etype1957, _size1954); + this->primaryKeys.resize(_size1954); + uint32_t _i1958; + for (_i1958 = 0; _i1958 < _size1954; ++_i1958) { - xfer += this->primaryKeys[_i1952].read(iprot); + xfer += this->primaryKeys[_i1958].read(iprot); } xfer += iprot->readListEnd(); } @@ -7921,14 +7921,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->foreignKeys.clear(); - uint32_t _size1953; - ::apache::thrift::protocol::TType _etype1956; - xfer += iprot->readListBegin(_etype1956, _size1953); - this->foreignKeys.resize(_size1953); - uint32_t _i1957; - for (_i1957 = 0; _i1957 < _size1953; ++_i1957) + uint32_t _size1959; + ::apache::thrift::protocol::TType _etype1962; + xfer += iprot->readListBegin(_etype1962, _size1959); + this->foreignKeys.resize(_size1959); + uint32_t _i1963; + for (_i1963 = 0; _i1963 < _size1959; ++_i1963) { - xfer += this->foreignKeys[_i1957].read(iprot); + xfer += this->foreignKeys[_i1963].read(iprot); } xfer += iprot->readListEnd(); } @@ -7941,14 +7941,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->uniqueConstraints.clear(); - uint32_t _size1958; - ::apache::thrift::protocol::TType _etype1961; - xfer += iprot->readListBegin(_etype1961, _size1958); - this->uniqueConstraints.resize(_size1958); - uint32_t _i1962; - for (_i1962 = 0; _i1962 < _size1958; ++_i1962) + uint32_t _size1964; + ::apache::thrift::protocol::TType _etype1967; + xfer += iprot->readListBegin(_etype1967, _size1964); + this->uniqueConstraints.resize(_size1964); + uint32_t _i1968; + for (_i1968 = 0; _i1968 < _size1964; ++_i1968) { - xfer += this->uniqueConstraints[_i1962].read(iprot); + xfer += this->uniqueConstraints[_i1968].read(iprot); } xfer += iprot->readListEnd(); } @@ -7961,14 +7961,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->notNullConstraints.clear(); - uint32_t _size1963; - ::apache::thrift::protocol::TType _etype1966; - xfer += iprot->readListBegin(_etype1966, _size1963); - this->notNullConstraints.resize(_size1963); - uint32_t _i1967; - for (_i1967 = 0; _i1967 < _size1963; ++_i1967) + uint32_t _size1969; + ::apache::thrift::protocol::TType _etype1972; + xfer += iprot->readListBegin(_etype1972, _size1969); + this->notNullConstraints.resize(_size1969); + uint32_t _i1973; + for (_i1973 = 0; _i1973 < _size1969; ++_i1973) { - xfer += this->notNullConstraints[_i1967].read(iprot); + xfer += this->notNullConstraints[_i1973].read(iprot); } xfer += iprot->readListEnd(); } @@ -7981,14 +7981,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->defaultConstraints.clear(); - uint32_t _size1968; - ::apache::thrift::protocol::TType _etype1971; - xfer += iprot->readListBegin(_etype1971, _size1968); - this->defaultConstraints.resize(_size1968); - uint32_t _i1972; - for (_i1972 = 0; _i1972 < _size1968; ++_i1972) + uint32_t _size1974; + ::apache::thrift::protocol::TType _etype1977; + xfer += iprot->readListBegin(_etype1977, _size1974); + this->defaultConstraints.resize(_size1974); + uint32_t _i1978; + for (_i1978 = 0; _i1978 < _size1974; ++_i1978) { - xfer += this->defaultConstraints[_i1972].read(iprot); + xfer += this->defaultConstraints[_i1978].read(iprot); } xfer += iprot->readListEnd(); } @@ -8001,14 +8001,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->checkConstraints.clear(); - uint32_t _size1973; - ::apache::thrift::protocol::TType _etype1976; - xfer += iprot->readListBegin(_etype1976, _size1973); - this->checkConstraints.resize(_size1973); - uint32_t _i1977; - for (_i1977 = 0; _i1977 < _size1973; ++_i1977) + uint32_t _size1979; + ::apache::thrift::protocol::TType _etype1982; + xfer += iprot->readListBegin(_etype1982, _size1979); + this->checkConstraints.resize(_size1979); + uint32_t _i1983; + for (_i1983 = 0; _i1983 < _size1979; ++_i1983) { - xfer += this->checkConstraints[_i1977].read(iprot); + xfer += this->checkConstraints[_i1983].read(iprot); } xfer += iprot->readListEnd(); } @@ -8041,10 +8041,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); - std::vector ::const_iterator _iter1978; - for (_iter1978 = this->primaryKeys.begin(); _iter1978 != this->primaryKeys.end(); ++_iter1978) + std::vector ::const_iterator _iter1984; + for (_iter1984 = this->primaryKeys.begin(); _iter1984 != this->primaryKeys.end(); ++_iter1984) { - xfer += (*_iter1978).write(oprot); + xfer += (*_iter1984).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8053,10 +8053,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); - std::vector ::const_iterator _iter1979; - for (_iter1979 = this->foreignKeys.begin(); _iter1979 != this->foreignKeys.end(); ++_iter1979) + std::vector ::const_iterator _iter1985; + for (_iter1985 = this->foreignKeys.begin(); _iter1985 != this->foreignKeys.end(); ++_iter1985) { - xfer += (*_iter1979).write(oprot); + xfer += (*_iter1985).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8065,10 +8065,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); - std::vector ::const_iterator _iter1980; - for (_iter1980 = this->uniqueConstraints.begin(); _iter1980 != this->uniqueConstraints.end(); ++_iter1980) + std::vector ::const_iterator _iter1986; + for (_iter1986 = this->uniqueConstraints.begin(); _iter1986 != this->uniqueConstraints.end(); ++_iter1986) { - xfer += (*_iter1980).write(oprot); + xfer += (*_iter1986).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8077,10 +8077,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); - std::vector ::const_iterator _iter1981; - for (_iter1981 = this->notNullConstraints.begin(); _iter1981 != this->notNullConstraints.end(); ++_iter1981) + std::vector ::const_iterator _iter1987; + for (_iter1987 = this->notNullConstraints.begin(); _iter1987 != this->notNullConstraints.end(); ++_iter1987) { - xfer += (*_iter1981).write(oprot); + xfer += (*_iter1987).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8089,10 +8089,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->defaultConstraints.size())); - std::vector ::const_iterator _iter1982; - for (_iter1982 = this->defaultConstraints.begin(); _iter1982 != this->defaultConstraints.end(); ++_iter1982) + std::vector ::const_iterator _iter1988; + for (_iter1988 = this->defaultConstraints.begin(); _iter1988 != this->defaultConstraints.end(); ++_iter1988) { - xfer += (*_iter1982).write(oprot); + xfer += (*_iter1988).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8101,10 +8101,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache: xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->checkConstraints.size())); - std::vector ::const_iterator _iter1983; - for (_iter1983 = this->checkConstraints.begin(); _iter1983 != this->checkConstraints.end(); ++_iter1983) + std::vector ::const_iterator _iter1989; + for (_iter1989 = this->checkConstraints.begin(); _iter1989 != this->checkConstraints.end(); ++_iter1989) { - xfer += (*_iter1983).write(oprot); + xfer += (*_iter1989).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8132,10 +8132,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->primaryKeys)).size())); - std::vector ::const_iterator _iter1984; - for (_iter1984 = (*(this->primaryKeys)).begin(); _iter1984 != (*(this->primaryKeys)).end(); ++_iter1984) + std::vector ::const_iterator _iter1990; + for (_iter1990 = (*(this->primaryKeys)).begin(); _iter1990 != (*(this->primaryKeys)).end(); ++_iter1990) { - xfer += (*_iter1984).write(oprot); + xfer += (*_iter1990).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8144,10 +8144,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->foreignKeys)).size())); - std::vector ::const_iterator _iter1985; - for (_iter1985 = (*(this->foreignKeys)).begin(); _iter1985 != (*(this->foreignKeys)).end(); ++_iter1985) + std::vector ::const_iterator _iter1991; + for (_iter1991 = (*(this->foreignKeys)).begin(); _iter1991 != (*(this->foreignKeys)).end(); ++_iter1991) { - xfer += (*_iter1985).write(oprot); + xfer += (*_iter1991).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8156,10 +8156,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->uniqueConstraints)).size())); - std::vector ::const_iterator _iter1986; - for (_iter1986 = (*(this->uniqueConstraints)).begin(); _iter1986 != (*(this->uniqueConstraints)).end(); ++_iter1986) + std::vector ::const_iterator _iter1992; + for (_iter1992 = (*(this->uniqueConstraints)).begin(); _iter1992 != (*(this->uniqueConstraints)).end(); ++_iter1992) { - xfer += (*_iter1986).write(oprot); + xfer += (*_iter1992).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8168,10 +8168,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->notNullConstraints)).size())); - std::vector ::const_iterator _iter1987; - for (_iter1987 = (*(this->notNullConstraints)).begin(); _iter1987 != (*(this->notNullConstraints)).end(); ++_iter1987) + std::vector ::const_iterator _iter1993; + for (_iter1993 = (*(this->notNullConstraints)).begin(); _iter1993 != (*(this->notNullConstraints)).end(); ++_iter1993) { - xfer += (*_iter1987).write(oprot); + xfer += (*_iter1993).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8180,10 +8180,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->defaultConstraints)).size())); - std::vector ::const_iterator _iter1988; - for (_iter1988 = (*(this->defaultConstraints)).begin(); _iter1988 != (*(this->defaultConstraints)).end(); ++_iter1988) + std::vector ::const_iterator _iter1994; + for (_iter1994 = (*(this->defaultConstraints)).begin(); _iter1994 != (*(this->defaultConstraints)).end(); ++_iter1994) { - xfer += (*_iter1988).write(oprot); + xfer += (*_iter1994).write(oprot); } xfer += oprot->writeListEnd(); } @@ -8192,10 +8192,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->checkConstraints)).size())); - std::vector ::const_iterator _iter1989; - for (_iter1989 = (*(this->checkConstraints)).begin(); _iter1989 != (*(this->checkConstraints)).end(); ++_iter1989) + std::vector ::const_iterator _iter1995; + for (_iter1995 = (*(this->checkConstraints)).begin(); _iter1995 != (*(this->checkConstraints)).end(); ++_iter1995) { - xfer += (*_iter1989).write(oprot); + xfer += (*_iter1995).write(oprot); } xfer += oprot->writeListEnd(); } @@ -10877,14 +10877,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partNames.clear(); - uint32_t _size1990; - ::apache::thrift::protocol::TType _etype1993; - xfer += iprot->readListBegin(_etype1993, _size1990); - this->partNames.resize(_size1990); - uint32_t _i1994; - for (_i1994 = 0; _i1994 < _size1990; ++_i1994) + uint32_t _size1996; + ::apache::thrift::protocol::TType _etype1999; + xfer += iprot->readListBegin(_etype1999, _size1996); + this->partNames.resize(_size1996); + uint32_t _i2000; + for (_i2000 = 0; _i2000 < _size1996; ++_i2000) { - xfer += iprot->readString(this->partNames[_i1994]); + xfer += iprot->readString(this->partNames[_i2000]); } xfer += iprot->readListEnd(); } @@ -10921,10 +10921,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); - std::vector ::const_iterator _iter1995; - for (_iter1995 = this->partNames.begin(); _iter1995 != this->partNames.end(); ++_iter1995) + std::vector ::const_iterator _iter2001; + for (_iter2001 = this->partNames.begin(); _iter2001 != this->partNames.end(); ++_iter2001) { - xfer += oprot->writeString((*_iter1995)); + xfer += oprot->writeString((*_iter2001)); } xfer += oprot->writeListEnd(); } @@ -10956,10 +10956,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->partNames)).size())); - std::vector ::const_iterator _iter1996; - for (_iter1996 = (*(this->partNames)).begin(); _iter1996 != (*(this->partNames)).end(); ++_iter1996) + std::vector ::const_iterator _iter2002; + for (_iter2002 = (*(this->partNames)).begin(); _iter2002 != (*(this->partNames)).end(); ++_iter2002) { - xfer += oprot->writeString((*_iter1996)); + xfer += oprot->writeString((*_iter2002)); } xfer += oprot->writeListEnd(); } @@ -11410,14 +11410,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size1997; - ::apache::thrift::protocol::TType _etype2000; - xfer += iprot->readListBegin(_etype2000, _size1997); - this->success.resize(_size1997); - uint32_t _i2001; - for (_i2001 = 0; _i2001 < _size1997; ++_i2001) + uint32_t _size2003; + ::apache::thrift::protocol::TType _etype2006; + xfer += iprot->readListBegin(_etype2006, _size2003); + this->success.resize(_size2003); + uint32_t _i2007; + for (_i2007 = 0; _i2007 < _size2003; ++_i2007) { - xfer += iprot->readString(this->success[_i2001]); + xfer += iprot->readString(this->success[_i2007]); } xfer += iprot->readListEnd(); } @@ -11456,10 +11456,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2002; - for (_iter2002 = this->success.begin(); _iter2002 != this->success.end(); ++_iter2002) + std::vector ::const_iterator _iter2008; + for (_iter2008 = this->success.begin(); _iter2008 != this->success.end(); ++_iter2008) { - xfer += oprot->writeString((*_iter2002)); + xfer += oprot->writeString((*_iter2008)); } xfer += oprot->writeListEnd(); } @@ -11504,14 +11504,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2003; - ::apache::thrift::protocol::TType _etype2006; - xfer += iprot->readListBegin(_etype2006, _size2003); - (*(this->success)).resize(_size2003); - uint32_t _i2007; - for (_i2007 = 0; _i2007 < _size2003; ++_i2007) + uint32_t _size2009; + ::apache::thrift::protocol::TType _etype2012; + xfer += iprot->readListBegin(_etype2012, _size2009); + (*(this->success)).resize(_size2009); + uint32_t _i2013; + for (_i2013 = 0; _i2013 < _size2009; ++_i2013) { - xfer += iprot->readString((*(this->success))[_i2007]); + xfer += iprot->readString((*(this->success))[_i2013]); } xfer += iprot->readListEnd(); } @@ -11681,14 +11681,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2008; - ::apache::thrift::protocol::TType _etype2011; - xfer += iprot->readListBegin(_etype2011, _size2008); - this->success.resize(_size2008); - uint32_t _i2012; - for (_i2012 = 0; _i2012 < _size2008; ++_i2012) + uint32_t _size2014; + ::apache::thrift::protocol::TType _etype2017; + xfer += iprot->readListBegin(_etype2017, _size2014); + this->success.resize(_size2014); + uint32_t _i2018; + for (_i2018 = 0; _i2018 < _size2014; ++_i2018) { - xfer += iprot->readString(this->success[_i2012]); + xfer += iprot->readString(this->success[_i2018]); } xfer += iprot->readListEnd(); } @@ -11727,10 +11727,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift:: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2013; - for (_iter2013 = this->success.begin(); _iter2013 != this->success.end(); ++_iter2013) + std::vector ::const_iterator _iter2019; + for (_iter2019 = this->success.begin(); _iter2019 != this->success.end(); ++_iter2019) { - xfer += oprot->writeString((*_iter2013)); + xfer += oprot->writeString((*_iter2019)); } xfer += oprot->writeListEnd(); } @@ -11775,14 +11775,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2014; - ::apache::thrift::protocol::TType _etype2017; - xfer += iprot->readListBegin(_etype2017, _size2014); - (*(this->success)).resize(_size2014); - uint32_t _i2018; - for (_i2018 = 0; _i2018 < _size2014; ++_i2018) + uint32_t _size2020; + ::apache::thrift::protocol::TType _etype2023; + xfer += iprot->readListBegin(_etype2023, _size2020); + (*(this->success)).resize(_size2020); + uint32_t _i2024; + for (_i2024 = 0; _i2024 < _size2020; ++_i2024) { - xfer += iprot->readString((*(this->success))[_i2018]); + xfer += iprot->readString((*(this->success))[_i2024]); } xfer += iprot->readListEnd(); } @@ -11899,14 +11899,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2019; - ::apache::thrift::protocol::TType _etype2022; - xfer += iprot->readListBegin(_etype2022, _size2019); - this->success.resize(_size2019); - uint32_t _i2023; - for (_i2023 = 0; _i2023 < _size2019; ++_i2023) + uint32_t _size2025; + ::apache::thrift::protocol::TType _etype2028; + xfer += iprot->readListBegin(_etype2028, _size2025); + this->success.resize(_size2025); + uint32_t _i2029; + for (_i2029 = 0; _i2029 < _size2025; ++_i2029) { - xfer += this->success[_i2023].read(iprot); + xfer += this->success[_i2029].read(iprot); } xfer += iprot->readListEnd(); } @@ -11945,10 +11945,10 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector
::const_iterator _iter2024; - for (_iter2024 = this->success.begin(); _iter2024 != this->success.end(); ++_iter2024) + std::vector
::const_iterator _iter2030; + for (_iter2030 = this->success.begin(); _iter2030 != this->success.end(); ++_iter2030) { - xfer += (*_iter2024).write(oprot); + xfer += (*_iter2030).write(oprot); } xfer += oprot->writeListEnd(); } @@ -11993,14 +11993,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_pre if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2025; - ::apache::thrift::protocol::TType _etype2028; - xfer += iprot->readListBegin(_etype2028, _size2025); - (*(this->success)).resize(_size2025); - uint32_t _i2029; - for (_i2029 = 0; _i2029 < _size2025; ++_i2029) + uint32_t _size2031; + ::apache::thrift::protocol::TType _etype2034; + xfer += iprot->readListBegin(_etype2034, _size2031); + (*(this->success)).resize(_size2031); + uint32_t _i2035; + for (_i2035 = 0; _i2035 < _size2031; ++_i2035) { - xfer += (*(this->success))[_i2029].read(iprot); + xfer += (*(this->success))[_i2035].read(iprot); } xfer += iprot->readListEnd(); } @@ -12138,14 +12138,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2030; - ::apache::thrift::protocol::TType _etype2033; - xfer += iprot->readListBegin(_etype2033, _size2030); - this->success.resize(_size2030); - uint32_t _i2034; - for (_i2034 = 0; _i2034 < _size2030; ++_i2034) + uint32_t _size2036; + ::apache::thrift::protocol::TType _etype2039; + xfer += iprot->readListBegin(_etype2039, _size2036); + this->success.resize(_size2036); + uint32_t _i2040; + for (_i2040 = 0; _i2040 < _size2036; ++_i2040) { - xfer += iprot->readString(this->success[_i2034]); + xfer += iprot->readString(this->success[_i2040]); } xfer += iprot->readListEnd(); } @@ -12184,10 +12184,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write( xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2035; - for (_iter2035 = this->success.begin(); _iter2035 != this->success.end(); ++_iter2035) + std::vector ::const_iterator _iter2041; + for (_iter2041 = this->success.begin(); _iter2041 != this->success.end(); ++_iter2041) { - xfer += oprot->writeString((*_iter2035)); + xfer += oprot->writeString((*_iter2041)); } xfer += oprot->writeListEnd(); } @@ -12232,14 +12232,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read( if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2036; - ::apache::thrift::protocol::TType _etype2039; - xfer += iprot->readListBegin(_etype2039, _size2036); - (*(this->success)).resize(_size2036); - uint32_t _i2040; - for (_i2040 = 0; _i2040 < _size2036; ++_i2040) + uint32_t _size2042; + ::apache::thrift::protocol::TType _etype2045; + xfer += iprot->readListBegin(_etype2045, _size2042); + (*(this->success)).resize(_size2042); + uint32_t _i2046; + for (_i2046 = 0; _i2046 < _size2042; ++_i2046) { - xfer += iprot->readString((*(this->success))[_i2040]); + xfer += iprot->readString((*(this->success))[_i2046]); } xfer += iprot->readListEnd(); } @@ -12314,14 +12314,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tbl_types.clear(); - uint32_t _size2041; - ::apache::thrift::protocol::TType _etype2044; - xfer += iprot->readListBegin(_etype2044, _size2041); - this->tbl_types.resize(_size2041); - uint32_t _i2045; - for (_i2045 = 0; _i2045 < _size2041; ++_i2045) + uint32_t _size2047; + ::apache::thrift::protocol::TType _etype2050; + xfer += iprot->readListBegin(_etype2050, _size2047); + this->tbl_types.resize(_size2047); + uint32_t _i2051; + for (_i2051 = 0; _i2051 < _size2047; ++_i2051) { - xfer += iprot->readString(this->tbl_types[_i2045]); + xfer += iprot->readString(this->tbl_types[_i2051]); } xfer += iprot->readListEnd(); } @@ -12358,10 +12358,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_types.size())); - std::vector ::const_iterator _iter2046; - for (_iter2046 = this->tbl_types.begin(); _iter2046 != this->tbl_types.end(); ++_iter2046) + std::vector ::const_iterator _iter2052; + for (_iter2052 = this->tbl_types.begin(); _iter2052 != this->tbl_types.end(); ++_iter2052) { - xfer += oprot->writeString((*_iter2046)); + xfer += oprot->writeString((*_iter2052)); } xfer += oprot->writeListEnd(); } @@ -12393,10 +12393,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_types)).size())); - std::vector ::const_iterator _iter2047; - for (_iter2047 = (*(this->tbl_types)).begin(); _iter2047 != (*(this->tbl_types)).end(); ++_iter2047) + std::vector ::const_iterator _iter2053; + for (_iter2053 = (*(this->tbl_types)).begin(); _iter2053 != (*(this->tbl_types)).end(); ++_iter2053) { - xfer += oprot->writeString((*_iter2047)); + xfer += oprot->writeString((*_iter2053)); } xfer += oprot->writeListEnd(); } @@ -12437,14 +12437,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2048; - ::apache::thrift::protocol::TType _etype2051; - xfer += iprot->readListBegin(_etype2051, _size2048); - this->success.resize(_size2048); - uint32_t _i2052; - for (_i2052 = 0; _i2052 < _size2048; ++_i2052) + uint32_t _size2054; + ::apache::thrift::protocol::TType _etype2057; + xfer += iprot->readListBegin(_etype2057, _size2054); + this->success.resize(_size2054); + uint32_t _i2058; + for (_i2058 = 0; _i2058 < _size2054; ++_i2058) { - xfer += this->success[_i2052].read(iprot); + xfer += this->success[_i2058].read(iprot); } xfer += iprot->readListEnd(); } @@ -12483,10 +12483,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2053; - for (_iter2053 = this->success.begin(); _iter2053 != this->success.end(); ++_iter2053) + std::vector ::const_iterator _iter2059; + for (_iter2059 = this->success.begin(); _iter2059 != this->success.end(); ++_iter2059) { - xfer += (*_iter2053).write(oprot); + xfer += (*_iter2059).write(oprot); } xfer += oprot->writeListEnd(); } @@ -12531,14 +12531,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2054; - ::apache::thrift::protocol::TType _etype2057; - xfer += iprot->readListBegin(_etype2057, _size2054); - (*(this->success)).resize(_size2054); - uint32_t _i2058; - for (_i2058 = 0; _i2058 < _size2054; ++_i2058) + uint32_t _size2060; + ::apache::thrift::protocol::TType _etype2063; + xfer += iprot->readListBegin(_etype2063, _size2060); + (*(this->success)).resize(_size2060); + uint32_t _i2064; + for (_i2064 = 0; _i2064 < _size2060; ++_i2064) { - xfer += (*(this->success))[_i2058].read(iprot); + xfer += (*(this->success))[_i2064].read(iprot); } xfer += iprot->readListEnd(); } @@ -12676,14 +12676,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2059; - ::apache::thrift::protocol::TType _etype2062; - xfer += iprot->readListBegin(_etype2062, _size2059); - this->success.resize(_size2059); - uint32_t _i2063; - for (_i2063 = 0; _i2063 < _size2059; ++_i2063) + uint32_t _size2065; + ::apache::thrift::protocol::TType _etype2068; + xfer += iprot->readListBegin(_etype2068, _size2065); + this->success.resize(_size2065); + uint32_t _i2069; + for (_i2069 = 0; _i2069 < _size2065; ++_i2069) { - xfer += iprot->readString(this->success[_i2063]); + xfer += iprot->readString(this->success[_i2069]); } xfer += iprot->readListEnd(); } @@ -12722,10 +12722,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2064; - for (_iter2064 = this->success.begin(); _iter2064 != this->success.end(); ++_iter2064) + std::vector ::const_iterator _iter2070; + for (_iter2070 = this->success.begin(); _iter2070 != this->success.end(); ++_iter2070) { - xfer += oprot->writeString((*_iter2064)); + xfer += oprot->writeString((*_iter2070)); } xfer += oprot->writeListEnd(); } @@ -12770,14 +12770,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2065; - ::apache::thrift::protocol::TType _etype2068; - xfer += iprot->readListBegin(_etype2068, _size2065); - (*(this->success)).resize(_size2065); - uint32_t _i2069; - for (_i2069 = 0; _i2069 < _size2065; ++_i2069) + uint32_t _size2071; + ::apache::thrift::protocol::TType _etype2074; + xfer += iprot->readListBegin(_etype2074, _size2071); + (*(this->success)).resize(_size2071); + uint32_t _i2075; + for (_i2075 = 0; _i2075 < _size2071; ++_i2075) { - xfer += iprot->readString((*(this->success))[_i2069]); + xfer += iprot->readString((*(this->success))[_i2075]); } xfer += iprot->readListEnd(); } @@ -13087,14 +13087,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tbl_names.clear(); - uint32_t _size2070; - ::apache::thrift::protocol::TType _etype2073; - xfer += iprot->readListBegin(_etype2073, _size2070); - this->tbl_names.resize(_size2070); - uint32_t _i2074; - for (_i2074 = 0; _i2074 < _size2070; ++_i2074) + uint32_t _size2076; + ::apache::thrift::protocol::TType _etype2079; + xfer += iprot->readListBegin(_etype2079, _size2076); + this->tbl_names.resize(_size2076); + uint32_t _i2080; + for (_i2080 = 0; _i2080 < _size2076; ++_i2080) { - xfer += iprot->readString(this->tbl_names[_i2074]); + xfer += iprot->readString(this->tbl_names[_i2080]); } xfer += iprot->readListEnd(); } @@ -13127,10 +13127,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_names.size())); - std::vector ::const_iterator _iter2075; - for (_iter2075 = this->tbl_names.begin(); _iter2075 != this->tbl_names.end(); ++_iter2075) + std::vector ::const_iterator _iter2081; + for (_iter2081 = this->tbl_names.begin(); _iter2081 != this->tbl_names.end(); ++_iter2081) { - xfer += oprot->writeString((*_iter2075)); + xfer += oprot->writeString((*_iter2081)); } xfer += oprot->writeListEnd(); } @@ -13158,10 +13158,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_names)).size())); - std::vector ::const_iterator _iter2076; - for (_iter2076 = (*(this->tbl_names)).begin(); _iter2076 != (*(this->tbl_names)).end(); ++_iter2076) + std::vector ::const_iterator _iter2082; + for (_iter2082 = (*(this->tbl_names)).begin(); _iter2082 != (*(this->tbl_names)).end(); ++_iter2082) { - xfer += oprot->writeString((*_iter2076)); + xfer += oprot->writeString((*_iter2082)); } xfer += oprot->writeListEnd(); } @@ -13202,14 +13202,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2077; - ::apache::thrift::protocol::TType _etype2080; - xfer += iprot->readListBegin(_etype2080, _size2077); - this->success.resize(_size2077); - uint32_t _i2081; - for (_i2081 = 0; _i2081 < _size2077; ++_i2081) + uint32_t _size2083; + ::apache::thrift::protocol::TType _etype2086; + xfer += iprot->readListBegin(_etype2086, _size2083); + this->success.resize(_size2083); + uint32_t _i2087; + for (_i2087 = 0; _i2087 < _size2083; ++_i2087) { - xfer += this->success[_i2081].read(iprot); + xfer += this->success[_i2087].read(iprot); } xfer += iprot->readListEnd(); } @@ -13240,10 +13240,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector
::const_iterator _iter2082; - for (_iter2082 = this->success.begin(); _iter2082 != this->success.end(); ++_iter2082) + std::vector
::const_iterator _iter2088; + for (_iter2088 = this->success.begin(); _iter2088 != this->success.end(); ++_iter2088) { - xfer += (*_iter2082).write(oprot); + xfer += (*_iter2088).write(oprot); } xfer += oprot->writeListEnd(); } @@ -13284,14 +13284,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2083; - ::apache::thrift::protocol::TType _etype2086; - xfer += iprot->readListBegin(_etype2086, _size2083); - (*(this->success)).resize(_size2083); - uint32_t _i2087; - for (_i2087 = 0; _i2087 < _size2083; ++_i2087) + uint32_t _size2089; + ::apache::thrift::protocol::TType _etype2092; + xfer += iprot->readListBegin(_etype2092, _size2089); + (*(this->success)).resize(_size2089); + uint32_t _i2093; + for (_i2093 = 0; _i2093 < _size2089; ++_i2093) { - xfer += (*(this->success))[_i2087].read(iprot); + xfer += (*(this->success))[_i2093].read(iprot); } xfer += iprot->readListEnd(); } @@ -13421,14 +13421,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2088; - ::apache::thrift::protocol::TType _etype2091; - xfer += iprot->readListBegin(_etype2091, _size2088); - this->success.resize(_size2088); - uint32_t _i2092; - for (_i2092 = 0; _i2092 < _size2088; ++_i2092) + uint32_t _size2094; + ::apache::thrift::protocol::TType _etype2097; + xfer += iprot->readListBegin(_etype2097, _size2094); + this->success.resize(_size2094); + uint32_t _i2098; + for (_i2098 = 0; _i2098 < _size2094; ++_i2098) { - xfer += this->success[_i2092].read(iprot); + xfer += this->success[_i2098].read(iprot); } xfer += iprot->readListEnd(); } @@ -13467,10 +13467,10 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2093; - for (_iter2093 = this->success.begin(); _iter2093 != this->success.end(); ++_iter2093) + std::vector ::const_iterator _iter2099; + for (_iter2099 = this->success.begin(); _iter2099 != this->success.end(); ++_iter2099) { - xfer += (*_iter2093).write(oprot); + xfer += (*_iter2099).write(oprot); } xfer += oprot->writeListEnd(); } @@ -13515,14 +13515,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2094; - ::apache::thrift::protocol::TType _etype2097; - xfer += iprot->readListBegin(_etype2097, _size2094); - (*(this->success)).resize(_size2094); - uint32_t _i2098; - for (_i2098 = 0; _i2098 < _size2094; ++_i2098) + uint32_t _size2100; + ::apache::thrift::protocol::TType _etype2103; + xfer += iprot->readListBegin(_etype2103, _size2100); + (*(this->success)).resize(_size2100); + uint32_t _i2104; + for (_i2104 = 0; _i2104 < _size2100; ++_i2104) { - xfer += (*(this->success))[_i2098].read(iprot); + xfer += (*(this->success))[_i2104].read(iprot); } xfer += iprot->readListEnd(); } @@ -14704,14 +14704,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2099; - ::apache::thrift::protocol::TType _etype2102; - xfer += iprot->readListBegin(_etype2102, _size2099); - this->success.resize(_size2099); - uint32_t _i2103; - for (_i2103 = 0; _i2103 < _size2099; ++_i2103) + uint32_t _size2105; + ::apache::thrift::protocol::TType _etype2108; + xfer += iprot->readListBegin(_etype2108, _size2105); + this->success.resize(_size2105); + uint32_t _i2109; + for (_i2109 = 0; _i2109 < _size2105; ++_i2109) { - xfer += iprot->readString(this->success[_i2103]); + xfer += iprot->readString(this->success[_i2109]); } xfer += iprot->readListEnd(); } @@ -14766,10 +14766,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2104; - for (_iter2104 = this->success.begin(); _iter2104 != this->success.end(); ++_iter2104) + std::vector ::const_iterator _iter2110; + for (_iter2110 = this->success.begin(); _iter2110 != this->success.end(); ++_iter2110) { - xfer += oprot->writeString((*_iter2104)); + xfer += oprot->writeString((*_iter2110)); } xfer += oprot->writeListEnd(); } @@ -14822,14 +14822,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2105; - ::apache::thrift::protocol::TType _etype2108; - xfer += iprot->readListBegin(_etype2108, _size2105); - (*(this->success)).resize(_size2105); - uint32_t _i2109; - for (_i2109 = 0; _i2109 < _size2105; ++_i2109) + uint32_t _size2111; + ::apache::thrift::protocol::TType _etype2114; + xfer += iprot->readListBegin(_etype2114, _size2111); + (*(this->success)).resize(_size2111); + uint32_t _i2115; + for (_i2115 = 0; _i2115 < _size2111; ++_i2115) { - xfer += iprot->readString((*(this->success))[_i2109]); + xfer += iprot->readString((*(this->success))[_i2115]); } xfer += iprot->readListEnd(); } @@ -16390,14 +16390,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2110; - ::apache::thrift::protocol::TType _etype2113; - xfer += iprot->readListBegin(_etype2113, _size2110); - this->new_parts.resize(_size2110); - uint32_t _i2114; - for (_i2114 = 0; _i2114 < _size2110; ++_i2114) + uint32_t _size2116; + ::apache::thrift::protocol::TType _etype2119; + xfer += iprot->readListBegin(_etype2119, _size2116); + this->new_parts.resize(_size2116); + uint32_t _i2120; + for (_i2120 = 0; _i2120 < _size2116; ++_i2120) { - xfer += this->new_parts[_i2114].read(iprot); + xfer += this->new_parts[_i2120].read(iprot); } xfer += iprot->readListEnd(); } @@ -16426,10 +16426,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2115; - for (_iter2115 = this->new_parts.begin(); _iter2115 != this->new_parts.end(); ++_iter2115) + std::vector ::const_iterator _iter2121; + for (_iter2121 = this->new_parts.begin(); _iter2121 != this->new_parts.end(); ++_iter2121) { - xfer += (*_iter2115).write(oprot); + xfer += (*_iter2121).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16453,10 +16453,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2116; - for (_iter2116 = (*(this->new_parts)).begin(); _iter2116 != (*(this->new_parts)).end(); ++_iter2116) + std::vector ::const_iterator _iter2122; + for (_iter2122 = (*(this->new_parts)).begin(); _iter2122 != (*(this->new_parts)).end(); ++_iter2122) { - xfer += (*_iter2116).write(oprot); + xfer += (*_iter2122).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16665,14 +16665,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2117; - ::apache::thrift::protocol::TType _etype2120; - xfer += iprot->readListBegin(_etype2120, _size2117); - this->new_parts.resize(_size2117); - uint32_t _i2121; - for (_i2121 = 0; _i2121 < _size2117; ++_i2121) + uint32_t _size2123; + ::apache::thrift::protocol::TType _etype2126; + xfer += iprot->readListBegin(_etype2126, _size2123); + this->new_parts.resize(_size2123); + uint32_t _i2127; + for (_i2127 = 0; _i2127 < _size2123; ++_i2127) { - xfer += this->new_parts[_i2121].read(iprot); + xfer += this->new_parts[_i2127].read(iprot); } xfer += iprot->readListEnd(); } @@ -16701,10 +16701,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift:: xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2122; - for (_iter2122 = this->new_parts.begin(); _iter2122 != this->new_parts.end(); ++_iter2122) + std::vector ::const_iterator _iter2128; + for (_iter2128 = this->new_parts.begin(); _iter2128 != this->new_parts.end(); ++_iter2128) { - xfer += (*_iter2122).write(oprot); + xfer += (*_iter2128).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16728,10 +16728,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift: xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2123; - for (_iter2123 = (*(this->new_parts)).begin(); _iter2123 != (*(this->new_parts)).end(); ++_iter2123) + std::vector ::const_iterator _iter2129; + for (_iter2129 = (*(this->new_parts)).begin(); _iter2129 != (*(this->new_parts)).end(); ++_iter2129) { - xfer += (*_iter2123).write(oprot); + xfer += (*_iter2129).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16956,14 +16956,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2124; - ::apache::thrift::protocol::TType _etype2127; - xfer += iprot->readListBegin(_etype2127, _size2124); - this->part_vals.resize(_size2124); - uint32_t _i2128; - for (_i2128 = 0; _i2128 < _size2124; ++_i2128) + uint32_t _size2130; + ::apache::thrift::protocol::TType _etype2133; + xfer += iprot->readListBegin(_etype2133, _size2130); + this->part_vals.resize(_size2130); + uint32_t _i2134; + for (_i2134 = 0; _i2134 < _size2130; ++_i2134) { - xfer += iprot->readString(this->part_vals[_i2128]); + xfer += iprot->readString(this->part_vals[_i2134]); } xfer += iprot->readListEnd(); } @@ -17000,10 +17000,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2129; - for (_iter2129 = this->part_vals.begin(); _iter2129 != this->part_vals.end(); ++_iter2129) + std::vector ::const_iterator _iter2135; + for (_iter2135 = this->part_vals.begin(); _iter2135 != this->part_vals.end(); ++_iter2135) { - xfer += oprot->writeString((*_iter2129)); + xfer += oprot->writeString((*_iter2135)); } xfer += oprot->writeListEnd(); } @@ -17035,10 +17035,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2130; - for (_iter2130 = (*(this->part_vals)).begin(); _iter2130 != (*(this->part_vals)).end(); ++_iter2130) + std::vector ::const_iterator _iter2136; + for (_iter2136 = (*(this->part_vals)).begin(); _iter2136 != (*(this->part_vals)).end(); ++_iter2136) { - xfer += oprot->writeString((*_iter2130)); + xfer += oprot->writeString((*_iter2136)); } xfer += oprot->writeListEnd(); } @@ -17510,14 +17510,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2131; - ::apache::thrift::protocol::TType _etype2134; - xfer += iprot->readListBegin(_etype2134, _size2131); - this->part_vals.resize(_size2131); - uint32_t _i2135; - for (_i2135 = 0; _i2135 < _size2131; ++_i2135) + uint32_t _size2137; + ::apache::thrift::protocol::TType _etype2140; + xfer += iprot->readListBegin(_etype2140, _size2137); + this->part_vals.resize(_size2137); + uint32_t _i2141; + for (_i2141 = 0; _i2141 < _size2137; ++_i2141) { - xfer += iprot->readString(this->part_vals[_i2135]); + xfer += iprot->readString(this->part_vals[_i2141]); } xfer += iprot->readListEnd(); } @@ -17562,10 +17562,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2136; - for (_iter2136 = this->part_vals.begin(); _iter2136 != this->part_vals.end(); ++_iter2136) + std::vector ::const_iterator _iter2142; + for (_iter2142 = this->part_vals.begin(); _iter2142 != this->part_vals.end(); ++_iter2142) { - xfer += oprot->writeString((*_iter2136)); + xfer += oprot->writeString((*_iter2142)); } xfer += oprot->writeListEnd(); } @@ -17601,10 +17601,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2137; - for (_iter2137 = (*(this->part_vals)).begin(); _iter2137 != (*(this->part_vals)).end(); ++_iter2137) + std::vector ::const_iterator _iter2143; + for (_iter2143 = (*(this->part_vals)).begin(); _iter2143 != (*(this->part_vals)).end(); ++_iter2143) { - xfer += oprot->writeString((*_iter2137)); + xfer += oprot->writeString((*_iter2143)); } xfer += oprot->writeListEnd(); } @@ -18407,14 +18407,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2138; - ::apache::thrift::protocol::TType _etype2141; - xfer += iprot->readListBegin(_etype2141, _size2138); - this->part_vals.resize(_size2138); - uint32_t _i2142; - for (_i2142 = 0; _i2142 < _size2138; ++_i2142) + uint32_t _size2144; + ::apache::thrift::protocol::TType _etype2147; + xfer += iprot->readListBegin(_etype2147, _size2144); + this->part_vals.resize(_size2144); + uint32_t _i2148; + for (_i2148 = 0; _i2148 < _size2144; ++_i2148) { - xfer += iprot->readString(this->part_vals[_i2142]); + xfer += iprot->readString(this->part_vals[_i2148]); } xfer += iprot->readListEnd(); } @@ -18459,10 +18459,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2143; - for (_iter2143 = this->part_vals.begin(); _iter2143 != this->part_vals.end(); ++_iter2143) + std::vector ::const_iterator _iter2149; + for (_iter2149 = this->part_vals.begin(); _iter2149 != this->part_vals.end(); ++_iter2149) { - xfer += oprot->writeString((*_iter2143)); + xfer += oprot->writeString((*_iter2149)); } xfer += oprot->writeListEnd(); } @@ -18498,10 +18498,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2144; - for (_iter2144 = (*(this->part_vals)).begin(); _iter2144 != (*(this->part_vals)).end(); ++_iter2144) + std::vector ::const_iterator _iter2150; + for (_iter2150 = (*(this->part_vals)).begin(); _iter2150 != (*(this->part_vals)).end(); ++_iter2150) { - xfer += oprot->writeString((*_iter2144)); + xfer += oprot->writeString((*_iter2150)); } xfer += oprot->writeListEnd(); } @@ -18710,14 +18710,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read( if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2145; - ::apache::thrift::protocol::TType _etype2148; - xfer += iprot->readListBegin(_etype2148, _size2145); - this->part_vals.resize(_size2145); - uint32_t _i2149; - for (_i2149 = 0; _i2149 < _size2145; ++_i2149) + uint32_t _size2151; + ::apache::thrift::protocol::TType _etype2154; + xfer += iprot->readListBegin(_etype2154, _size2151); + this->part_vals.resize(_size2151); + uint32_t _i2155; + for (_i2155 = 0; _i2155 < _size2151; ++_i2155) { - xfer += iprot->readString(this->part_vals[_i2149]); + xfer += iprot->readString(this->part_vals[_i2155]); } xfer += iprot->readListEnd(); } @@ -18770,10 +18770,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2150; - for (_iter2150 = this->part_vals.begin(); _iter2150 != this->part_vals.end(); ++_iter2150) + std::vector ::const_iterator _iter2156; + for (_iter2156 = this->part_vals.begin(); _iter2156 != this->part_vals.end(); ++_iter2156) { - xfer += oprot->writeString((*_iter2150)); + xfer += oprot->writeString((*_iter2156)); } xfer += oprot->writeListEnd(); } @@ -18813,10 +18813,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2151; - for (_iter2151 = (*(this->part_vals)).begin(); _iter2151 != (*(this->part_vals)).end(); ++_iter2151) + std::vector ::const_iterator _iter2157; + for (_iter2157 = (*(this->part_vals)).begin(); _iter2157 != (*(this->part_vals)).end(); ++_iter2157) { - xfer += oprot->writeString((*_iter2151)); + xfer += oprot->writeString((*_iter2157)); } xfer += oprot->writeListEnd(); } @@ -19822,14 +19822,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2152; - ::apache::thrift::protocol::TType _etype2155; - xfer += iprot->readListBegin(_etype2155, _size2152); - this->part_vals.resize(_size2152); - uint32_t _i2156; - for (_i2156 = 0; _i2156 < _size2152; ++_i2156) + uint32_t _size2158; + ::apache::thrift::protocol::TType _etype2161; + xfer += iprot->readListBegin(_etype2161, _size2158); + this->part_vals.resize(_size2158); + uint32_t _i2162; + for (_i2162 = 0; _i2162 < _size2158; ++_i2162) { - xfer += iprot->readString(this->part_vals[_i2156]); + xfer += iprot->readString(this->part_vals[_i2162]); } xfer += iprot->readListEnd(); } @@ -19866,10 +19866,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2157; - for (_iter2157 = this->part_vals.begin(); _iter2157 != this->part_vals.end(); ++_iter2157) + std::vector ::const_iterator _iter2163; + for (_iter2163 = this->part_vals.begin(); _iter2163 != this->part_vals.end(); ++_iter2163) { - xfer += oprot->writeString((*_iter2157)); + xfer += oprot->writeString((*_iter2163)); } xfer += oprot->writeListEnd(); } @@ -19901,10 +19901,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2158; - for (_iter2158 = (*(this->part_vals)).begin(); _iter2158 != (*(this->part_vals)).end(); ++_iter2158) + std::vector ::const_iterator _iter2164; + for (_iter2164 = (*(this->part_vals)).begin(); _iter2164 != (*(this->part_vals)).end(); ++_iter2164) { - xfer += oprot->writeString((*_iter2158)); + xfer += oprot->writeString((*_iter2164)); } xfer += oprot->writeListEnd(); } @@ -20320,17 +20320,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_MAP) { { this->partitionSpecs.clear(); - uint32_t _size2159; - ::apache::thrift::protocol::TType _ktype2160; - ::apache::thrift::protocol::TType _vtype2161; - xfer += iprot->readMapBegin(_ktype2160, _vtype2161, _size2159); - uint32_t _i2163; - for (_i2163 = 0; _i2163 < _size2159; ++_i2163) + uint32_t _size2165; + ::apache::thrift::protocol::TType _ktype2166; + ::apache::thrift::protocol::TType _vtype2167; + xfer += iprot->readMapBegin(_ktype2166, _vtype2167, _size2165); + uint32_t _i2169; + for (_i2169 = 0; _i2169 < _size2165; ++_i2169) { - std::string _key2164; - xfer += iprot->readString(_key2164); - std::string& _val2165 = this->partitionSpecs[_key2164]; - xfer += iprot->readString(_val2165); + std::string _key2170; + xfer += iprot->readString(_key2170); + std::string& _val2171 = this->partitionSpecs[_key2170]; + xfer += iprot->readString(_val2171); } xfer += iprot->readMapEnd(); } @@ -20391,11 +20391,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); - std::map ::const_iterator _iter2166; - for (_iter2166 = this->partitionSpecs.begin(); _iter2166 != this->partitionSpecs.end(); ++_iter2166) + std::map ::const_iterator _iter2172; + for (_iter2172 = this->partitionSpecs.begin(); _iter2172 != this->partitionSpecs.end(); ++_iter2172) { - xfer += oprot->writeString(_iter2166->first); - xfer += oprot->writeString(_iter2166->second); + xfer += oprot->writeString(_iter2172->first); + xfer += oprot->writeString(_iter2172->second); } xfer += oprot->writeMapEnd(); } @@ -20435,11 +20435,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); - std::map ::const_iterator _iter2167; - for (_iter2167 = (*(this->partitionSpecs)).begin(); _iter2167 != (*(this->partitionSpecs)).end(); ++_iter2167) + std::map ::const_iterator _iter2173; + for (_iter2173 = (*(this->partitionSpecs)).begin(); _iter2173 != (*(this->partitionSpecs)).end(); ++_iter2173) { - xfer += oprot->writeString(_iter2167->first); - xfer += oprot->writeString(_iter2167->second); + xfer += oprot->writeString(_iter2173->first); + xfer += oprot->writeString(_iter2173->second); } xfer += oprot->writeMapEnd(); } @@ -20684,17 +20684,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_MAP) { { this->partitionSpecs.clear(); - uint32_t _size2168; - ::apache::thrift::protocol::TType _ktype2169; - ::apache::thrift::protocol::TType _vtype2170; - xfer += iprot->readMapBegin(_ktype2169, _vtype2170, _size2168); - uint32_t _i2172; - for (_i2172 = 0; _i2172 < _size2168; ++_i2172) + uint32_t _size2174; + ::apache::thrift::protocol::TType _ktype2175; + ::apache::thrift::protocol::TType _vtype2176; + xfer += iprot->readMapBegin(_ktype2175, _vtype2176, _size2174); + uint32_t _i2178; + for (_i2178 = 0; _i2178 < _size2174; ++_i2178) { - std::string _key2173; - xfer += iprot->readString(_key2173); - std::string& _val2174 = this->partitionSpecs[_key2173]; - xfer += iprot->readString(_val2174); + std::string _key2179; + xfer += iprot->readString(_key2179); + std::string& _val2180 = this->partitionSpecs[_key2179]; + xfer += iprot->readString(_val2180); } xfer += iprot->readMapEnd(); } @@ -20755,11 +20755,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); - std::map ::const_iterator _iter2175; - for (_iter2175 = this->partitionSpecs.begin(); _iter2175 != this->partitionSpecs.end(); ++_iter2175) + std::map ::const_iterator _iter2181; + for (_iter2181 = this->partitionSpecs.begin(); _iter2181 != this->partitionSpecs.end(); ++_iter2181) { - xfer += oprot->writeString(_iter2175->first); - xfer += oprot->writeString(_iter2175->second); + xfer += oprot->writeString(_iter2181->first); + xfer += oprot->writeString(_iter2181->second); } xfer += oprot->writeMapEnd(); } @@ -20799,11 +20799,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift:: xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); - std::map ::const_iterator _iter2176; - for (_iter2176 = (*(this->partitionSpecs)).begin(); _iter2176 != (*(this->partitionSpecs)).end(); ++_iter2176) + std::map ::const_iterator _iter2182; + for (_iter2182 = (*(this->partitionSpecs)).begin(); _iter2182 != (*(this->partitionSpecs)).end(); ++_iter2182) { - xfer += oprot->writeString(_iter2176->first); - xfer += oprot->writeString(_iter2176->second); + xfer += oprot->writeString(_iter2182->first); + xfer += oprot->writeString(_iter2182->second); } xfer += oprot->writeMapEnd(); } @@ -20860,14 +20860,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2177; - ::apache::thrift::protocol::TType _etype2180; - xfer += iprot->readListBegin(_etype2180, _size2177); - this->success.resize(_size2177); - uint32_t _i2181; - for (_i2181 = 0; _i2181 < _size2177; ++_i2181) + uint32_t _size2183; + ::apache::thrift::protocol::TType _etype2186; + xfer += iprot->readListBegin(_etype2186, _size2183); + this->success.resize(_size2183); + uint32_t _i2187; + for (_i2187 = 0; _i2187 < _size2183; ++_i2187) { - xfer += this->success[_i2181].read(iprot); + xfer += this->success[_i2187].read(iprot); } xfer += iprot->readListEnd(); } @@ -20930,10 +20930,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2182; - for (_iter2182 = this->success.begin(); _iter2182 != this->success.end(); ++_iter2182) + std::vector ::const_iterator _iter2188; + for (_iter2188 = this->success.begin(); _iter2188 != this->success.end(); ++_iter2188) { - xfer += (*_iter2182).write(oprot); + xfer += (*_iter2188).write(oprot); } xfer += oprot->writeListEnd(); } @@ -20990,14 +20990,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2183; - ::apache::thrift::protocol::TType _etype2186; - xfer += iprot->readListBegin(_etype2186, _size2183); - (*(this->success)).resize(_size2183); - uint32_t _i2187; - for (_i2187 = 0; _i2187 < _size2183; ++_i2187) + uint32_t _size2189; + ::apache::thrift::protocol::TType _etype2192; + xfer += iprot->readListBegin(_etype2192, _size2189); + (*(this->success)).resize(_size2189); + uint32_t _i2193; + for (_i2193 = 0; _i2193 < _size2189; ++_i2193) { - xfer += (*(this->success))[_i2187].read(iprot); + xfer += (*(this->success))[_i2193].read(iprot); } xfer += iprot->readListEnd(); } @@ -21096,14 +21096,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2188; - ::apache::thrift::protocol::TType _etype2191; - xfer += iprot->readListBegin(_etype2191, _size2188); - this->part_vals.resize(_size2188); - uint32_t _i2192; - for (_i2192 = 0; _i2192 < _size2188; ++_i2192) + uint32_t _size2194; + ::apache::thrift::protocol::TType _etype2197; + xfer += iprot->readListBegin(_etype2197, _size2194); + this->part_vals.resize(_size2194); + uint32_t _i2198; + for (_i2198 = 0; _i2198 < _size2194; ++_i2198) { - xfer += iprot->readString(this->part_vals[_i2192]); + xfer += iprot->readString(this->part_vals[_i2198]); } xfer += iprot->readListEnd(); } @@ -21124,14 +21124,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2193; - ::apache::thrift::protocol::TType _etype2196; - xfer += iprot->readListBegin(_etype2196, _size2193); - this->group_names.resize(_size2193); - uint32_t _i2197; - for (_i2197 = 0; _i2197 < _size2193; ++_i2197) + uint32_t _size2199; + ::apache::thrift::protocol::TType _etype2202; + xfer += iprot->readListBegin(_etype2202, _size2199); + this->group_names.resize(_size2199); + uint32_t _i2203; + for (_i2203 = 0; _i2203 < _size2199; ++_i2203) { - xfer += iprot->readString(this->group_names[_i2197]); + xfer += iprot->readString(this->group_names[_i2203]); } xfer += iprot->readListEnd(); } @@ -21168,10 +21168,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2198; - for (_iter2198 = this->part_vals.begin(); _iter2198 != this->part_vals.end(); ++_iter2198) + std::vector ::const_iterator _iter2204; + for (_iter2204 = this->part_vals.begin(); _iter2204 != this->part_vals.end(); ++_iter2204) { - xfer += oprot->writeString((*_iter2198)); + xfer += oprot->writeString((*_iter2204)); } xfer += oprot->writeListEnd(); } @@ -21184,10 +21184,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2199; - for (_iter2199 = this->group_names.begin(); _iter2199 != this->group_names.end(); ++_iter2199) + std::vector ::const_iterator _iter2205; + for (_iter2205 = this->group_names.begin(); _iter2205 != this->group_names.end(); ++_iter2205) { - xfer += oprot->writeString((*_iter2199)); + xfer += oprot->writeString((*_iter2205)); } xfer += oprot->writeListEnd(); } @@ -21219,10 +21219,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2200; - for (_iter2200 = (*(this->part_vals)).begin(); _iter2200 != (*(this->part_vals)).end(); ++_iter2200) + std::vector ::const_iterator _iter2206; + for (_iter2206 = (*(this->part_vals)).begin(); _iter2206 != (*(this->part_vals)).end(); ++_iter2206) { - xfer += oprot->writeString((*_iter2200)); + xfer += oprot->writeString((*_iter2206)); } xfer += oprot->writeListEnd(); } @@ -21235,10 +21235,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2201; - for (_iter2201 = (*(this->group_names)).begin(); _iter2201 != (*(this->group_names)).end(); ++_iter2201) + std::vector ::const_iterator _iter2207; + for (_iter2207 = (*(this->group_names)).begin(); _iter2207 != (*(this->group_names)).end(); ++_iter2207) { - xfer += oprot->writeString((*_iter2201)); + xfer += oprot->writeString((*_iter2207)); } xfer += oprot->writeListEnd(); } @@ -21797,14 +21797,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2202; - ::apache::thrift::protocol::TType _etype2205; - xfer += iprot->readListBegin(_etype2205, _size2202); - this->success.resize(_size2202); - uint32_t _i2206; - for (_i2206 = 0; _i2206 < _size2202; ++_i2206) + uint32_t _size2208; + ::apache::thrift::protocol::TType _etype2211; + xfer += iprot->readListBegin(_etype2211, _size2208); + this->success.resize(_size2208); + uint32_t _i2212; + for (_i2212 = 0; _i2212 < _size2208; ++_i2212) { - xfer += this->success[_i2206].read(iprot); + xfer += this->success[_i2212].read(iprot); } xfer += iprot->readListEnd(); } @@ -21851,10 +21851,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2207; - for (_iter2207 = this->success.begin(); _iter2207 != this->success.end(); ++_iter2207) + std::vector ::const_iterator _iter2213; + for (_iter2213 = this->success.begin(); _iter2213 != this->success.end(); ++_iter2213) { - xfer += (*_iter2207).write(oprot); + xfer += (*_iter2213).write(oprot); } xfer += oprot->writeListEnd(); } @@ -21903,14 +21903,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2208; - ::apache::thrift::protocol::TType _etype2211; - xfer += iprot->readListBegin(_etype2211, _size2208); - (*(this->success)).resize(_size2208); - uint32_t _i2212; - for (_i2212 = 0; _i2212 < _size2208; ++_i2212) + uint32_t _size2214; + ::apache::thrift::protocol::TType _etype2217; + xfer += iprot->readListBegin(_etype2217, _size2214); + (*(this->success)).resize(_size2214); + uint32_t _i2218; + for (_i2218 = 0; _i2218 < _size2214; ++_i2218) { - xfer += (*(this->success))[_i2212].read(iprot); + xfer += (*(this->success))[_i2218].read(iprot); } xfer += iprot->readListEnd(); } @@ -22236,14 +22236,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2213; - ::apache::thrift::protocol::TType _etype2216; - xfer += iprot->readListBegin(_etype2216, _size2213); - this->group_names.resize(_size2213); - uint32_t _i2217; - for (_i2217 = 0; _i2217 < _size2213; ++_i2217) + uint32_t _size2219; + ::apache::thrift::protocol::TType _etype2222; + xfer += iprot->readListBegin(_etype2222, _size2219); + this->group_names.resize(_size2219); + uint32_t _i2223; + for (_i2223 = 0; _i2223 < _size2219; ++_i2223) { - xfer += iprot->readString(this->group_names[_i2217]); + xfer += iprot->readString(this->group_names[_i2223]); } xfer += iprot->readListEnd(); } @@ -22288,10 +22288,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2218; - for (_iter2218 = this->group_names.begin(); _iter2218 != this->group_names.end(); ++_iter2218) + std::vector ::const_iterator _iter2224; + for (_iter2224 = this->group_names.begin(); _iter2224 != this->group_names.end(); ++_iter2224) { - xfer += oprot->writeString((*_iter2218)); + xfer += oprot->writeString((*_iter2224)); } xfer += oprot->writeListEnd(); } @@ -22331,10 +22331,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2219; - for (_iter2219 = (*(this->group_names)).begin(); _iter2219 != (*(this->group_names)).end(); ++_iter2219) + std::vector ::const_iterator _iter2225; + for (_iter2225 = (*(this->group_names)).begin(); _iter2225 != (*(this->group_names)).end(); ++_iter2225) { - xfer += oprot->writeString((*_iter2219)); + xfer += oprot->writeString((*_iter2225)); } xfer += oprot->writeListEnd(); } @@ -22375,14 +22375,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2220; - ::apache::thrift::protocol::TType _etype2223; - xfer += iprot->readListBegin(_etype2223, _size2220); - this->success.resize(_size2220); - uint32_t _i2224; - for (_i2224 = 0; _i2224 < _size2220; ++_i2224) + uint32_t _size2226; + ::apache::thrift::protocol::TType _etype2229; + xfer += iprot->readListBegin(_etype2229, _size2226); + this->success.resize(_size2226); + uint32_t _i2230; + for (_i2230 = 0; _i2230 < _size2226; ++_i2230) { - xfer += this->success[_i2224].read(iprot); + xfer += this->success[_i2230].read(iprot); } xfer += iprot->readListEnd(); } @@ -22429,10 +22429,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2225; - for (_iter2225 = this->success.begin(); _iter2225 != this->success.end(); ++_iter2225) + std::vector ::const_iterator _iter2231; + for (_iter2231 = this->success.begin(); _iter2231 != this->success.end(); ++_iter2231) { - xfer += (*_iter2225).write(oprot); + xfer += (*_iter2231).write(oprot); } xfer += oprot->writeListEnd(); } @@ -22481,14 +22481,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2226; - ::apache::thrift::protocol::TType _etype2229; - xfer += iprot->readListBegin(_etype2229, _size2226); - (*(this->success)).resize(_size2226); - uint32_t _i2230; - for (_i2230 = 0; _i2230 < _size2226; ++_i2230) + uint32_t _size2232; + ::apache::thrift::protocol::TType _etype2235; + xfer += iprot->readListBegin(_etype2235, _size2232); + (*(this->success)).resize(_size2232); + uint32_t _i2236; + for (_i2236 = 0; _i2236 < _size2232; ++_i2236) { - xfer += (*(this->success))[_i2230].read(iprot); + xfer += (*(this->success))[_i2236].read(iprot); } xfer += iprot->readListEnd(); } @@ -22666,14 +22666,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2231; - ::apache::thrift::protocol::TType _etype2234; - xfer += iprot->readListBegin(_etype2234, _size2231); - this->success.resize(_size2231); - uint32_t _i2235; - for (_i2235 = 0; _i2235 < _size2231; ++_i2235) + uint32_t _size2237; + ::apache::thrift::protocol::TType _etype2240; + xfer += iprot->readListBegin(_etype2240, _size2237); + this->success.resize(_size2237); + uint32_t _i2241; + for (_i2241 = 0; _i2241 < _size2237; ++_i2241) { - xfer += this->success[_i2235].read(iprot); + xfer += this->success[_i2241].read(iprot); } xfer += iprot->readListEnd(); } @@ -22720,10 +22720,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2236; - for (_iter2236 = this->success.begin(); _iter2236 != this->success.end(); ++_iter2236) + std::vector ::const_iterator _iter2242; + for (_iter2242 = this->success.begin(); _iter2242 != this->success.end(); ++_iter2242) { - xfer += (*_iter2236).write(oprot); + xfer += (*_iter2242).write(oprot); } xfer += oprot->writeListEnd(); } @@ -22772,14 +22772,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2237; - ::apache::thrift::protocol::TType _etype2240; - xfer += iprot->readListBegin(_etype2240, _size2237); - (*(this->success)).resize(_size2237); - uint32_t _i2241; - for (_i2241 = 0; _i2241 < _size2237; ++_i2241) + uint32_t _size2243; + ::apache::thrift::protocol::TType _etype2246; + xfer += iprot->readListBegin(_etype2246, _size2243); + (*(this->success)).resize(_size2243); + uint32_t _i2247; + for (_i2247 = 0; _i2247 < _size2243; ++_i2247) { - xfer += (*(this->success))[_i2241].read(iprot); + xfer += (*(this->success))[_i2247].read(iprot); } xfer += iprot->readListEnd(); } @@ -22957,14 +22957,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2242; - ::apache::thrift::protocol::TType _etype2245; - xfer += iprot->readListBegin(_etype2245, _size2242); - this->success.resize(_size2242); - uint32_t _i2246; - for (_i2246 = 0; _i2246 < _size2242; ++_i2246) + uint32_t _size2248; + ::apache::thrift::protocol::TType _etype2251; + xfer += iprot->readListBegin(_etype2251, _size2248); + this->success.resize(_size2248); + uint32_t _i2252; + for (_i2252 = 0; _i2252 < _size2248; ++_i2252) { - xfer += iprot->readString(this->success[_i2246]); + xfer += iprot->readString(this->success[_i2252]); } xfer += iprot->readListEnd(); } @@ -23011,10 +23011,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2247; - for (_iter2247 = this->success.begin(); _iter2247 != this->success.end(); ++_iter2247) + std::vector ::const_iterator _iter2253; + for (_iter2253 = this->success.begin(); _iter2253 != this->success.end(); ++_iter2253) { - xfer += oprot->writeString((*_iter2247)); + xfer += oprot->writeString((*_iter2253)); } xfer += oprot->writeListEnd(); } @@ -23063,14 +23063,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2248; - ::apache::thrift::protocol::TType _etype2251; - xfer += iprot->readListBegin(_etype2251, _size2248); - (*(this->success)).resize(_size2248); - uint32_t _i2252; - for (_i2252 = 0; _i2252 < _size2248; ++_i2252) + uint32_t _size2254; + ::apache::thrift::protocol::TType _etype2257; + xfer += iprot->readListBegin(_etype2257, _size2254); + (*(this->success)).resize(_size2254); + uint32_t _i2258; + for (_i2258 = 0; _i2258 < _size2254; ++_i2258) { - xfer += iprot->readString((*(this->success))[_i2252]); + xfer += iprot->readString((*(this->success))[_i2258]); } xfer += iprot->readListEnd(); } @@ -23380,14 +23380,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2253; - ::apache::thrift::protocol::TType _etype2256; - xfer += iprot->readListBegin(_etype2256, _size2253); - this->part_vals.resize(_size2253); - uint32_t _i2257; - for (_i2257 = 0; _i2257 < _size2253; ++_i2257) + uint32_t _size2259; + ::apache::thrift::protocol::TType _etype2262; + xfer += iprot->readListBegin(_etype2262, _size2259); + this->part_vals.resize(_size2259); + uint32_t _i2263; + for (_i2263 = 0; _i2263 < _size2259; ++_i2263) { - xfer += iprot->readString(this->part_vals[_i2257]); + xfer += iprot->readString(this->part_vals[_i2263]); } xfer += iprot->readListEnd(); } @@ -23432,10 +23432,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2258; - for (_iter2258 = this->part_vals.begin(); _iter2258 != this->part_vals.end(); ++_iter2258) + std::vector ::const_iterator _iter2264; + for (_iter2264 = this->part_vals.begin(); _iter2264 != this->part_vals.end(); ++_iter2264) { - xfer += oprot->writeString((*_iter2258)); + xfer += oprot->writeString((*_iter2264)); } xfer += oprot->writeListEnd(); } @@ -23471,10 +23471,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2259; - for (_iter2259 = (*(this->part_vals)).begin(); _iter2259 != (*(this->part_vals)).end(); ++_iter2259) + std::vector ::const_iterator _iter2265; + for (_iter2265 = (*(this->part_vals)).begin(); _iter2265 != (*(this->part_vals)).end(); ++_iter2265) { - xfer += oprot->writeString((*_iter2259)); + xfer += oprot->writeString((*_iter2265)); } xfer += oprot->writeListEnd(); } @@ -23519,14 +23519,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2260; - ::apache::thrift::protocol::TType _etype2263; - xfer += iprot->readListBegin(_etype2263, _size2260); - this->success.resize(_size2260); - uint32_t _i2264; - for (_i2264 = 0; _i2264 < _size2260; ++_i2264) + uint32_t _size2266; + ::apache::thrift::protocol::TType _etype2269; + xfer += iprot->readListBegin(_etype2269, _size2266); + this->success.resize(_size2266); + uint32_t _i2270; + for (_i2270 = 0; _i2270 < _size2266; ++_i2270) { - xfer += this->success[_i2264].read(iprot); + xfer += this->success[_i2270].read(iprot); } xfer += iprot->readListEnd(); } @@ -23573,10 +23573,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2265; - for (_iter2265 = this->success.begin(); _iter2265 != this->success.end(); ++_iter2265) + std::vector ::const_iterator _iter2271; + for (_iter2271 = this->success.begin(); _iter2271 != this->success.end(); ++_iter2271) { - xfer += (*_iter2265).write(oprot); + xfer += (*_iter2271).write(oprot); } xfer += oprot->writeListEnd(); } @@ -23625,14 +23625,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2266; - ::apache::thrift::protocol::TType _etype2269; - xfer += iprot->readListBegin(_etype2269, _size2266); - (*(this->success)).resize(_size2266); - uint32_t _i2270; - for (_i2270 = 0; _i2270 < _size2266; ++_i2270) + uint32_t _size2272; + ::apache::thrift::protocol::TType _etype2275; + xfer += iprot->readListBegin(_etype2275, _size2272); + (*(this->success)).resize(_size2272); + uint32_t _i2276; + for (_i2276 = 0; _i2276 < _size2272; ++_i2276) { - xfer += (*(this->success))[_i2270].read(iprot); + xfer += (*(this->success))[_i2276].read(iprot); } xfer += iprot->readListEnd(); } @@ -23715,14 +23715,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2271; - ::apache::thrift::protocol::TType _etype2274; - xfer += iprot->readListBegin(_etype2274, _size2271); - this->part_vals.resize(_size2271); - uint32_t _i2275; - for (_i2275 = 0; _i2275 < _size2271; ++_i2275) + uint32_t _size2277; + ::apache::thrift::protocol::TType _etype2280; + xfer += iprot->readListBegin(_etype2280, _size2277); + this->part_vals.resize(_size2277); + uint32_t _i2281; + for (_i2281 = 0; _i2281 < _size2277; ++_i2281) { - xfer += iprot->readString(this->part_vals[_i2275]); + xfer += iprot->readString(this->part_vals[_i2281]); } xfer += iprot->readListEnd(); } @@ -23751,14 +23751,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2276; - ::apache::thrift::protocol::TType _etype2279; - xfer += iprot->readListBegin(_etype2279, _size2276); - this->group_names.resize(_size2276); - uint32_t _i2280; - for (_i2280 = 0; _i2280 < _size2276; ++_i2280) + uint32_t _size2282; + ::apache::thrift::protocol::TType _etype2285; + xfer += iprot->readListBegin(_etype2285, _size2282); + this->group_names.resize(_size2282); + uint32_t _i2286; + for (_i2286 = 0; _i2286 < _size2282; ++_i2286) { - xfer += iprot->readString(this->group_names[_i2280]); + xfer += iprot->readString(this->group_names[_i2286]); } xfer += iprot->readListEnd(); } @@ -23795,10 +23795,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2281; - for (_iter2281 = this->part_vals.begin(); _iter2281 != this->part_vals.end(); ++_iter2281) + std::vector ::const_iterator _iter2287; + for (_iter2287 = this->part_vals.begin(); _iter2287 != this->part_vals.end(); ++_iter2287) { - xfer += oprot->writeString((*_iter2281)); + xfer += oprot->writeString((*_iter2287)); } xfer += oprot->writeListEnd(); } @@ -23815,10 +23815,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2282; - for (_iter2282 = this->group_names.begin(); _iter2282 != this->group_names.end(); ++_iter2282) + std::vector ::const_iterator _iter2288; + for (_iter2288 = this->group_names.begin(); _iter2288 != this->group_names.end(); ++_iter2288) { - xfer += oprot->writeString((*_iter2282)); + xfer += oprot->writeString((*_iter2288)); } xfer += oprot->writeListEnd(); } @@ -23850,10 +23850,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache:: xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2283; - for (_iter2283 = (*(this->part_vals)).begin(); _iter2283 != (*(this->part_vals)).end(); ++_iter2283) + std::vector ::const_iterator _iter2289; + for (_iter2289 = (*(this->part_vals)).begin(); _iter2289 != (*(this->part_vals)).end(); ++_iter2289) { - xfer += oprot->writeString((*_iter2283)); + xfer += oprot->writeString((*_iter2289)); } xfer += oprot->writeListEnd(); } @@ -23870,10 +23870,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache:: xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2284; - for (_iter2284 = (*(this->group_names)).begin(); _iter2284 != (*(this->group_names)).end(); ++_iter2284) + std::vector ::const_iterator _iter2290; + for (_iter2290 = (*(this->group_names)).begin(); _iter2290 != (*(this->group_names)).end(); ++_iter2290) { - xfer += oprot->writeString((*_iter2284)); + xfer += oprot->writeString((*_iter2290)); } xfer += oprot->writeListEnd(); } @@ -23914,14 +23914,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2285; - ::apache::thrift::protocol::TType _etype2288; - xfer += iprot->readListBegin(_etype2288, _size2285); - this->success.resize(_size2285); - uint32_t _i2289; - for (_i2289 = 0; _i2289 < _size2285; ++_i2289) + uint32_t _size2291; + ::apache::thrift::protocol::TType _etype2294; + xfer += iprot->readListBegin(_etype2294, _size2291); + this->success.resize(_size2291); + uint32_t _i2295; + for (_i2295 = 0; _i2295 < _size2291; ++_i2295) { - xfer += this->success[_i2289].read(iprot); + xfer += this->success[_i2295].read(iprot); } xfer += iprot->readListEnd(); } @@ -23968,10 +23968,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache: xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2290; - for (_iter2290 = this->success.begin(); _iter2290 != this->success.end(); ++_iter2290) + std::vector ::const_iterator _iter2296; + for (_iter2296 = this->success.begin(); _iter2296 != this->success.end(); ++_iter2296) { - xfer += (*_iter2290).write(oprot); + xfer += (*_iter2296).write(oprot); } xfer += oprot->writeListEnd(); } @@ -24020,14 +24020,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2291; - ::apache::thrift::protocol::TType _etype2294; - xfer += iprot->readListBegin(_etype2294, _size2291); - (*(this->success)).resize(_size2291); - uint32_t _i2295; - for (_i2295 = 0; _i2295 < _size2291; ++_i2295) + uint32_t _size2297; + ::apache::thrift::protocol::TType _etype2300; + xfer += iprot->readListBegin(_etype2300, _size2297); + (*(this->success)).resize(_size2297); + uint32_t _i2301; + for (_i2301 = 0; _i2301 < _size2297; ++_i2301) { - xfer += (*(this->success))[_i2295].read(iprot); + xfer += (*(this->success))[_i2301].read(iprot); } xfer += iprot->readListEnd(); } @@ -24337,14 +24337,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2296; - ::apache::thrift::protocol::TType _etype2299; - xfer += iprot->readListBegin(_etype2299, _size2296); - this->part_vals.resize(_size2296); - uint32_t _i2300; - for (_i2300 = 0; _i2300 < _size2296; ++_i2300) + uint32_t _size2302; + ::apache::thrift::protocol::TType _etype2305; + xfer += iprot->readListBegin(_etype2305, _size2302); + this->part_vals.resize(_size2302); + uint32_t _i2306; + for (_i2306 = 0; _i2306 < _size2302; ++_i2306) { - xfer += iprot->readString(this->part_vals[_i2300]); + xfer += iprot->readString(this->part_vals[_i2306]); } xfer += iprot->readListEnd(); } @@ -24389,10 +24389,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2301; - for (_iter2301 = this->part_vals.begin(); _iter2301 != this->part_vals.end(); ++_iter2301) + std::vector ::const_iterator _iter2307; + for (_iter2307 = this->part_vals.begin(); _iter2307 != this->part_vals.end(); ++_iter2307) { - xfer += oprot->writeString((*_iter2301)); + xfer += oprot->writeString((*_iter2307)); } xfer += oprot->writeListEnd(); } @@ -24428,10 +24428,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrif xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2302; - for (_iter2302 = (*(this->part_vals)).begin(); _iter2302 != (*(this->part_vals)).end(); ++_iter2302) + std::vector ::const_iterator _iter2308; + for (_iter2308 = (*(this->part_vals)).begin(); _iter2308 != (*(this->part_vals)).end(); ++_iter2308) { - xfer += oprot->writeString((*_iter2302)); + xfer += oprot->writeString((*_iter2308)); } xfer += oprot->writeListEnd(); } @@ -24476,14 +24476,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2303; - ::apache::thrift::protocol::TType _etype2306; - xfer += iprot->readListBegin(_etype2306, _size2303); - this->success.resize(_size2303); - uint32_t _i2307; - for (_i2307 = 0; _i2307 < _size2303; ++_i2307) + uint32_t _size2309; + ::apache::thrift::protocol::TType _etype2312; + xfer += iprot->readListBegin(_etype2312, _size2309); + this->success.resize(_size2309); + uint32_t _i2313; + for (_i2313 = 0; _i2313 < _size2309; ++_i2313) { - xfer += iprot->readString(this->success[_i2307]); + xfer += iprot->readString(this->success[_i2313]); } xfer += iprot->readListEnd(); } @@ -24530,10 +24530,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2308; - for (_iter2308 = this->success.begin(); _iter2308 != this->success.end(); ++_iter2308) + std::vector ::const_iterator _iter2314; + for (_iter2314 = this->success.begin(); _iter2314 != this->success.end(); ++_iter2314) { - xfer += oprot->writeString((*_iter2308)); + xfer += oprot->writeString((*_iter2314)); } xfer += oprot->writeListEnd(); } @@ -24582,14 +24582,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2309; - ::apache::thrift::protocol::TType _etype2312; - xfer += iprot->readListBegin(_etype2312, _size2309); - (*(this->success)).resize(_size2309); - uint32_t _i2313; - for (_i2313 = 0; _i2313 < _size2309; ++_i2313) + uint32_t _size2315; + ::apache::thrift::protocol::TType _etype2318; + xfer += iprot->readListBegin(_etype2318, _size2315); + (*(this->success)).resize(_size2315); + uint32_t _i2319; + for (_i2319 = 0; _i2319 < _size2315; ++_i2319) { - xfer += iprot->readString((*(this->success))[_i2313]); + xfer += iprot->readString((*(this->success))[_i2319]); } xfer += iprot->readListEnd(); } @@ -24962,14 +24962,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2314; - ::apache::thrift::protocol::TType _etype2317; - xfer += iprot->readListBegin(_etype2317, _size2314); - this->success.resize(_size2314); - uint32_t _i2318; - for (_i2318 = 0; _i2318 < _size2314; ++_i2318) + uint32_t _size2320; + ::apache::thrift::protocol::TType _etype2323; + xfer += iprot->readListBegin(_etype2323, _size2320); + this->success.resize(_size2320); + uint32_t _i2324; + for (_i2324 = 0; _i2324 < _size2320; ++_i2324) { - xfer += iprot->readString(this->success[_i2318]); + xfer += iprot->readString(this->success[_i2324]); } xfer += iprot->readListEnd(); } @@ -25016,10 +25016,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2319; - for (_iter2319 = this->success.begin(); _iter2319 != this->success.end(); ++_iter2319) + std::vector ::const_iterator _iter2325; + for (_iter2325 = this->success.begin(); _iter2325 != this->success.end(); ++_iter2325) { - xfer += oprot->writeString((*_iter2319)); + xfer += oprot->writeString((*_iter2325)); } xfer += oprot->writeListEnd(); } @@ -25068,14 +25068,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2320; - ::apache::thrift::protocol::TType _etype2323; - xfer += iprot->readListBegin(_etype2323, _size2320); - (*(this->success)).resize(_size2320); - uint32_t _i2324; - for (_i2324 = 0; _i2324 < _size2320; ++_i2324) + uint32_t _size2326; + ::apache::thrift::protocol::TType _etype2329; + xfer += iprot->readListBegin(_etype2329, _size2326); + (*(this->success)).resize(_size2326); + uint32_t _i2330; + for (_i2330 = 0; _i2330 < _size2326; ++_i2330) { - xfer += iprot->readString((*(this->success))[_i2324]); + xfer += iprot->readString((*(this->success))[_i2330]); } xfer += iprot->readListEnd(); } @@ -25269,14 +25269,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2325; - ::apache::thrift::protocol::TType _etype2328; - xfer += iprot->readListBegin(_etype2328, _size2325); - this->success.resize(_size2325); - uint32_t _i2329; - for (_i2329 = 0; _i2329 < _size2325; ++_i2329) + uint32_t _size2331; + ::apache::thrift::protocol::TType _etype2334; + xfer += iprot->readListBegin(_etype2334, _size2331); + this->success.resize(_size2331); + uint32_t _i2335; + for (_i2335 = 0; _i2335 < _size2331; ++_i2335) { - xfer += this->success[_i2329].read(iprot); + xfer += this->success[_i2335].read(iprot); } xfer += iprot->readListEnd(); } @@ -25323,10 +25323,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2330; - for (_iter2330 = this->success.begin(); _iter2330 != this->success.end(); ++_iter2330) + std::vector ::const_iterator _iter2336; + for (_iter2336 = this->success.begin(); _iter2336 != this->success.end(); ++_iter2336) { - xfer += (*_iter2330).write(oprot); + xfer += (*_iter2336).write(oprot); } xfer += oprot->writeListEnd(); } @@ -25375,14 +25375,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2331; - ::apache::thrift::protocol::TType _etype2334; - xfer += iprot->readListBegin(_etype2334, _size2331); - (*(this->success)).resize(_size2331); - uint32_t _i2335; - for (_i2335 = 0; _i2335 < _size2331; ++_i2335) + uint32_t _size2337; + ::apache::thrift::protocol::TType _etype2340; + xfer += iprot->readListBegin(_etype2340, _size2337); + (*(this->success)).resize(_size2337); + uint32_t _i2341; + for (_i2341 = 0; _i2341 < _size2337; ++_i2341) { - xfer += (*(this->success))[_i2335].read(iprot); + xfer += (*(this->success))[_i2341].read(iprot); } xfer += iprot->readListEnd(); } @@ -25528,14 +25528,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_result::read(::apache: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2336; - ::apache::thrift::protocol::TType _etype2339; - xfer += iprot->readListBegin(_etype2339, _size2336); - this->success.resize(_size2336); - uint32_t _i2340; - for (_i2340 = 0; _i2340 < _size2336; ++_i2340) + uint32_t _size2342; + ::apache::thrift::protocol::TType _etype2345; + xfer += iprot->readListBegin(_etype2345, _size2342); + this->success.resize(_size2342); + uint32_t _i2346; + for (_i2346 = 0; _i2346 < _size2342; ++_i2346) { - xfer += this->success[_i2340].read(iprot); + xfer += this->success[_i2346].read(iprot); } xfer += iprot->readListEnd(); } @@ -25582,10 +25582,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_result::write(::apache xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2341; - for (_iter2341 = this->success.begin(); _iter2341 != this->success.end(); ++_iter2341) + std::vector ::const_iterator _iter2347; + for (_iter2347 = this->success.begin(); _iter2347 != this->success.end(); ++_iter2347) { - xfer += (*_iter2341).write(oprot); + xfer += (*_iter2347).write(oprot); } xfer += oprot->writeListEnd(); } @@ -25634,14 +25634,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_presult::read(::apache if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2342; - ::apache::thrift::protocol::TType _etype2345; - xfer += iprot->readListBegin(_etype2345, _size2342); - (*(this->success)).resize(_size2342); - uint32_t _i2346; - for (_i2346 = 0; _i2346 < _size2342; ++_i2346) + uint32_t _size2348; + ::apache::thrift::protocol::TType _etype2351; + xfer += iprot->readListBegin(_etype2351, _size2348); + (*(this->success)).resize(_size2348); + uint32_t _i2352; + for (_i2352 = 0; _i2352 < _size2348; ++_i2352) { - xfer += (*(this->success))[_i2346].read(iprot); + xfer += (*(this->success))[_i2352].read(iprot); } xfer += iprot->readListEnd(); } @@ -25835,14 +25835,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2347; - ::apache::thrift::protocol::TType _etype2350; - xfer += iprot->readListBegin(_etype2350, _size2347); - this->success.resize(_size2347); - uint32_t _i2351; - for (_i2351 = 0; _i2351 < _size2347; ++_i2351) + uint32_t _size2353; + ::apache::thrift::protocol::TType _etype2356; + xfer += iprot->readListBegin(_etype2356, _size2353); + this->success.resize(_size2353); + uint32_t _i2357; + for (_i2357 = 0; _i2357 < _size2353; ++_i2357) { - xfer += this->success[_i2351].read(iprot); + xfer += this->success[_i2357].read(iprot); } xfer += iprot->readListEnd(); } @@ -25889,10 +25889,10 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2352; - for (_iter2352 = this->success.begin(); _iter2352 != this->success.end(); ++_iter2352) + std::vector ::const_iterator _iter2358; + for (_iter2358 = this->success.begin(); _iter2358 != this->success.end(); ++_iter2358) { - xfer += (*_iter2352).write(oprot); + xfer += (*_iter2358).write(oprot); } xfer += oprot->writeListEnd(); } @@ -25941,14 +25941,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2353; - ::apache::thrift::protocol::TType _etype2356; - xfer += iprot->readListBegin(_etype2356, _size2353); - (*(this->success)).resize(_size2353); - uint32_t _i2357; - for (_i2357 = 0; _i2357 < _size2353; ++_i2357) + uint32_t _size2359; + ::apache::thrift::protocol::TType _etype2362; + xfer += iprot->readListBegin(_etype2362, _size2359); + (*(this->success)).resize(_size2359); + uint32_t _i2363; + for (_i2363 = 0; _i2363 < _size2359; ++_i2363) { - xfer += (*(this->success))[_i2357].read(iprot); + xfer += (*(this->success))[_i2363].read(iprot); } xfer += iprot->readListEnd(); } @@ -26744,14 +26744,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->names.clear(); - uint32_t _size2358; - ::apache::thrift::protocol::TType _etype2361; - xfer += iprot->readListBegin(_etype2361, _size2358); - this->names.resize(_size2358); - uint32_t _i2362; - for (_i2362 = 0; _i2362 < _size2358; ++_i2362) + uint32_t _size2364; + ::apache::thrift::protocol::TType _etype2367; + xfer += iprot->readListBegin(_etype2367, _size2364); + this->names.resize(_size2364); + uint32_t _i2368; + for (_i2368 = 0; _i2368 < _size2364; ++_i2368) { - xfer += iprot->readString(this->names[_i2362]); + xfer += iprot->readString(this->names[_i2368]); } xfer += iprot->readListEnd(); } @@ -26788,10 +26788,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrif xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); - std::vector ::const_iterator _iter2363; - for (_iter2363 = this->names.begin(); _iter2363 != this->names.end(); ++_iter2363) + std::vector ::const_iterator _iter2369; + for (_iter2369 = this->names.begin(); _iter2369 != this->names.end(); ++_iter2369) { - xfer += oprot->writeString((*_iter2363)); + xfer += oprot->writeString((*_iter2369)); } xfer += oprot->writeListEnd(); } @@ -26823,10 +26823,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thri xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->names)).size())); - std::vector ::const_iterator _iter2364; - for (_iter2364 = (*(this->names)).begin(); _iter2364 != (*(this->names)).end(); ++_iter2364) + std::vector ::const_iterator _iter2370; + for (_iter2370 = (*(this->names)).begin(); _iter2370 != (*(this->names)).end(); ++_iter2370) { - xfer += oprot->writeString((*_iter2364)); + xfer += oprot->writeString((*_iter2370)); } xfer += oprot->writeListEnd(); } @@ -26867,14 +26867,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2365; - ::apache::thrift::protocol::TType _etype2368; - xfer += iprot->readListBegin(_etype2368, _size2365); - this->success.resize(_size2365); - uint32_t _i2369; - for (_i2369 = 0; _i2369 < _size2365; ++_i2369) + uint32_t _size2371; + ::apache::thrift::protocol::TType _etype2374; + xfer += iprot->readListBegin(_etype2374, _size2371); + this->success.resize(_size2371); + uint32_t _i2375; + for (_i2375 = 0; _i2375 < _size2371; ++_i2375) { - xfer += this->success[_i2369].read(iprot); + xfer += this->success[_i2375].read(iprot); } xfer += iprot->readListEnd(); } @@ -26929,10 +26929,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2370; - for (_iter2370 = this->success.begin(); _iter2370 != this->success.end(); ++_iter2370) + std::vector ::const_iterator _iter2376; + for (_iter2376 = this->success.begin(); _iter2376 != this->success.end(); ++_iter2376) { - xfer += (*_iter2370).write(oprot); + xfer += (*_iter2376).write(oprot); } xfer += oprot->writeListEnd(); } @@ -26985,14 +26985,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2371; - ::apache::thrift::protocol::TType _etype2374; - xfer += iprot->readListBegin(_etype2374, _size2371); - (*(this->success)).resize(_size2371); - uint32_t _i2375; - for (_i2375 = 0; _i2375 < _size2371; ++_i2375) + uint32_t _size2377; + ::apache::thrift::protocol::TType _etype2380; + xfer += iprot->readListBegin(_etype2380, _size2377); + (*(this->success)).resize(_size2377); + uint32_t _i2381; + for (_i2381 = 0; _i2381 < _size2377; ++_i2381) { - xfer += (*(this->success))[_i2375].read(iprot); + xfer += (*(this->success))[_i2381].read(iprot); } xfer += iprot->readListEnd(); } @@ -28023,14 +28023,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2376; - ::apache::thrift::protocol::TType _etype2379; - xfer += iprot->readListBegin(_etype2379, _size2376); - this->new_parts.resize(_size2376); - uint32_t _i2380; - for (_i2380 = 0; _i2380 < _size2376; ++_i2380) + uint32_t _size2382; + ::apache::thrift::protocol::TType _etype2385; + xfer += iprot->readListBegin(_etype2385, _size2382); + this->new_parts.resize(_size2382); + uint32_t _i2386; + for (_i2386 = 0; _i2386 < _size2382; ++_i2386) { - xfer += this->new_parts[_i2380].read(iprot); + xfer += this->new_parts[_i2386].read(iprot); } xfer += iprot->readListEnd(); } @@ -28067,10 +28067,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2381; - for (_iter2381 = this->new_parts.begin(); _iter2381 != this->new_parts.end(); ++_iter2381) + std::vector ::const_iterator _iter2387; + for (_iter2387 = this->new_parts.begin(); _iter2387 != this->new_parts.end(); ++_iter2387) { - xfer += (*_iter2381).write(oprot); + xfer += (*_iter2387).write(oprot); } xfer += oprot->writeListEnd(); } @@ -28102,10 +28102,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2382; - for (_iter2382 = (*(this->new_parts)).begin(); _iter2382 != (*(this->new_parts)).end(); ++_iter2382) + std::vector ::const_iterator _iter2388; + for (_iter2388 = (*(this->new_parts)).begin(); _iter2388 != (*(this->new_parts)).end(); ++_iter2388) { - xfer += (*_iter2382).write(oprot); + xfer += (*_iter2388).write(oprot); } xfer += oprot->writeListEnd(); } @@ -28290,14 +28290,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::rea if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size2383; - ::apache::thrift::protocol::TType _etype2386; - xfer += iprot->readListBegin(_etype2386, _size2383); - this->new_parts.resize(_size2383); - uint32_t _i2387; - for (_i2387 = 0; _i2387 < _size2383; ++_i2387) + uint32_t _size2389; + ::apache::thrift::protocol::TType _etype2392; + xfer += iprot->readListBegin(_etype2392, _size2389); + this->new_parts.resize(_size2389); + uint32_t _i2393; + for (_i2393 = 0; _i2393 < _size2389; ++_i2393) { - xfer += this->new_parts[_i2387].read(iprot); + xfer += this->new_parts[_i2393].read(iprot); } xfer += iprot->readListEnd(); } @@ -28342,10 +28342,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::wri xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter2388; - for (_iter2388 = this->new_parts.begin(); _iter2388 != this->new_parts.end(); ++_iter2388) + std::vector ::const_iterator _iter2394; + for (_iter2394 = this->new_parts.begin(); _iter2394 != this->new_parts.end(); ++_iter2394) { - xfer += (*_iter2388).write(oprot); + xfer += (*_iter2394).write(oprot); } xfer += oprot->writeListEnd(); } @@ -28381,10 +28381,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::wr xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter2389; - for (_iter2389 = (*(this->new_parts)).begin(); _iter2389 != (*(this->new_parts)).end(); ++_iter2389) + std::vector ::const_iterator _iter2395; + for (_iter2395 = (*(this->new_parts)).begin(); _iter2395 != (*(this->new_parts)).end(); ++_iter2395) { - xfer += (*_iter2389).write(oprot); + xfer += (*_iter2395).write(oprot); } xfer += oprot->writeListEnd(); } @@ -29055,14 +29055,14 @@ uint32_t ThriftHiveMetastore_rename_partition_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2390; - ::apache::thrift::protocol::TType _etype2393; - xfer += iprot->readListBegin(_etype2393, _size2390); - this->part_vals.resize(_size2390); - uint32_t _i2394; - for (_i2394 = 0; _i2394 < _size2390; ++_i2394) + uint32_t _size2396; + ::apache::thrift::protocol::TType _etype2399; + xfer += iprot->readListBegin(_etype2399, _size2396); + this->part_vals.resize(_size2396); + uint32_t _i2400; + for (_i2400 = 0; _i2400 < _size2396; ++_i2400) { - xfer += iprot->readString(this->part_vals[_i2394]); + xfer += iprot->readString(this->part_vals[_i2400]); } xfer += iprot->readListEnd(); } @@ -29107,10 +29107,10 @@ uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2395; - for (_iter2395 = this->part_vals.begin(); _iter2395 != this->part_vals.end(); ++_iter2395) + std::vector ::const_iterator _iter2401; + for (_iter2401 = this->part_vals.begin(); _iter2401 != this->part_vals.end(); ++_iter2401) { - xfer += oprot->writeString((*_iter2395)); + xfer += oprot->writeString((*_iter2401)); } xfer += oprot->writeListEnd(); } @@ -29146,10 +29146,10 @@ uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2396; - for (_iter2396 = (*(this->part_vals)).begin(); _iter2396 != (*(this->part_vals)).end(); ++_iter2396) + std::vector ::const_iterator _iter2402; + for (_iter2402 = (*(this->part_vals)).begin(); _iter2402 != (*(this->part_vals)).end(); ++_iter2402) { - xfer += oprot->writeString((*_iter2396)); + xfer += oprot->writeString((*_iter2402)); } xfer += oprot->writeListEnd(); } @@ -29549,14 +29549,14 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::read(::ap if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size2397; - ::apache::thrift::protocol::TType _etype2400; - xfer += iprot->readListBegin(_etype2400, _size2397); - this->part_vals.resize(_size2397); - uint32_t _i2401; - for (_i2401 = 0; _i2401 < _size2397; ++_i2401) + uint32_t _size2403; + ::apache::thrift::protocol::TType _etype2406; + xfer += iprot->readListBegin(_etype2406, _size2403); + this->part_vals.resize(_size2403); + uint32_t _i2407; + for (_i2407 = 0; _i2407 < _size2403; ++_i2407) { - xfer += iprot->readString(this->part_vals[_i2401]); + xfer += iprot->readString(this->part_vals[_i2407]); } xfer += iprot->readListEnd(); } @@ -29593,10 +29593,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::a xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter2402; - for (_iter2402 = this->part_vals.begin(); _iter2402 != this->part_vals.end(); ++_iter2402) + std::vector ::const_iterator _iter2408; + for (_iter2408 = this->part_vals.begin(); _iter2408 != this->part_vals.end(); ++_iter2408) { - xfer += oprot->writeString((*_iter2402)); + xfer += oprot->writeString((*_iter2408)); } xfer += oprot->writeListEnd(); } @@ -29624,10 +29624,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(:: xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter2403; - for (_iter2403 = (*(this->part_vals)).begin(); _iter2403 != (*(this->part_vals)).end(); ++_iter2403) + std::vector ::const_iterator _iter2409; + for (_iter2409 = (*(this->part_vals)).begin(); _iter2409 != (*(this->part_vals)).end(); ++_iter2409) { - xfer += oprot->writeString((*_iter2403)); + xfer += oprot->writeString((*_iter2409)); } xfer += oprot->writeListEnd(); } @@ -30102,14 +30102,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2404; - ::apache::thrift::protocol::TType _etype2407; - xfer += iprot->readListBegin(_etype2407, _size2404); - this->success.resize(_size2404); - uint32_t _i2408; - for (_i2408 = 0; _i2408 < _size2404; ++_i2408) + uint32_t _size2410; + ::apache::thrift::protocol::TType _etype2413; + xfer += iprot->readListBegin(_etype2413, _size2410); + this->success.resize(_size2410); + uint32_t _i2414; + for (_i2414 = 0; _i2414 < _size2410; ++_i2414) { - xfer += iprot->readString(this->success[_i2408]); + xfer += iprot->readString(this->success[_i2414]); } xfer += iprot->readListEnd(); } @@ -30148,10 +30148,10 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thri xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2409; - for (_iter2409 = this->success.begin(); _iter2409 != this->success.end(); ++_iter2409) + std::vector ::const_iterator _iter2415; + for (_iter2415 = this->success.begin(); _iter2415 != this->success.end(); ++_iter2415) { - xfer += oprot->writeString((*_iter2409)); + xfer += oprot->writeString((*_iter2415)); } xfer += oprot->writeListEnd(); } @@ -30196,14 +30196,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2410; - ::apache::thrift::protocol::TType _etype2413; - xfer += iprot->readListBegin(_etype2413, _size2410); - (*(this->success)).resize(_size2410); - uint32_t _i2414; - for (_i2414 = 0; _i2414 < _size2410; ++_i2414) + uint32_t _size2416; + ::apache::thrift::protocol::TType _etype2419; + xfer += iprot->readListBegin(_etype2419, _size2416); + (*(this->success)).resize(_size2416); + uint32_t _i2420; + for (_i2420 = 0; _i2420 < _size2416; ++_i2420) { - xfer += iprot->readString((*(this->success))[_i2414]); + xfer += iprot->readString((*(this->success))[_i2420]); } xfer += iprot->readListEnd(); } @@ -30341,17 +30341,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_MAP) { { this->success.clear(); - uint32_t _size2415; - ::apache::thrift::protocol::TType _ktype2416; - ::apache::thrift::protocol::TType _vtype2417; - xfer += iprot->readMapBegin(_ktype2416, _vtype2417, _size2415); - uint32_t _i2419; - for (_i2419 = 0; _i2419 < _size2415; ++_i2419) + uint32_t _size2421; + ::apache::thrift::protocol::TType _ktype2422; + ::apache::thrift::protocol::TType _vtype2423; + xfer += iprot->readMapBegin(_ktype2422, _vtype2423, _size2421); + uint32_t _i2425; + for (_i2425 = 0; _i2425 < _size2421; ++_i2425) { - std::string _key2420; - xfer += iprot->readString(_key2420); - std::string& _val2421 = this->success[_key2420]; - xfer += iprot->readString(_val2421); + std::string _key2426; + xfer += iprot->readString(_key2426); + std::string& _val2427 = this->success[_key2426]; + xfer += iprot->readString(_val2427); } xfer += iprot->readMapEnd(); } @@ -30390,11 +30390,11 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thri xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::map ::const_iterator _iter2422; - for (_iter2422 = this->success.begin(); _iter2422 != this->success.end(); ++_iter2422) + std::map ::const_iterator _iter2428; + for (_iter2428 = this->success.begin(); _iter2428 != this->success.end(); ++_iter2428) { - xfer += oprot->writeString(_iter2422->first); - xfer += oprot->writeString(_iter2422->second); + xfer += oprot->writeString(_iter2428->first); + xfer += oprot->writeString(_iter2428->second); } xfer += oprot->writeMapEnd(); } @@ -30439,17 +30439,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_MAP) { { (*(this->success)).clear(); - uint32_t _size2423; - ::apache::thrift::protocol::TType _ktype2424; - ::apache::thrift::protocol::TType _vtype2425; - xfer += iprot->readMapBegin(_ktype2424, _vtype2425, _size2423); - uint32_t _i2427; - for (_i2427 = 0; _i2427 < _size2423; ++_i2427) + uint32_t _size2429; + ::apache::thrift::protocol::TType _ktype2430; + ::apache::thrift::protocol::TType _vtype2431; + xfer += iprot->readMapBegin(_ktype2430, _vtype2431, _size2429); + uint32_t _i2433; + for (_i2433 = 0; _i2433 < _size2429; ++_i2433) { - std::string _key2428; - xfer += iprot->readString(_key2428); - std::string& _val2429 = (*(this->success))[_key2428]; - xfer += iprot->readString(_val2429); + std::string _key2434; + xfer += iprot->readString(_key2434); + std::string& _val2435 = (*(this->success))[_key2434]; + xfer += iprot->readString(_val2435); } xfer += iprot->readMapEnd(); } @@ -30524,17 +30524,17 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_MAP) { { this->part_vals.clear(); - uint32_t _size2430; - ::apache::thrift::protocol::TType _ktype2431; - ::apache::thrift::protocol::TType _vtype2432; - xfer += iprot->readMapBegin(_ktype2431, _vtype2432, _size2430); - uint32_t _i2434; - for (_i2434 = 0; _i2434 < _size2430; ++_i2434) + uint32_t _size2436; + ::apache::thrift::protocol::TType _ktype2437; + ::apache::thrift::protocol::TType _vtype2438; + xfer += iprot->readMapBegin(_ktype2437, _vtype2438, _size2436); + uint32_t _i2440; + for (_i2440 = 0; _i2440 < _size2436; ++_i2440) { - std::string _key2435; - xfer += iprot->readString(_key2435); - std::string& _val2436 = this->part_vals[_key2435]; - xfer += iprot->readString(_val2436); + std::string _key2441; + xfer += iprot->readString(_key2441); + std::string& _val2442 = this->part_vals[_key2441]; + xfer += iprot->readString(_val2442); } xfer += iprot->readMapEnd(); } @@ -30545,9 +30545,9 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift:: break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2437; - xfer += iprot->readI32(ecast2437); - this->eventType = static_cast(ecast2437); + int32_t ecast2443; + xfer += iprot->readI32(ecast2443); + this->eventType = static_cast(ecast2443); this->__isset.eventType = true; } else { xfer += iprot->skip(ftype); @@ -30581,11 +30581,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift: xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::map ::const_iterator _iter2438; - for (_iter2438 = this->part_vals.begin(); _iter2438 != this->part_vals.end(); ++_iter2438) + std::map ::const_iterator _iter2444; + for (_iter2444 = this->part_vals.begin(); _iter2444 != this->part_vals.end(); ++_iter2444) { - xfer += oprot->writeString(_iter2438->first); - xfer += oprot->writeString(_iter2438->second); + xfer += oprot->writeString(_iter2444->first); + xfer += oprot->writeString(_iter2444->second); } xfer += oprot->writeMapEnd(); } @@ -30621,11 +30621,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::map ::const_iterator _iter2439; - for (_iter2439 = (*(this->part_vals)).begin(); _iter2439 != (*(this->part_vals)).end(); ++_iter2439) + std::map ::const_iterator _iter2445; + for (_iter2445 = (*(this->part_vals)).begin(); _iter2445 != (*(this->part_vals)).end(); ++_iter2445) { - xfer += oprot->writeString(_iter2439->first); - xfer += oprot->writeString(_iter2439->second); + xfer += oprot->writeString(_iter2445->first); + xfer += oprot->writeString(_iter2445->second); } xfer += oprot->writeMapEnd(); } @@ -30894,17 +30894,17 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_MAP) { { this->part_vals.clear(); - uint32_t _size2440; - ::apache::thrift::protocol::TType _ktype2441; - ::apache::thrift::protocol::TType _vtype2442; - xfer += iprot->readMapBegin(_ktype2441, _vtype2442, _size2440); - uint32_t _i2444; - for (_i2444 = 0; _i2444 < _size2440; ++_i2444) + uint32_t _size2446; + ::apache::thrift::protocol::TType _ktype2447; + ::apache::thrift::protocol::TType _vtype2448; + xfer += iprot->readMapBegin(_ktype2447, _vtype2448, _size2446); + uint32_t _i2450; + for (_i2450 = 0; _i2450 < _size2446; ++_i2450) { - std::string _key2445; - xfer += iprot->readString(_key2445); - std::string& _val2446 = this->part_vals[_key2445]; - xfer += iprot->readString(_val2446); + std::string _key2451; + xfer += iprot->readString(_key2451); + std::string& _val2452 = this->part_vals[_key2451]; + xfer += iprot->readString(_val2452); } xfer += iprot->readMapEnd(); } @@ -30915,9 +30915,9 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2447; - xfer += iprot->readI32(ecast2447); - this->eventType = static_cast(ecast2447); + int32_t ecast2453; + xfer += iprot->readI32(ecast2453); + this->eventType = static_cast(ecast2453); this->__isset.eventType = true; } else { xfer += iprot->skip(ftype); @@ -30951,11 +30951,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thr xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::map ::const_iterator _iter2448; - for (_iter2448 = this->part_vals.begin(); _iter2448 != this->part_vals.end(); ++_iter2448) + std::map ::const_iterator _iter2454; + for (_iter2454 = this->part_vals.begin(); _iter2454 != this->part_vals.end(); ++_iter2454) { - xfer += oprot->writeString(_iter2448->first); - xfer += oprot->writeString(_iter2448->second); + xfer += oprot->writeString(_iter2454->first); + xfer += oprot->writeString(_iter2454->second); } xfer += oprot->writeMapEnd(); } @@ -30991,11 +30991,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::th xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::map ::const_iterator _iter2449; - for (_iter2449 = (*(this->part_vals)).begin(); _iter2449 != (*(this->part_vals)).end(); ++_iter2449) + std::map ::const_iterator _iter2455; + for (_iter2455 = (*(this->part_vals)).begin(); _iter2455 != (*(this->part_vals)).end(); ++_iter2455) { - xfer += oprot->writeString(_iter2449->first); - xfer += oprot->writeString(_iter2449->second); + xfer += oprot->writeString(_iter2455->first); + xfer += oprot->writeString(_iter2455->second); } xfer += oprot->writeMapEnd(); } @@ -37124,14 +37124,14 @@ uint32_t ThriftHiveMetastore_get_functions_result::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2450; - ::apache::thrift::protocol::TType _etype2453; - xfer += iprot->readListBegin(_etype2453, _size2450); - this->success.resize(_size2450); - uint32_t _i2454; - for (_i2454 = 0; _i2454 < _size2450; ++_i2454) + uint32_t _size2456; + ::apache::thrift::protocol::TType _etype2459; + xfer += iprot->readListBegin(_etype2459, _size2456); + this->success.resize(_size2456); + uint32_t _i2460; + for (_i2460 = 0; _i2460 < _size2456; ++_i2460) { - xfer += iprot->readString(this->success[_i2454]); + xfer += iprot->readString(this->success[_i2460]); } xfer += iprot->readListEnd(); } @@ -37170,10 +37170,10 @@ uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::proto xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2455; - for (_iter2455 = this->success.begin(); _iter2455 != this->success.end(); ++_iter2455) + std::vector ::const_iterator _iter2461; + for (_iter2461 = this->success.begin(); _iter2461 != this->success.end(); ++_iter2461) { - xfer += oprot->writeString((*_iter2455)); + xfer += oprot->writeString((*_iter2461)); } xfer += oprot->writeListEnd(); } @@ -37218,14 +37218,14 @@ uint32_t ThriftHiveMetastore_get_functions_presult::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2456; - ::apache::thrift::protocol::TType _etype2459; - xfer += iprot->readListBegin(_etype2459, _size2456); - (*(this->success)).resize(_size2456); - uint32_t _i2460; - for (_i2460 = 0; _i2460 < _size2456; ++_i2460) + uint32_t _size2462; + ::apache::thrift::protocol::TType _etype2465; + xfer += iprot->readListBegin(_etype2465, _size2462); + (*(this->success)).resize(_size2462); + uint32_t _i2466; + for (_i2466 = 0; _i2466 < _size2462; ++_i2466) { - xfer += iprot->readString((*(this->success))[_i2460]); + xfer += iprot->readString((*(this->success))[_i2466]); } xfer += iprot->readListEnd(); } @@ -38185,14 +38185,14 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2461; - ::apache::thrift::protocol::TType _etype2464; - xfer += iprot->readListBegin(_etype2464, _size2461); - this->success.resize(_size2461); - uint32_t _i2465; - for (_i2465 = 0; _i2465 < _size2461; ++_i2465) + uint32_t _size2467; + ::apache::thrift::protocol::TType _etype2470; + xfer += iprot->readListBegin(_etype2470, _size2467); + this->success.resize(_size2467); + uint32_t _i2471; + for (_i2471 = 0; _i2471 < _size2467; ++_i2471) { - xfer += iprot->readString(this->success[_i2465]); + xfer += iprot->readString(this->success[_i2471]); } xfer += iprot->readListEnd(); } @@ -38231,10 +38231,10 @@ uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::prot xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2466; - for (_iter2466 = this->success.begin(); _iter2466 != this->success.end(); ++_iter2466) + std::vector ::const_iterator _iter2472; + for (_iter2472 = this->success.begin(); _iter2472 != this->success.end(); ++_iter2472) { - xfer += oprot->writeString((*_iter2466)); + xfer += oprot->writeString((*_iter2472)); } xfer += oprot->writeListEnd(); } @@ -38279,14 +38279,14 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2467; - ::apache::thrift::protocol::TType _etype2470; - xfer += iprot->readListBegin(_etype2470, _size2467); - (*(this->success)).resize(_size2467); - uint32_t _i2471; - for (_i2471 = 0; _i2471 < _size2467; ++_i2471) + uint32_t _size2473; + ::apache::thrift::protocol::TType _etype2476; + xfer += iprot->readListBegin(_etype2476, _size2473); + (*(this->success)).resize(_size2473); + uint32_t _i2477; + for (_i2477 = 0; _i2477 < _size2473; ++_i2477) { - xfer += iprot->readString((*(this->success))[_i2471]); + xfer += iprot->readString((*(this->success))[_i2477]); } xfer += iprot->readListEnd(); } @@ -38359,9 +38359,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2472; - xfer += iprot->readI32(ecast2472); - this->principal_type = static_cast(ecast2472); + int32_t ecast2478; + xfer += iprot->readI32(ecast2478); + this->principal_type = static_cast(ecast2478); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -38377,9 +38377,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T break; case 5: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2473; - xfer += iprot->readI32(ecast2473); - this->grantorType = static_cast(ecast2473); + int32_t ecast2479; + xfer += iprot->readI32(ecast2479); + this->grantorType = static_cast(ecast2479); this->__isset.grantorType = true; } else { xfer += iprot->skip(ftype); @@ -38650,9 +38650,9 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol:: break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2474; - xfer += iprot->readI32(ecast2474); - this->principal_type = static_cast(ecast2474); + int32_t ecast2480; + xfer += iprot->readI32(ecast2480); + this->principal_type = static_cast(ecast2480); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -38883,9 +38883,9 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2475; - xfer += iprot->readI32(ecast2475); - this->principal_type = static_cast(ecast2475); + int32_t ecast2481; + xfer += iprot->readI32(ecast2481); + this->principal_type = static_cast(ecast2481); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -38974,14 +38974,14 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2476; - ::apache::thrift::protocol::TType _etype2479; - xfer += iprot->readListBegin(_etype2479, _size2476); - this->success.resize(_size2476); - uint32_t _i2480; - for (_i2480 = 0; _i2480 < _size2476; ++_i2480) + uint32_t _size2482; + ::apache::thrift::protocol::TType _etype2485; + xfer += iprot->readListBegin(_etype2485, _size2482); + this->success.resize(_size2482); + uint32_t _i2486; + for (_i2486 = 0; _i2486 < _size2482; ++_i2486) { - xfer += this->success[_i2480].read(iprot); + xfer += this->success[_i2486].read(iprot); } xfer += iprot->readListEnd(); } @@ -39020,10 +39020,10 @@ uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2481; - for (_iter2481 = this->success.begin(); _iter2481 != this->success.end(); ++_iter2481) + std::vector ::const_iterator _iter2487; + for (_iter2487 = this->success.begin(); _iter2487 != this->success.end(); ++_iter2487) { - xfer += (*_iter2481).write(oprot); + xfer += (*_iter2487).write(oprot); } xfer += oprot->writeListEnd(); } @@ -39068,14 +39068,14 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2482; - ::apache::thrift::protocol::TType _etype2485; - xfer += iprot->readListBegin(_etype2485, _size2482); - (*(this->success)).resize(_size2482); - uint32_t _i2486; - for (_i2486 = 0; _i2486 < _size2482; ++_i2486) + uint32_t _size2488; + ::apache::thrift::protocol::TType _etype2491; + xfer += iprot->readListBegin(_etype2491, _size2488); + (*(this->success)).resize(_size2488); + uint32_t _i2492; + for (_i2492 = 0; _i2492 < _size2488; ++_i2492) { - xfer += (*(this->success))[_i2486].read(iprot); + xfer += (*(this->success))[_i2492].read(iprot); } xfer += iprot->readListEnd(); } @@ -39771,14 +39771,14 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2487; - ::apache::thrift::protocol::TType _etype2490; - xfer += iprot->readListBegin(_etype2490, _size2487); - this->group_names.resize(_size2487); - uint32_t _i2491; - for (_i2491 = 0; _i2491 < _size2487; ++_i2491) + uint32_t _size2493; + ::apache::thrift::protocol::TType _etype2496; + xfer += iprot->readListBegin(_etype2496, _size2493); + this->group_names.resize(_size2493); + uint32_t _i2497; + for (_i2497 = 0; _i2497 < _size2493; ++_i2497) { - xfer += iprot->readString(this->group_names[_i2491]); + xfer += iprot->readString(this->group_names[_i2497]); } xfer += iprot->readListEnd(); } @@ -39815,10 +39815,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2492; - for (_iter2492 = this->group_names.begin(); _iter2492 != this->group_names.end(); ++_iter2492) + std::vector ::const_iterator _iter2498; + for (_iter2498 = this->group_names.begin(); _iter2498 != this->group_names.end(); ++_iter2498) { - xfer += oprot->writeString((*_iter2492)); + xfer += oprot->writeString((*_iter2498)); } xfer += oprot->writeListEnd(); } @@ -39850,10 +39850,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2493; - for (_iter2493 = (*(this->group_names)).begin(); _iter2493 != (*(this->group_names)).end(); ++_iter2493) + std::vector ::const_iterator _iter2499; + for (_iter2499 = (*(this->group_names)).begin(); _iter2499 != (*(this->group_names)).end(); ++_iter2499) { - xfer += oprot->writeString((*_iter2493)); + xfer += oprot->writeString((*_iter2499)); } xfer += oprot->writeListEnd(); } @@ -40028,9 +40028,9 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast2494; - xfer += iprot->readI32(ecast2494); - this->principal_type = static_cast(ecast2494); + int32_t ecast2500; + xfer += iprot->readI32(ecast2500); + this->principal_type = static_cast(ecast2500); this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -40135,14 +40135,14 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2495; - ::apache::thrift::protocol::TType _etype2498; - xfer += iprot->readListBegin(_etype2498, _size2495); - this->success.resize(_size2495); - uint32_t _i2499; - for (_i2499 = 0; _i2499 < _size2495; ++_i2499) + uint32_t _size2501; + ::apache::thrift::protocol::TType _etype2504; + xfer += iprot->readListBegin(_etype2504, _size2501); + this->success.resize(_size2501); + uint32_t _i2505; + for (_i2505 = 0; _i2505 < _size2501; ++_i2505) { - xfer += this->success[_i2499].read(iprot); + xfer += this->success[_i2505].read(iprot); } xfer += iprot->readListEnd(); } @@ -40181,10 +40181,10 @@ uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2500; - for (_iter2500 = this->success.begin(); _iter2500 != this->success.end(); ++_iter2500) + std::vector ::const_iterator _iter2506; + for (_iter2506 = this->success.begin(); _iter2506 != this->success.end(); ++_iter2506) { - xfer += (*_iter2500).write(oprot); + xfer += (*_iter2506).write(oprot); } xfer += oprot->writeListEnd(); } @@ -40229,14 +40229,14 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2501; - ::apache::thrift::protocol::TType _etype2504; - xfer += iprot->readListBegin(_etype2504, _size2501); - (*(this->success)).resize(_size2501); - uint32_t _i2505; - for (_i2505 = 0; _i2505 < _size2501; ++_i2505) + uint32_t _size2507; + ::apache::thrift::protocol::TType _etype2510; + xfer += iprot->readListBegin(_etype2510, _size2507); + (*(this->success)).resize(_size2507); + uint32_t _i2511; + for (_i2511 = 0; _i2511 < _size2507; ++_i2511) { - xfer += (*(this->success))[_i2505].read(iprot); + xfer += (*(this->success))[_i2511].read(iprot); } xfer += iprot->readListEnd(); } @@ -41163,14 +41163,14 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size2506; - ::apache::thrift::protocol::TType _etype2509; - xfer += iprot->readListBegin(_etype2509, _size2506); - this->group_names.resize(_size2506); - uint32_t _i2510; - for (_i2510 = 0; _i2510 < _size2506; ++_i2510) + uint32_t _size2512; + ::apache::thrift::protocol::TType _etype2515; + xfer += iprot->readListBegin(_etype2515, _size2512); + this->group_names.resize(_size2512); + uint32_t _i2516; + for (_i2516 = 0; _i2516 < _size2512; ++_i2516) { - xfer += iprot->readString(this->group_names[_i2510]); + xfer += iprot->readString(this->group_names[_i2516]); } xfer += iprot->readListEnd(); } @@ -41203,10 +41203,10 @@ uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TPr xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter2511; - for (_iter2511 = this->group_names.begin(); _iter2511 != this->group_names.end(); ++_iter2511) + std::vector ::const_iterator _iter2517; + for (_iter2517 = this->group_names.begin(); _iter2517 != this->group_names.end(); ++_iter2517) { - xfer += oprot->writeString((*_iter2511)); + xfer += oprot->writeString((*_iter2517)); } xfer += oprot->writeListEnd(); } @@ -41234,10 +41234,10 @@ uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TP xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter2512; - for (_iter2512 = (*(this->group_names)).begin(); _iter2512 != (*(this->group_names)).end(); ++_iter2512) + std::vector ::const_iterator _iter2518; + for (_iter2518 = (*(this->group_names)).begin(); _iter2518 != (*(this->group_names)).end(); ++_iter2518) { - xfer += oprot->writeString((*_iter2512)); + xfer += oprot->writeString((*_iter2518)); } xfer += oprot->writeListEnd(); } @@ -41278,14 +41278,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2513; - ::apache::thrift::protocol::TType _etype2516; - xfer += iprot->readListBegin(_etype2516, _size2513); - this->success.resize(_size2513); - uint32_t _i2517; - for (_i2517 = 0; _i2517 < _size2513; ++_i2517) + uint32_t _size2519; + ::apache::thrift::protocol::TType _etype2522; + xfer += iprot->readListBegin(_etype2522, _size2519); + this->success.resize(_size2519); + uint32_t _i2523; + for (_i2523 = 0; _i2523 < _size2519; ++_i2523) { - xfer += iprot->readString(this->success[_i2517]); + xfer += iprot->readString(this->success[_i2523]); } xfer += iprot->readListEnd(); } @@ -41324,10 +41324,10 @@ uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::T xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2518; - for (_iter2518 = this->success.begin(); _iter2518 != this->success.end(); ++_iter2518) + std::vector ::const_iterator _iter2524; + for (_iter2524 = this->success.begin(); _iter2524 != this->success.end(); ++_iter2524) { - xfer += oprot->writeString((*_iter2518)); + xfer += oprot->writeString((*_iter2524)); } xfer += oprot->writeListEnd(); } @@ -41372,14 +41372,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2519; - ::apache::thrift::protocol::TType _etype2522; - xfer += iprot->readListBegin(_etype2522, _size2519); - (*(this->success)).resize(_size2519); - uint32_t _i2523; - for (_i2523 = 0; _i2523 < _size2519; ++_i2523) + uint32_t _size2525; + ::apache::thrift::protocol::TType _etype2528; + xfer += iprot->readListBegin(_etype2528, _size2525); + (*(this->success)).resize(_size2525); + uint32_t _i2529; + for (_i2529 = 0; _i2529 < _size2525; ++_i2529) { - xfer += iprot->readString((*(this->success))[_i2523]); + xfer += iprot->readString((*(this->success))[_i2529]); } xfer += iprot->readListEnd(); } @@ -42690,14 +42690,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2524; - ::apache::thrift::protocol::TType _etype2527; - xfer += iprot->readListBegin(_etype2527, _size2524); - this->success.resize(_size2524); - uint32_t _i2528; - for (_i2528 = 0; _i2528 < _size2524; ++_i2528) + uint32_t _size2530; + ::apache::thrift::protocol::TType _etype2533; + xfer += iprot->readListBegin(_etype2533, _size2530); + this->success.resize(_size2530); + uint32_t _i2534; + for (_i2534 = 0; _i2534 < _size2530; ++_i2534) { - xfer += iprot->readString(this->success[_i2528]); + xfer += iprot->readString(this->success[_i2534]); } xfer += iprot->readListEnd(); } @@ -42728,10 +42728,10 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2529; - for (_iter2529 = this->success.begin(); _iter2529 != this->success.end(); ++_iter2529) + std::vector ::const_iterator _iter2535; + for (_iter2535 = this->success.begin(); _iter2535 != this->success.end(); ++_iter2535) { - xfer += oprot->writeString((*_iter2529)); + xfer += oprot->writeString((*_iter2535)); } xfer += oprot->writeListEnd(); } @@ -42772,14 +42772,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2530; - ::apache::thrift::protocol::TType _etype2533; - xfer += iprot->readListBegin(_etype2533, _size2530); - (*(this->success)).resize(_size2530); - uint32_t _i2534; - for (_i2534 = 0; _i2534 < _size2530; ++_i2534) + uint32_t _size2536; + ::apache::thrift::protocol::TType _etype2539; + xfer += iprot->readListBegin(_etype2539, _size2536); + (*(this->success)).resize(_size2536); + uint32_t _i2540; + for (_i2540 = 0; _i2540 < _size2536; ++_i2540) { - xfer += iprot->readString((*(this->success))[_i2534]); + xfer += iprot->readString((*(this->success))[_i2540]); } xfer += iprot->readListEnd(); } @@ -43505,14 +43505,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2535; - ::apache::thrift::protocol::TType _etype2538; - xfer += iprot->readListBegin(_etype2538, _size2535); - this->success.resize(_size2535); - uint32_t _i2539; - for (_i2539 = 0; _i2539 < _size2535; ++_i2539) + uint32_t _size2541; + ::apache::thrift::protocol::TType _etype2544; + xfer += iprot->readListBegin(_etype2544, _size2541); + this->success.resize(_size2541); + uint32_t _i2545; + for (_i2545 = 0; _i2545 < _size2541; ++_i2545) { - xfer += iprot->readString(this->success[_i2539]); + xfer += iprot->readString(this->success[_i2545]); } xfer += iprot->readListEnd(); } @@ -43543,10 +43543,10 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::write(::apache::thrift::pro xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2540; - for (_iter2540 = this->success.begin(); _iter2540 != this->success.end(); ++_iter2540) + std::vector ::const_iterator _iter2546; + for (_iter2546 = this->success.begin(); _iter2546 != this->success.end(); ++_iter2546) { - xfer += oprot->writeString((*_iter2540)); + xfer += oprot->writeString((*_iter2546)); } xfer += oprot->writeListEnd(); } @@ -43587,14 +43587,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_presult::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2541; - ::apache::thrift::protocol::TType _etype2544; - xfer += iprot->readListBegin(_etype2544, _size2541); - (*(this->success)).resize(_size2541); - uint32_t _i2545; - for (_i2545 = 0; _i2545 < _size2541; ++_i2545) + uint32_t _size2547; + ::apache::thrift::protocol::TType _etype2550; + xfer += iprot->readListBegin(_etype2550, _size2547); + (*(this->success)).resize(_size2547); + uint32_t _i2551; + for (_i2551 = 0; _i2551 < _size2547; ++_i2551) { - xfer += iprot->readString((*(this->success))[_i2545]); + xfer += iprot->readString((*(this->success))[_i2551]); } xfer += iprot->readListEnd(); } @@ -45343,17 +45343,17 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::read(::apache::t if (ftype == ::apache::thrift::protocol::T_MAP) { { this->writeIds.clear(); - uint32_t _size2546; - ::apache::thrift::protocol::TType _ktype2547; - ::apache::thrift::protocol::TType _vtype2548; - xfer += iprot->readMapBegin(_ktype2547, _vtype2548, _size2546); - uint32_t _i2550; - for (_i2550 = 0; _i2550 < _size2546; ++_i2550) + uint32_t _size2552; + ::apache::thrift::protocol::TType _ktype2553; + ::apache::thrift::protocol::TType _vtype2554; + xfer += iprot->readMapBegin(_ktype2553, _vtype2554, _size2552); + uint32_t _i2556; + for (_i2556 = 0; _i2556 < _size2552; ++_i2556) { - std::string _key2551; - xfer += iprot->readString(_key2551); - int64_t& _val2552 = this->writeIds[_key2551]; - xfer += iprot->readI64(_val2552); + std::string _key2557; + xfer += iprot->readString(_key2557); + int64_t& _val2558 = this->writeIds[_key2557]; + xfer += iprot->readI64(_val2558); } xfer += iprot->readMapEnd(); } @@ -45386,11 +45386,11 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::write(::apache:: xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast(this->writeIds.size())); - std::map ::const_iterator _iter2553; - for (_iter2553 = this->writeIds.begin(); _iter2553 != this->writeIds.end(); ++_iter2553) + std::map ::const_iterator _iter2559; + for (_iter2559 = this->writeIds.begin(); _iter2559 != this->writeIds.end(); ++_iter2559) { - xfer += oprot->writeString(_iter2553->first); - xfer += oprot->writeI64(_iter2553->second); + xfer += oprot->writeString(_iter2559->first); + xfer += oprot->writeI64(_iter2559->second); } xfer += oprot->writeMapEnd(); } @@ -45418,11 +45418,11 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_pargs::write(::apache: xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2); { xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast((*(this->writeIds)).size())); - std::map ::const_iterator _iter2554; - for (_iter2554 = (*(this->writeIds)).begin(); _iter2554 != (*(this->writeIds)).end(); ++_iter2554) + std::map ::const_iterator _iter2560; + for (_iter2560 = (*(this->writeIds)).begin(); _iter2560 != (*(this->writeIds)).end(); ++_iter2560) { - xfer += oprot->writeString(_iter2554->first); - xfer += oprot->writeI64(_iter2554->second); + xfer += oprot->writeString(_iter2560->first); + xfer += oprot->writeI64(_iter2560->second); } xfer += oprot->writeMapEnd(); } @@ -49322,14 +49322,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2555; - ::apache::thrift::protocol::TType _etype2558; - xfer += iprot->readListBegin(_etype2558, _size2555); - this->success.resize(_size2555); - uint32_t _i2559; - for (_i2559 = 0; _i2559 < _size2555; ++_i2559) + uint32_t _size2561; + ::apache::thrift::protocol::TType _etype2564; + xfer += iprot->readListBegin(_etype2564, _size2561); + this->success.resize(_size2561); + uint32_t _i2565; + for (_i2565 = 0; _i2565 < _size2561; ++_i2565) { - xfer += iprot->readString(this->success[_i2559]); + xfer += iprot->readString(this->success[_i2565]); } xfer += iprot->readListEnd(); } @@ -49360,10 +49360,10 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2560; - for (_iter2560 = this->success.begin(); _iter2560 != this->success.end(); ++_iter2560) + std::vector ::const_iterator _iter2566; + for (_iter2566 = this->success.begin(); _iter2566 != this->success.end(); ++_iter2566) { - xfer += oprot->writeString((*_iter2560)); + xfer += oprot->writeString((*_iter2566)); } xfer += oprot->writeListEnd(); } @@ -49404,14 +49404,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2561; - ::apache::thrift::protocol::TType _etype2564; - xfer += iprot->readListBegin(_etype2564, _size2561); - (*(this->success)).resize(_size2561); - uint32_t _i2565; - for (_i2565 = 0; _i2565 < _size2561; ++_i2565) + uint32_t _size2567; + ::apache::thrift::protocol::TType _etype2570; + xfer += iprot->readListBegin(_etype2570, _size2567); + (*(this->success)).resize(_size2567); + uint32_t _i2571; + for (_i2571 = 0; _i2571 < _size2567; ++_i2571) { - xfer += iprot->readString((*(this->success))[_i2565]); + xfer += iprot->readString((*(this->success))[_i2571]); } xfer += iprot->readListEnd(); } @@ -59334,14 +59334,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2566; - ::apache::thrift::protocol::TType _etype2569; - xfer += iprot->readListBegin(_etype2569, _size2566); - this->success.resize(_size2566); - uint32_t _i2570; - for (_i2570 = 0; _i2570 < _size2566; ++_i2570) + uint32_t _size2572; + ::apache::thrift::protocol::TType _etype2575; + xfer += iprot->readListBegin(_etype2575, _size2572); + this->success.resize(_size2572); + uint32_t _i2576; + for (_i2576 = 0; _i2576 < _size2572; ++_i2576) { - xfer += this->success[_i2570].read(iprot); + xfer += this->success[_i2576].read(iprot); } xfer += iprot->readListEnd(); } @@ -59388,10 +59388,10 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::write(::apache::thr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2571; - for (_iter2571 = this->success.begin(); _iter2571 != this->success.end(); ++_iter2571) + std::vector ::const_iterator _iter2577; + for (_iter2577 = this->success.begin(); _iter2577 != this->success.end(); ++_iter2577) { - xfer += (*_iter2571).write(oprot); + xfer += (*_iter2577).write(oprot); } xfer += oprot->writeListEnd(); } @@ -59440,14 +59440,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2572; - ::apache::thrift::protocol::TType _etype2575; - xfer += iprot->readListBegin(_etype2575, _size2572); - (*(this->success)).resize(_size2572); - uint32_t _i2576; - for (_i2576 = 0; _i2576 < _size2572; ++_i2576) + uint32_t _size2578; + ::apache::thrift::protocol::TType _etype2581; + xfer += iprot->readListBegin(_etype2581, _size2578); + (*(this->success)).resize(_size2578); + uint32_t _i2582; + for (_i2582 = 0; _i2582 < _size2578; ++_i2582) { - xfer += (*(this->success))[_i2576].read(iprot); + xfer += (*(this->success))[_i2582].read(iprot); } xfer += iprot->readListEnd(); } @@ -61500,14 +61500,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2577; - ::apache::thrift::protocol::TType _etype2580; - xfer += iprot->readListBegin(_etype2580, _size2577); - this->success.resize(_size2577); - uint32_t _i2581; - for (_i2581 = 0; _i2581 < _size2577; ++_i2581) + uint32_t _size2583; + ::apache::thrift::protocol::TType _etype2586; + xfer += iprot->readListBegin(_etype2586, _size2583); + this->success.resize(_size2583); + uint32_t _i2587; + for (_i2587 = 0; _i2587 < _size2583; ++_i2587) { - xfer += this->success[_i2581].read(iprot); + xfer += this->success[_i2587].read(iprot); } xfer += iprot->readListEnd(); } @@ -61546,10 +61546,10 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::write(::apache::thrift::p xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2582; - for (_iter2582 = this->success.begin(); _iter2582 != this->success.end(); ++_iter2582) + std::vector ::const_iterator _iter2588; + for (_iter2588 = this->success.begin(); _iter2588 != this->success.end(); ++_iter2588) { - xfer += (*_iter2582).write(oprot); + xfer += (*_iter2588).write(oprot); } xfer += oprot->writeListEnd(); } @@ -61594,14 +61594,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2583; - ::apache::thrift::protocol::TType _etype2586; - xfer += iprot->readListBegin(_etype2586, _size2583); - (*(this->success)).resize(_size2583); - uint32_t _i2587; - for (_i2587 = 0; _i2587 < _size2583; ++_i2587) + uint32_t _size2589; + ::apache::thrift::protocol::TType _etype2592; + xfer += iprot->readListBegin(_etype2592, _size2589); + (*(this->success)).resize(_size2589); + uint32_t _i2593; + for (_i2593 = 0; _i2593 < _size2589; ++_i2593) { - xfer += (*(this->success))[_i2587].read(iprot); + xfer += (*(this->success))[_i2593].read(iprot); } xfer += iprot->readListEnd(); } @@ -64036,14 +64036,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2588; - ::apache::thrift::protocol::TType _etype2591; - xfer += iprot->readListBegin(_etype2591, _size2588); - this->success.resize(_size2588); - uint32_t _i2592; - for (_i2592 = 0; _i2592 < _size2588; ++_i2592) + uint32_t _size2594; + ::apache::thrift::protocol::TType _etype2597; + xfer += iprot->readListBegin(_etype2597, _size2594); + this->success.resize(_size2594); + uint32_t _i2598; + for (_i2598 = 0; _i2598 < _size2594; ++_i2598) { - xfer += iprot->readString(this->success[_i2592]); + xfer += iprot->readString(this->success[_i2598]); } xfer += iprot->readListEnd(); } @@ -64082,10 +64082,10 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::write(::apache::t xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2593; - for (_iter2593 = this->success.begin(); _iter2593 != this->success.end(); ++_iter2593) + std::vector ::const_iterator _iter2599; + for (_iter2599 = this->success.begin(); _iter2599 != this->success.end(); ++_iter2599) { - xfer += oprot->writeString((*_iter2593)); + xfer += oprot->writeString((*_iter2599)); } xfer += oprot->writeListEnd(); } @@ -64130,14 +64130,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2594; - ::apache::thrift::protocol::TType _etype2597; - xfer += iprot->readListBegin(_etype2597, _size2594); - (*(this->success)).resize(_size2594); - uint32_t _i2598; - for (_i2598 = 0; _i2598 < _size2594; ++_i2598) + uint32_t _size2600; + ::apache::thrift::protocol::TType _etype2603; + xfer += iprot->readListBegin(_etype2603, _size2600); + (*(this->success)).resize(_size2600); + uint32_t _i2604; + for (_i2604 = 0; _i2604 < _size2600; ++_i2604) { - xfer += iprot->readString((*(this->success))[_i2598]); + xfer += iprot->readString((*(this->success))[_i2604]); } xfer += iprot->readListEnd(); } @@ -64689,14 +64689,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2599; - ::apache::thrift::protocol::TType _etype2602; - xfer += iprot->readListBegin(_etype2602, _size2599); - this->success.resize(_size2599); - uint32_t _i2603; - for (_i2603 = 0; _i2603 < _size2599; ++_i2603) + uint32_t _size2605; + ::apache::thrift::protocol::TType _etype2608; + xfer += iprot->readListBegin(_etype2608, _size2605); + this->success.resize(_size2605); + uint32_t _i2609; + for (_i2609 = 0; _i2609 < _size2605; ++_i2609) { - xfer += iprot->readString(this->success[_i2603]); + xfer += iprot->readString(this->success[_i2609]); } xfer += iprot->readListEnd(); } @@ -64735,10 +64735,10 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::write(::apache::thrift::pr xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter2604; - for (_iter2604 = this->success.begin(); _iter2604 != this->success.end(); ++_iter2604) + std::vector ::const_iterator _iter2610; + for (_iter2610 = this->success.begin(); _iter2610 != this->success.end(); ++_iter2610) { - xfer += oprot->writeString((*_iter2604)); + xfer += oprot->writeString((*_iter2610)); } xfer += oprot->writeListEnd(); } @@ -64783,14 +64783,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_presult::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2605; - ::apache::thrift::protocol::TType _etype2608; - xfer += iprot->readListBegin(_etype2608, _size2605); - (*(this->success)).resize(_size2605); - uint32_t _i2609; - for (_i2609 = 0; _i2609 < _size2605; ++_i2609) + uint32_t _size2611; + ::apache::thrift::protocol::TType _etype2614; + xfer += iprot->readListBegin(_etype2614, _size2611); + (*(this->success)).resize(_size2611); + uint32_t _i2615; + for (_i2615 = 0; _i2615 < _size2611; ++_i2615) { - xfer += iprot->readString((*(this->success))[_i2609]); + xfer += iprot->readString((*(this->success))[_i2615]); } xfer += iprot->readListEnd(); } @@ -65115,14 +65115,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size2610; - ::apache::thrift::protocol::TType _etype2613; - xfer += iprot->readListBegin(_etype2613, _size2610); - this->success.resize(_size2610); - uint32_t _i2614; - for (_i2614 = 0; _i2614 < _size2610; ++_i2614) + uint32_t _size2616; + ::apache::thrift::protocol::TType _etype2619; + xfer += iprot->readListBegin(_etype2619, _size2616); + this->success.resize(_size2616); + uint32_t _i2620; + for (_i2620 = 0; _i2620 < _size2616; ++_i2620) { - xfer += this->success[_i2614].read(iprot); + xfer += this->success[_i2620].read(iprot); } xfer += iprot->readListEnd(); } @@ -65161,10 +65161,10 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::write(::apache::th xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter2615; - for (_iter2615 = this->success.begin(); _iter2615 != this->success.end(); ++_iter2615) + std::vector ::const_iterator _iter2621; + for (_iter2621 = this->success.begin(); _iter2621 != this->success.end(); ++_iter2621) { - xfer += (*_iter2615).write(oprot); + xfer += (*_iter2621).write(oprot); } xfer += oprot->writeListEnd(); } @@ -65209,14 +65209,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size2616; - ::apache::thrift::protocol::TType _etype2619; - xfer += iprot->readListBegin(_etype2619, _size2616); - (*(this->success)).resize(_size2616); - uint32_t _i2620; - for (_i2620 = 0; _i2620 < _size2616; ++_i2620) + uint32_t _size2622; + ::apache::thrift::protocol::TType _etype2625; + xfer += iprot->readListBegin(_etype2625, _size2622); + (*(this->success)).resize(_size2622); + uint32_t _i2626; + for (_i2626 = 0; _i2626 < _size2622; ++_i2626) { - xfer += (*(this->success))[_i2620].read(iprot); + xfer += (*(this->success))[_i2626].read(iprot); } xfer += iprot->readListEnd(); } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index 401d63df93fe..2a7a8d65b5e4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -50167,6 +50167,11 @@ void GetPartitionsPsWithAuthRequest::__set_excludeParamKeyPattern(const std::str this->excludeParamKeyPattern = val; __isset.excludeParamKeyPattern = true; } + +void GetPartitionsPsWithAuthRequest::__set_partNames(const std::vector & val) { + this->partNames = val; +__isset.partNames = true; +} std::ostream& operator<<(std::ostream& out, const GetPartitionsPsWithAuthRequest& obj) { obj.printTo(out); @@ -50317,6 +50322,26 @@ uint32_t GetPartitionsPsWithAuthRequest::read(::apache::thrift::protocol::TProto xfer += iprot->skip(ftype); } break; + case 13: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partNames.clear(); + uint32_t _size1771; + ::apache::thrift::protocol::TType _etype1774; + xfer += iprot->readListBegin(_etype1774, _size1771); + this->partNames.resize(_size1771); + uint32_t _i1775; + for (_i1775 = 0; _i1775 < _size1771; ++_i1775) + { + xfer += iprot->readString(this->partNames[_i1775]); + } + xfer += iprot->readListEnd(); + } + this->__isset.partNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -50355,10 +50380,10 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeFieldBegin("partVals", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partVals.size())); - std::vector ::const_iterator _iter1771; - for (_iter1771 = this->partVals.begin(); _iter1771 != this->partVals.end(); ++_iter1771) + std::vector ::const_iterator _iter1776; + for (_iter1776 = this->partVals.begin(); _iter1776 != this->partVals.end(); ++_iter1776) { - xfer += oprot->writeString((*_iter1771)); + xfer += oprot->writeString((*_iter1776)); } xfer += oprot->writeListEnd(); } @@ -50378,10 +50403,10 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeFieldBegin("groupNames", ::apache::thrift::protocol::T_LIST, 7); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->groupNames.size())); - std::vector ::const_iterator _iter1772; - for (_iter1772 = this->groupNames.begin(); _iter1772 != this->groupNames.end(); ++_iter1772) + std::vector ::const_iterator _iter1777; + for (_iter1777 = this->groupNames.begin(); _iter1777 != this->groupNames.end(); ++_iter1777) { - xfer += oprot->writeString((*_iter1772)); + xfer += oprot->writeString((*_iter1777)); } xfer += oprot->writeListEnd(); } @@ -50412,6 +50437,19 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt xfer += oprot->writeString(this->excludeParamKeyPattern); xfer += oprot->writeFieldEnd(); } + if (this->__isset.partNames) { + xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 13); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); + std::vector ::const_iterator _iter1778; + for (_iter1778 = this->partNames.begin(); _iter1778 != this->partNames.end(); ++_iter1778) + { + xfer += oprot->writeString((*_iter1778)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -50431,38 +50469,41 @@ void swap(GetPartitionsPsWithAuthRequest &a, GetPartitionsPsWithAuthRequest &b) swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition); swap(a.includeParamKeyPattern, b.includeParamKeyPattern); swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern); + swap(a.partNames, b.partNames); swap(a.__isset, b.__isset); } -GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitionsPsWithAuthRequest& other1773) { - catName = other1773.catName; - dbName = other1773.dbName; - tblName = other1773.tblName; - partVals = other1773.partVals; - maxParts = other1773.maxParts; - userName = other1773.userName; - groupNames = other1773.groupNames; - validWriteIdList = other1773.validWriteIdList; - id = other1773.id; - skipColumnSchemaForPartition = other1773.skipColumnSchemaForPartition; - includeParamKeyPattern = other1773.includeParamKeyPattern; - excludeParamKeyPattern = other1773.excludeParamKeyPattern; - __isset = other1773.__isset; -} -GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1774) { - catName = other1774.catName; - dbName = other1774.dbName; - tblName = other1774.tblName; - partVals = other1774.partVals; - maxParts = other1774.maxParts; - userName = other1774.userName; - groupNames = other1774.groupNames; - validWriteIdList = other1774.validWriteIdList; - id = other1774.id; - skipColumnSchemaForPartition = other1774.skipColumnSchemaForPartition; - includeParamKeyPattern = other1774.includeParamKeyPattern; - excludeParamKeyPattern = other1774.excludeParamKeyPattern; - __isset = other1774.__isset; +GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitionsPsWithAuthRequest& other1779) { + catName = other1779.catName; + dbName = other1779.dbName; + tblName = other1779.tblName; + partVals = other1779.partVals; + maxParts = other1779.maxParts; + userName = other1779.userName; + groupNames = other1779.groupNames; + validWriteIdList = other1779.validWriteIdList; + id = other1779.id; + skipColumnSchemaForPartition = other1779.skipColumnSchemaForPartition; + includeParamKeyPattern = other1779.includeParamKeyPattern; + excludeParamKeyPattern = other1779.excludeParamKeyPattern; + partNames = other1779.partNames; + __isset = other1779.__isset; +} +GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1780) { + catName = other1780.catName; + dbName = other1780.dbName; + tblName = other1780.tblName; + partVals = other1780.partVals; + maxParts = other1780.maxParts; + userName = other1780.userName; + groupNames = other1780.groupNames; + validWriteIdList = other1780.validWriteIdList; + id = other1780.id; + skipColumnSchemaForPartition = other1780.skipColumnSchemaForPartition; + includeParamKeyPattern = other1780.includeParamKeyPattern; + excludeParamKeyPattern = other1780.excludeParamKeyPattern; + partNames = other1780.partNames; + __isset = other1780.__isset; return *this; } void GetPartitionsPsWithAuthRequest::printTo(std::ostream& out) const { @@ -50480,6 +50521,7 @@ void GetPartitionsPsWithAuthRequest::printTo(std::ostream& out) const { out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "")); out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "")); out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "")); + out << ", " << "partNames="; (__isset.partNames ? (out << to_string(partNames)) : (out << "")); out << ")"; } @@ -50524,14 +50566,14 @@ uint32_t GetPartitionsPsWithAuthResponse::read(::apache::thrift::protocol::TProt if (ftype == ::apache::thrift::protocol::T_LIST) { { this->partitions.clear(); - uint32_t _size1775; - ::apache::thrift::protocol::TType _etype1778; - xfer += iprot->readListBegin(_etype1778, _size1775); - this->partitions.resize(_size1775); - uint32_t _i1779; - for (_i1779 = 0; _i1779 < _size1775; ++_i1779) + uint32_t _size1781; + ::apache::thrift::protocol::TType _etype1784; + xfer += iprot->readListBegin(_etype1784, _size1781); + this->partitions.resize(_size1781); + uint32_t _i1785; + for (_i1785 = 0; _i1785 < _size1781; ++_i1785) { - xfer += this->partitions[_i1779].read(iprot); + xfer += this->partitions[_i1785].read(iprot); } xfer += iprot->readListEnd(); } @@ -50562,10 +50604,10 @@ uint32_t GetPartitionsPsWithAuthResponse::write(::apache::thrift::protocol::TPro xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter1780; - for (_iter1780 = this->partitions.begin(); _iter1780 != this->partitions.end(); ++_iter1780) + std::vector ::const_iterator _iter1786; + for (_iter1786 = this->partitions.begin(); _iter1786 != this->partitions.end(); ++_iter1786) { - xfer += (*_iter1780).write(oprot); + xfer += (*_iter1786).write(oprot); } xfer += oprot->writeListEnd(); } @@ -50581,11 +50623,11 @@ void swap(GetPartitionsPsWithAuthResponse &a, GetPartitionsPsWithAuthResponse &b swap(a.partitions, b.partitions); } -GetPartitionsPsWithAuthResponse::GetPartitionsPsWithAuthResponse(const GetPartitionsPsWithAuthResponse& other1781) { - partitions = other1781.partitions; +GetPartitionsPsWithAuthResponse::GetPartitionsPsWithAuthResponse(const GetPartitionsPsWithAuthResponse& other1787) { + partitions = other1787.partitions; } -GetPartitionsPsWithAuthResponse& GetPartitionsPsWithAuthResponse::operator=(const GetPartitionsPsWithAuthResponse& other1782) { - partitions = other1782.partitions; +GetPartitionsPsWithAuthResponse& GetPartitionsPsWithAuthResponse::operator=(const GetPartitionsPsWithAuthResponse& other1788) { + partitions = other1788.partitions; return *this; } void GetPartitionsPsWithAuthResponse::printTo(std::ostream& out) const { @@ -50771,23 +50813,23 @@ void swap(ReplicationMetrics &a, ReplicationMetrics &b) { swap(a.__isset, b.__isset); } -ReplicationMetrics::ReplicationMetrics(const ReplicationMetrics& other1783) { - scheduledExecutionId = other1783.scheduledExecutionId; - policy = other1783.policy; - dumpExecutionId = other1783.dumpExecutionId; - metadata = other1783.metadata; - progress = other1783.progress; - messageFormat = other1783.messageFormat; - __isset = other1783.__isset; +ReplicationMetrics::ReplicationMetrics(const ReplicationMetrics& other1789) { + scheduledExecutionId = other1789.scheduledExecutionId; + policy = other1789.policy; + dumpExecutionId = other1789.dumpExecutionId; + metadata = other1789.metadata; + progress = other1789.progress; + messageFormat = other1789.messageFormat; + __isset = other1789.__isset; } -ReplicationMetrics& ReplicationMetrics::operator=(const ReplicationMetrics& other1784) { - scheduledExecutionId = other1784.scheduledExecutionId; - policy = other1784.policy; - dumpExecutionId = other1784.dumpExecutionId; - metadata = other1784.metadata; - progress = other1784.progress; - messageFormat = other1784.messageFormat; - __isset = other1784.__isset; +ReplicationMetrics& ReplicationMetrics::operator=(const ReplicationMetrics& other1790) { + scheduledExecutionId = other1790.scheduledExecutionId; + policy = other1790.policy; + dumpExecutionId = other1790.dumpExecutionId; + metadata = other1790.metadata; + progress = other1790.progress; + messageFormat = other1790.messageFormat; + __isset = other1790.__isset; return *this; } void ReplicationMetrics::printTo(std::ostream& out) const { @@ -50843,14 +50885,14 @@ uint32_t ReplicationMetricList::read(::apache::thrift::protocol::TProtocol* ipro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->replicationMetricList.clear(); - uint32_t _size1785; - ::apache::thrift::protocol::TType _etype1788; - xfer += iprot->readListBegin(_etype1788, _size1785); - this->replicationMetricList.resize(_size1785); - uint32_t _i1789; - for (_i1789 = 0; _i1789 < _size1785; ++_i1789) + uint32_t _size1791; + ::apache::thrift::protocol::TType _etype1794; + xfer += iprot->readListBegin(_etype1794, _size1791); + this->replicationMetricList.resize(_size1791); + uint32_t _i1795; + for (_i1795 = 0; _i1795 < _size1791; ++_i1795) { - xfer += this->replicationMetricList[_i1789].read(iprot); + xfer += this->replicationMetricList[_i1795].read(iprot); } xfer += iprot->readListEnd(); } @@ -50881,10 +50923,10 @@ uint32_t ReplicationMetricList::write(::apache::thrift::protocol::TProtocol* opr xfer += oprot->writeFieldBegin("replicationMetricList", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->replicationMetricList.size())); - std::vector ::const_iterator _iter1790; - for (_iter1790 = this->replicationMetricList.begin(); _iter1790 != this->replicationMetricList.end(); ++_iter1790) + std::vector ::const_iterator _iter1796; + for (_iter1796 = this->replicationMetricList.begin(); _iter1796 != this->replicationMetricList.end(); ++_iter1796) { - xfer += (*_iter1790).write(oprot); + xfer += (*_iter1796).write(oprot); } xfer += oprot->writeListEnd(); } @@ -50900,11 +50942,11 @@ void swap(ReplicationMetricList &a, ReplicationMetricList &b) { swap(a.replicationMetricList, b.replicationMetricList); } -ReplicationMetricList::ReplicationMetricList(const ReplicationMetricList& other1791) { - replicationMetricList = other1791.replicationMetricList; +ReplicationMetricList::ReplicationMetricList(const ReplicationMetricList& other1797) { + replicationMetricList = other1797.replicationMetricList; } -ReplicationMetricList& ReplicationMetricList::operator=(const ReplicationMetricList& other1792) { - replicationMetricList = other1792.replicationMetricList; +ReplicationMetricList& ReplicationMetricList::operator=(const ReplicationMetricList& other1798) { + replicationMetricList = other1798.replicationMetricList; return *this; } void ReplicationMetricList::printTo(std::ostream& out) const { @@ -51030,17 +51072,17 @@ void swap(GetReplicationMetricsRequest &a, GetReplicationMetricsRequest &b) { swap(a.__isset, b.__isset); } -GetReplicationMetricsRequest::GetReplicationMetricsRequest(const GetReplicationMetricsRequest& other1793) { - scheduledExecutionId = other1793.scheduledExecutionId; - policy = other1793.policy; - dumpExecutionId = other1793.dumpExecutionId; - __isset = other1793.__isset; +GetReplicationMetricsRequest::GetReplicationMetricsRequest(const GetReplicationMetricsRequest& other1799) { + scheduledExecutionId = other1799.scheduledExecutionId; + policy = other1799.policy; + dumpExecutionId = other1799.dumpExecutionId; + __isset = other1799.__isset; } -GetReplicationMetricsRequest& GetReplicationMetricsRequest::operator=(const GetReplicationMetricsRequest& other1794) { - scheduledExecutionId = other1794.scheduledExecutionId; - policy = other1794.policy; - dumpExecutionId = other1794.dumpExecutionId; - __isset = other1794.__isset; +GetReplicationMetricsRequest& GetReplicationMetricsRequest::operator=(const GetReplicationMetricsRequest& other1800) { + scheduledExecutionId = other1800.scheduledExecutionId; + policy = other1800.policy; + dumpExecutionId = other1800.dumpExecutionId; + __isset = other1800.__isset; return *this; } void GetReplicationMetricsRequest::printTo(std::ostream& out) const { @@ -51093,16 +51135,16 @@ uint32_t GetOpenTxnsRequest::read(::apache::thrift::protocol::TProtocol* iprot) if (ftype == ::apache::thrift::protocol::T_LIST) { { this->excludeTxnTypes.clear(); - uint32_t _size1795; - ::apache::thrift::protocol::TType _etype1798; - xfer += iprot->readListBegin(_etype1798, _size1795); - this->excludeTxnTypes.resize(_size1795); - uint32_t _i1799; - for (_i1799 = 0; _i1799 < _size1795; ++_i1799) + uint32_t _size1801; + ::apache::thrift::protocol::TType _etype1804; + xfer += iprot->readListBegin(_etype1804, _size1801); + this->excludeTxnTypes.resize(_size1801); + uint32_t _i1805; + for (_i1805 = 0; _i1805 < _size1801; ++_i1805) { - int32_t ecast1800; - xfer += iprot->readI32(ecast1800); - this->excludeTxnTypes[_i1799] = static_cast(ecast1800); + int32_t ecast1806; + xfer += iprot->readI32(ecast1806); + this->excludeTxnTypes[_i1805] = static_cast(ecast1806); } xfer += iprot->readListEnd(); } @@ -51132,10 +51174,10 @@ uint32_t GetOpenTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeFieldBegin("excludeTxnTypes", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->excludeTxnTypes.size())); - std::vector ::const_iterator _iter1801; - for (_iter1801 = this->excludeTxnTypes.begin(); _iter1801 != this->excludeTxnTypes.end(); ++_iter1801) + std::vector ::const_iterator _iter1807; + for (_iter1807 = this->excludeTxnTypes.begin(); _iter1807 != this->excludeTxnTypes.end(); ++_iter1807) { - xfer += oprot->writeI32(static_cast((*_iter1801))); + xfer += oprot->writeI32(static_cast((*_iter1807))); } xfer += oprot->writeListEnd(); } @@ -51152,13 +51194,13 @@ void swap(GetOpenTxnsRequest &a, GetOpenTxnsRequest &b) { swap(a.__isset, b.__isset); } -GetOpenTxnsRequest::GetOpenTxnsRequest(const GetOpenTxnsRequest& other1802) { - excludeTxnTypes = other1802.excludeTxnTypes; - __isset = other1802.__isset; +GetOpenTxnsRequest::GetOpenTxnsRequest(const GetOpenTxnsRequest& other1808) { + excludeTxnTypes = other1808.excludeTxnTypes; + __isset = other1808.__isset; } -GetOpenTxnsRequest& GetOpenTxnsRequest::operator=(const GetOpenTxnsRequest& other1803) { - excludeTxnTypes = other1803.excludeTxnTypes; - __isset = other1803.__isset; +GetOpenTxnsRequest& GetOpenTxnsRequest::operator=(const GetOpenTxnsRequest& other1809) { + excludeTxnTypes = other1809.excludeTxnTypes; + __isset = other1809.__isset; return *this; } void GetOpenTxnsRequest::printTo(std::ostream& out) const { @@ -51286,15 +51328,15 @@ void swap(StoredProcedureRequest &a, StoredProcedureRequest &b) { swap(a.procName, b.procName); } -StoredProcedureRequest::StoredProcedureRequest(const StoredProcedureRequest& other1804) { - catName = other1804.catName; - dbName = other1804.dbName; - procName = other1804.procName; +StoredProcedureRequest::StoredProcedureRequest(const StoredProcedureRequest& other1810) { + catName = other1810.catName; + dbName = other1810.dbName; + procName = other1810.procName; } -StoredProcedureRequest& StoredProcedureRequest::operator=(const StoredProcedureRequest& other1805) { - catName = other1805.catName; - dbName = other1805.dbName; - procName = other1805.procName; +StoredProcedureRequest& StoredProcedureRequest::operator=(const StoredProcedureRequest& other1811) { + catName = other1811.catName; + dbName = other1811.dbName; + procName = other1811.procName; return *this; } void StoredProcedureRequest::printTo(std::ostream& out) const { @@ -51404,15 +51446,15 @@ void swap(ListStoredProcedureRequest &a, ListStoredProcedureRequest &b) { swap(a.__isset, b.__isset); } -ListStoredProcedureRequest::ListStoredProcedureRequest(const ListStoredProcedureRequest& other1806) { - catName = other1806.catName; - dbName = other1806.dbName; - __isset = other1806.__isset; +ListStoredProcedureRequest::ListStoredProcedureRequest(const ListStoredProcedureRequest& other1812) { + catName = other1812.catName; + dbName = other1812.dbName; + __isset = other1812.__isset; } -ListStoredProcedureRequest& ListStoredProcedureRequest::operator=(const ListStoredProcedureRequest& other1807) { - catName = other1807.catName; - dbName = other1807.dbName; - __isset = other1807.__isset; +ListStoredProcedureRequest& ListStoredProcedureRequest::operator=(const ListStoredProcedureRequest& other1813) { + catName = other1813.catName; + dbName = other1813.dbName; + __isset = other1813.__isset; return *this; } void ListStoredProcedureRequest::printTo(std::ostream& out) const { @@ -51567,21 +51609,21 @@ void swap(StoredProcedure &a, StoredProcedure &b) { swap(a.__isset, b.__isset); } -StoredProcedure::StoredProcedure(const StoredProcedure& other1808) { - name = other1808.name; - dbName = other1808.dbName; - catName = other1808.catName; - ownerName = other1808.ownerName; - source = other1808.source; - __isset = other1808.__isset; +StoredProcedure::StoredProcedure(const StoredProcedure& other1814) { + name = other1814.name; + dbName = other1814.dbName; + catName = other1814.catName; + ownerName = other1814.ownerName; + source = other1814.source; + __isset = other1814.__isset; } -StoredProcedure& StoredProcedure::operator=(const StoredProcedure& other1809) { - name = other1809.name; - dbName = other1809.dbName; - catName = other1809.catName; - ownerName = other1809.ownerName; - source = other1809.source; - __isset = other1809.__isset; +StoredProcedure& StoredProcedure::operator=(const StoredProcedure& other1815) { + name = other1815.name; + dbName = other1815.dbName; + catName = other1815.catName; + ownerName = other1815.ownerName; + source = other1815.source; + __isset = other1815.__isset; return *this; } void StoredProcedure::printTo(std::ostream& out) const { @@ -51756,23 +51798,23 @@ void swap(AddPackageRequest &a, AddPackageRequest &b) { swap(a.__isset, b.__isset); } -AddPackageRequest::AddPackageRequest(const AddPackageRequest& other1810) { - catName = other1810.catName; - dbName = other1810.dbName; - packageName = other1810.packageName; - ownerName = other1810.ownerName; - header = other1810.header; - body = other1810.body; - __isset = other1810.__isset; +AddPackageRequest::AddPackageRequest(const AddPackageRequest& other1816) { + catName = other1816.catName; + dbName = other1816.dbName; + packageName = other1816.packageName; + ownerName = other1816.ownerName; + header = other1816.header; + body = other1816.body; + __isset = other1816.__isset; } -AddPackageRequest& AddPackageRequest::operator=(const AddPackageRequest& other1811) { - catName = other1811.catName; - dbName = other1811.dbName; - packageName = other1811.packageName; - ownerName = other1811.ownerName; - header = other1811.header; - body = other1811.body; - __isset = other1811.__isset; +AddPackageRequest& AddPackageRequest::operator=(const AddPackageRequest& other1817) { + catName = other1817.catName; + dbName = other1817.dbName; + packageName = other1817.packageName; + ownerName = other1817.ownerName; + header = other1817.header; + body = other1817.body; + __isset = other1817.__isset; return *this; } void AddPackageRequest::printTo(std::ostream& out) const { @@ -51905,15 +51947,15 @@ void swap(GetPackageRequest &a, GetPackageRequest &b) { swap(a.packageName, b.packageName); } -GetPackageRequest::GetPackageRequest(const GetPackageRequest& other1812) { - catName = other1812.catName; - dbName = other1812.dbName; - packageName = other1812.packageName; +GetPackageRequest::GetPackageRequest(const GetPackageRequest& other1818) { + catName = other1818.catName; + dbName = other1818.dbName; + packageName = other1818.packageName; } -GetPackageRequest& GetPackageRequest::operator=(const GetPackageRequest& other1813) { - catName = other1813.catName; - dbName = other1813.dbName; - packageName = other1813.packageName; +GetPackageRequest& GetPackageRequest::operator=(const GetPackageRequest& other1819) { + catName = other1819.catName; + dbName = other1819.dbName; + packageName = other1819.packageName; return *this; } void GetPackageRequest::printTo(std::ostream& out) const { @@ -52043,15 +52085,15 @@ void swap(DropPackageRequest &a, DropPackageRequest &b) { swap(a.packageName, b.packageName); } -DropPackageRequest::DropPackageRequest(const DropPackageRequest& other1814) { - catName = other1814.catName; - dbName = other1814.dbName; - packageName = other1814.packageName; +DropPackageRequest::DropPackageRequest(const DropPackageRequest& other1820) { + catName = other1820.catName; + dbName = other1820.dbName; + packageName = other1820.packageName; } -DropPackageRequest& DropPackageRequest::operator=(const DropPackageRequest& other1815) { - catName = other1815.catName; - dbName = other1815.dbName; - packageName = other1815.packageName; +DropPackageRequest& DropPackageRequest::operator=(const DropPackageRequest& other1821) { + catName = other1821.catName; + dbName = other1821.dbName; + packageName = other1821.packageName; return *this; } void DropPackageRequest::printTo(std::ostream& out) const { @@ -52161,15 +52203,15 @@ void swap(ListPackageRequest &a, ListPackageRequest &b) { swap(a.__isset, b.__isset); } -ListPackageRequest::ListPackageRequest(const ListPackageRequest& other1816) { - catName = other1816.catName; - dbName = other1816.dbName; - __isset = other1816.__isset; +ListPackageRequest::ListPackageRequest(const ListPackageRequest& other1822) { + catName = other1822.catName; + dbName = other1822.dbName; + __isset = other1822.__isset; } -ListPackageRequest& ListPackageRequest::operator=(const ListPackageRequest& other1817) { - catName = other1817.catName; - dbName = other1817.dbName; - __isset = other1817.__isset; +ListPackageRequest& ListPackageRequest::operator=(const ListPackageRequest& other1823) { + catName = other1823.catName; + dbName = other1823.dbName; + __isset = other1823.__isset; return *this; } void ListPackageRequest::printTo(std::ostream& out) const { @@ -52341,23 +52383,23 @@ void swap(Package &a, Package &b) { swap(a.__isset, b.__isset); } -Package::Package(const Package& other1818) { - catName = other1818.catName; - dbName = other1818.dbName; - packageName = other1818.packageName; - ownerName = other1818.ownerName; - header = other1818.header; - body = other1818.body; - __isset = other1818.__isset; +Package::Package(const Package& other1824) { + catName = other1824.catName; + dbName = other1824.dbName; + packageName = other1824.packageName; + ownerName = other1824.ownerName; + header = other1824.header; + body = other1824.body; + __isset = other1824.__isset; } -Package& Package::operator=(const Package& other1819) { - catName = other1819.catName; - dbName = other1819.dbName; - packageName = other1819.packageName; - ownerName = other1819.ownerName; - header = other1819.header; - body = other1819.body; - __isset = other1819.__isset; +Package& Package::operator=(const Package& other1825) { + catName = other1825.catName; + dbName = other1825.dbName; + packageName = other1825.packageName; + ownerName = other1825.ownerName; + header = other1825.header; + body = other1825.body; + __isset = other1825.__isset; return *this; } void Package::printTo(std::ostream& out) const { @@ -52489,17 +52531,17 @@ void swap(GetAllWriteEventInfoRequest &a, GetAllWriteEventInfoRequest &b) { swap(a.__isset, b.__isset); } -GetAllWriteEventInfoRequest::GetAllWriteEventInfoRequest(const GetAllWriteEventInfoRequest& other1820) { - txnId = other1820.txnId; - dbName = other1820.dbName; - tableName = other1820.tableName; - __isset = other1820.__isset; +GetAllWriteEventInfoRequest::GetAllWriteEventInfoRequest(const GetAllWriteEventInfoRequest& other1826) { + txnId = other1826.txnId; + dbName = other1826.dbName; + tableName = other1826.tableName; + __isset = other1826.__isset; } -GetAllWriteEventInfoRequest& GetAllWriteEventInfoRequest::operator=(const GetAllWriteEventInfoRequest& other1821) { - txnId = other1821.txnId; - dbName = other1821.dbName; - tableName = other1821.tableName; - __isset = other1821.__isset; +GetAllWriteEventInfoRequest& GetAllWriteEventInfoRequest::operator=(const GetAllWriteEventInfoRequest& other1827) { + txnId = other1827.txnId; + dbName = other1827.dbName; + tableName = other1827.tableName; + __isset = other1827.__isset; return *this; } void GetAllWriteEventInfoRequest::printTo(std::ostream& out) const { @@ -52587,13 +52629,13 @@ void swap(MetaException &a, MetaException &b) { swap(a.__isset, b.__isset); } -MetaException::MetaException(const MetaException& other1822) : TException() { - message = other1822.message; - __isset = other1822.__isset; +MetaException::MetaException(const MetaException& other1828) : TException() { + message = other1828.message; + __isset = other1828.__isset; } -MetaException& MetaException::operator=(const MetaException& other1823) { - message = other1823.message; - __isset = other1823.__isset; +MetaException& MetaException::operator=(const MetaException& other1829) { + message = other1829.message; + __isset = other1829.__isset; return *this; } void MetaException::printTo(std::ostream& out) const { @@ -52690,13 +52732,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) { swap(a.__isset, b.__isset); } -UnknownTableException::UnknownTableException(const UnknownTableException& other1824) : TException() { - message = other1824.message; - __isset = other1824.__isset; +UnknownTableException::UnknownTableException(const UnknownTableException& other1830) : TException() { + message = other1830.message; + __isset = other1830.__isset; } -UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1825) { - message = other1825.message; - __isset = other1825.__isset; +UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1831) { + message = other1831.message; + __isset = other1831.__isset; return *this; } void UnknownTableException::printTo(std::ostream& out) const { @@ -52793,13 +52835,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) { swap(a.__isset, b.__isset); } -UnknownDBException::UnknownDBException(const UnknownDBException& other1826) : TException() { - message = other1826.message; - __isset = other1826.__isset; +UnknownDBException::UnknownDBException(const UnknownDBException& other1832) : TException() { + message = other1832.message; + __isset = other1832.__isset; } -UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1827) { - message = other1827.message; - __isset = other1827.__isset; +UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1833) { + message = other1833.message; + __isset = other1833.__isset; return *this; } void UnknownDBException::printTo(std::ostream& out) const { @@ -52896,13 +52938,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) { swap(a.__isset, b.__isset); } -AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1828) : TException() { - message = other1828.message; - __isset = other1828.__isset; +AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1834) : TException() { + message = other1834.message; + __isset = other1834.__isset; } -AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1829) { - message = other1829.message; - __isset = other1829.__isset; +AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1835) { + message = other1835.message; + __isset = other1835.__isset; return *this; } void AlreadyExistsException::printTo(std::ostream& out) const { @@ -52999,13 +53041,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) { swap(a.__isset, b.__isset); } -InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1830) : TException() { - message = other1830.message; - __isset = other1830.__isset; +InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1836) : TException() { + message = other1836.message; + __isset = other1836.__isset; } -InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1831) { - message = other1831.message; - __isset = other1831.__isset; +InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1837) { + message = other1837.message; + __isset = other1837.__isset; return *this; } void InvalidPartitionException::printTo(std::ostream& out) const { @@ -53102,13 +53144,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) { swap(a.__isset, b.__isset); } -UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1832) : TException() { - message = other1832.message; - __isset = other1832.__isset; +UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1838) : TException() { + message = other1838.message; + __isset = other1838.__isset; } -UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1833) { - message = other1833.message; - __isset = other1833.__isset; +UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1839) { + message = other1839.message; + __isset = other1839.__isset; return *this; } void UnknownPartitionException::printTo(std::ostream& out) const { @@ -53205,13 +53247,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) { swap(a.__isset, b.__isset); } -InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1834) : TException() { - message = other1834.message; - __isset = other1834.__isset; +InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1840) : TException() { + message = other1840.message; + __isset = other1840.__isset; } -InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1835) { - message = other1835.message; - __isset = other1835.__isset; +InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1841) { + message = other1841.message; + __isset = other1841.__isset; return *this; } void InvalidObjectException::printTo(std::ostream& out) const { @@ -53308,13 +53350,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) { swap(a.__isset, b.__isset); } -NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1836) : TException() { - message = other1836.message; - __isset = other1836.__isset; +NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1842) : TException() { + message = other1842.message; + __isset = other1842.__isset; } -NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1837) { - message = other1837.message; - __isset = other1837.__isset; +NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1843) { + message = other1843.message; + __isset = other1843.__isset; return *this; } void NoSuchObjectException::printTo(std::ostream& out) const { @@ -53411,13 +53453,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) { swap(a.__isset, b.__isset); } -InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1838) : TException() { - message = other1838.message; - __isset = other1838.__isset; +InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1844) : TException() { + message = other1844.message; + __isset = other1844.__isset; } -InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1839) { - message = other1839.message; - __isset = other1839.__isset; +InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1845) { + message = other1845.message; + __isset = other1845.__isset; return *this; } void InvalidOperationException::printTo(std::ostream& out) const { @@ -53514,13 +53556,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) { swap(a.__isset, b.__isset); } -ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1840) : TException() { - message = other1840.message; - __isset = other1840.__isset; +ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1846) : TException() { + message = other1846.message; + __isset = other1846.__isset; } -ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1841) { - message = other1841.message; - __isset = other1841.__isset; +ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1847) { + message = other1847.message; + __isset = other1847.__isset; return *this; } void ConfigValSecurityException::printTo(std::ostream& out) const { @@ -53617,13 +53659,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) { swap(a.__isset, b.__isset); } -InvalidInputException::InvalidInputException(const InvalidInputException& other1842) : TException() { - message = other1842.message; - __isset = other1842.__isset; +InvalidInputException::InvalidInputException(const InvalidInputException& other1848) : TException() { + message = other1848.message; + __isset = other1848.__isset; } -InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1843) { - message = other1843.message; - __isset = other1843.__isset; +InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1849) { + message = other1849.message; + __isset = other1849.__isset; return *this; } void InvalidInputException::printTo(std::ostream& out) const { @@ -53720,13 +53762,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) { swap(a.__isset, b.__isset); } -NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1844) : TException() { - message = other1844.message; - __isset = other1844.__isset; +NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1850) : TException() { + message = other1850.message; + __isset = other1850.__isset; } -NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1845) { - message = other1845.message; - __isset = other1845.__isset; +NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1851) { + message = other1851.message; + __isset = other1851.__isset; return *this; } void NoSuchTxnException::printTo(std::ostream& out) const { @@ -53823,13 +53865,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) { swap(a.__isset, b.__isset); } -TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1846) : TException() { - message = other1846.message; - __isset = other1846.__isset; +TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1852) : TException() { + message = other1852.message; + __isset = other1852.__isset; } -TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1847) { - message = other1847.message; - __isset = other1847.__isset; +TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1853) { + message = other1853.message; + __isset = other1853.__isset; return *this; } void TxnAbortedException::printTo(std::ostream& out) const { @@ -53926,13 +53968,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) { swap(a.__isset, b.__isset); } -TxnOpenException::TxnOpenException(const TxnOpenException& other1848) : TException() { - message = other1848.message; - __isset = other1848.__isset; +TxnOpenException::TxnOpenException(const TxnOpenException& other1854) : TException() { + message = other1854.message; + __isset = other1854.__isset; } -TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1849) { - message = other1849.message; - __isset = other1849.__isset; +TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1855) { + message = other1855.message; + __isset = other1855.__isset; return *this; } void TxnOpenException::printTo(std::ostream& out) const { @@ -54029,13 +54071,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) { swap(a.__isset, b.__isset); } -NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1850) : TException() { - message = other1850.message; - __isset = other1850.__isset; +NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1856) : TException() { + message = other1856.message; + __isset = other1856.__isset; } -NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1851) { - message = other1851.message; - __isset = other1851.__isset; +NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1857) { + message = other1857.message; + __isset = other1857.__isset; return *this; } void NoSuchLockException::printTo(std::ostream& out) const { @@ -54132,13 +54174,13 @@ void swap(CompactionAbortedException &a, CompactionAbortedException &b) { swap(a.__isset, b.__isset); } -CompactionAbortedException::CompactionAbortedException(const CompactionAbortedException& other1852) : TException() { - message = other1852.message; - __isset = other1852.__isset; +CompactionAbortedException::CompactionAbortedException(const CompactionAbortedException& other1858) : TException() { + message = other1858.message; + __isset = other1858.__isset; } -CompactionAbortedException& CompactionAbortedException::operator=(const CompactionAbortedException& other1853) { - message = other1853.message; - __isset = other1853.__isset; +CompactionAbortedException& CompactionAbortedException::operator=(const CompactionAbortedException& other1859) { + message = other1859.message; + __isset = other1859.__isset; return *this; } void CompactionAbortedException::printTo(std::ostream& out) const { @@ -54235,13 +54277,13 @@ void swap(NoSuchCompactionException &a, NoSuchCompactionException &b) { swap(a.__isset, b.__isset); } -NoSuchCompactionException::NoSuchCompactionException(const NoSuchCompactionException& other1854) : TException() { - message = other1854.message; - __isset = other1854.__isset; +NoSuchCompactionException::NoSuchCompactionException(const NoSuchCompactionException& other1860) : TException() { + message = other1860.message; + __isset = other1860.__isset; } -NoSuchCompactionException& NoSuchCompactionException::operator=(const NoSuchCompactionException& other1855) { - message = other1855.message; - __isset = other1855.__isset; +NoSuchCompactionException& NoSuchCompactionException::operator=(const NoSuchCompactionException& other1861) { + message = other1861.message; + __isset = other1861.__isset; return *this; } void NoSuchCompactionException::printTo(std::ostream& out) const { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h index 47d79d6ae08d..4e0a567345b8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -19720,7 +19720,7 @@ void swap(GetPartitionNamesPsResponse &a, GetPartitionNamesPsResponse &b); std::ostream& operator<<(std::ostream& out, const GetPartitionNamesPsResponse& obj); typedef struct _GetPartitionsPsWithAuthRequest__isset { - _GetPartitionsPsWithAuthRequest__isset() : catName(false), partVals(false), maxParts(true), userName(false), groupNames(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} + _GetPartitionsPsWithAuthRequest__isset() : catName(false), partVals(false), maxParts(true), userName(false), groupNames(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false), partNames(false) {} bool catName :1; bool partVals :1; bool maxParts :1; @@ -19731,6 +19731,7 @@ typedef struct _GetPartitionsPsWithAuthRequest__isset { bool skipColumnSchemaForPartition :1; bool includeParamKeyPattern :1; bool excludeParamKeyPattern :1; + bool partNames :1; } _GetPartitionsPsWithAuthRequest__isset; class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { @@ -19764,6 +19765,7 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { bool skipColumnSchemaForPartition; std::string includeParamKeyPattern; std::string excludeParamKeyPattern; + std::vector partNames; _GetPartitionsPsWithAuthRequest__isset __isset; @@ -19791,6 +19793,8 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { void __set_excludeParamKeyPattern(const std::string& val); + void __set_partNames(const std::vector & val); + bool operator == (const GetPartitionsPsWithAuthRequest & rhs) const { if (__isset.catName != rhs.__isset.catName) @@ -19837,6 +19841,10 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern)) return false; + if (__isset.partNames != rhs.__isset.partNames) + return false; + else if (__isset.partNames && !(partNames == rhs.partNames)) + return false; return true; } bool operator != (const GetPartitionsPsWithAuthRequest &rhs) const { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java index 9e3e03c5f308..9a77c9b3a30a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java @@ -321,15 +321,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetOpenTxnsRequest case 1: // EXCLUDE_TXN_TYPES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1520 = iprot.readListBegin(); - struct.excludeTxnTypes = new java.util.ArrayList(_list1520.size); - @org.apache.thrift.annotation.Nullable TxnType _elem1521; - for (int _i1522 = 0; _i1522 < _list1520.size; ++_i1522) + org.apache.thrift.protocol.TList _list1528 = iprot.readListBegin(); + struct.excludeTxnTypes = new java.util.ArrayList(_list1528.size); + @org.apache.thrift.annotation.Nullable TxnType _elem1529; + for (int _i1530 = 0; _i1530 < _list1528.size; ++_i1530) { - _elem1521 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); - if (_elem1521 != null) + _elem1529 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); + if (_elem1529 != null) { - struct.excludeTxnTypes.add(_elem1521); + struct.excludeTxnTypes.add(_elem1529); } } iprot.readListEnd(); @@ -357,9 +357,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetOpenTxnsRequest oprot.writeFieldBegin(EXCLUDE_TXN_TYPES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.excludeTxnTypes.size())); - for (TxnType _iter1523 : struct.excludeTxnTypes) + for (TxnType _iter1531 : struct.excludeTxnTypes) { - oprot.writeI32(_iter1523.getValue()); + oprot.writeI32(_iter1531.getValue()); } oprot.writeListEnd(); } @@ -391,9 +391,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsRequest if (struct.isSetExcludeTxnTypes()) { { oprot.writeI32(struct.excludeTxnTypes.size()); - for (TxnType _iter1524 : struct.excludeTxnTypes) + for (TxnType _iter1532 : struct.excludeTxnTypes) { - oprot.writeI32(_iter1524.getValue()); + oprot.writeI32(_iter1532.getValue()); } } } @@ -405,15 +405,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsRequest s java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1525 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32); - struct.excludeTxnTypes = new java.util.ArrayList(_list1525.size); - @org.apache.thrift.annotation.Nullable TxnType _elem1526; - for (int _i1527 = 0; _i1527 < _list1525.size; ++_i1527) + org.apache.thrift.protocol.TList _list1533 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32); + struct.excludeTxnTypes = new java.util.ArrayList(_list1533.size); + @org.apache.thrift.annotation.Nullable TxnType _elem1534; + for (int _i1535 = 0; _i1535 < _list1533.size; ++_i1535) { - _elem1526 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); - if (_elem1526 != null) + _elem1534 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32()); + if (_elem1534 != null) { - struct.excludeTxnTypes.add(_elem1526); + struct.excludeTxnTypes.add(_elem1534); } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java index 8cd28971b2ae..d490a6810f0a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java @@ -23,6 +23,7 @@ private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)10); private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)11); private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)12); + private static final org.apache.thrift.protocol.TField PART_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("partNames", org.apache.thrift.protocol.TType.LIST, (short)13); private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new GetPartitionsPsWithAuthRequestStandardSchemeFactory(); private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new GetPartitionsPsWithAuthRequestTupleSchemeFactory(); @@ -39,6 +40,7 @@ private boolean skipColumnSchemaForPartition; // optional private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional + private @org.apache.thrift.annotation.Nullable java.util.List partNames; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -53,7 +55,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { ID((short)9, "id"), SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)10, "skipColumnSchemaForPartition"), INCLUDE_PARAM_KEY_PATTERN((short)11, "includeParamKeyPattern"), - EXCLUDE_PARAM_KEY_PATTERN((short)12, "excludeParamKeyPattern"); + EXCLUDE_PARAM_KEY_PATTERN((short)12, "excludeParamKeyPattern"), + PART_NAMES((short)13, "partNames"); private static final java.util.Map byName = new java.util.HashMap(); @@ -93,6 +96,8 @@ public static _Fields findByThriftId(int fieldId) { return INCLUDE_PARAM_KEY_PATTERN; case 12: // EXCLUDE_PARAM_KEY_PATTERN return EXCLUDE_PARAM_KEY_PATTERN; + case 13: // PART_NAMES + return PART_NAMES; default: return null; } @@ -138,7 +143,7 @@ public java.lang.String getFieldName() { private static final int __ID_ISSET_ID = 1; private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 2; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.PART_VALS,_Fields.MAX_PARTS,_Fields.USER_NAME,_Fields.GROUP_NAMES,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.PART_VALS,_Fields.MAX_PARTS,_Fields.USER_NAME,_Fields.GROUP_NAMES,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN,_Fields.PART_NAMES}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -168,6 +173,9 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PART_NAMES, new org.apache.thrift.meta_data.FieldMetaData("partNames", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsPsWithAuthRequest.class, metaDataMap); } @@ -225,6 +233,10 @@ public GetPartitionsPsWithAuthRequest(GetPartitionsPsWithAuthRequest other) { if (other.isSetExcludeParamKeyPattern()) { this.excludeParamKeyPattern = other.excludeParamKeyPattern; } + if (other.isSetPartNames()) { + java.util.List __this__partNames = new java.util.ArrayList(other.partNames); + this.partNames = __this__partNames; + } } public GetPartitionsPsWithAuthRequest deepCopy() { @@ -248,6 +260,7 @@ public void clear() { this.skipColumnSchemaForPartition = false; this.includeParamKeyPattern = null; this.excludeParamKeyPattern = null; + this.partNames = null; } @org.apache.thrift.annotation.Nullable @@ -564,6 +577,46 @@ public void setExcludeParamKeyPatternIsSet(boolean value) { } } + public int getPartNamesSize() { + return (this.partNames == null) ? 0 : this.partNames.size(); + } + + @org.apache.thrift.annotation.Nullable + public java.util.Iterator getPartNamesIterator() { + return (this.partNames == null) ? null : this.partNames.iterator(); + } + + public void addToPartNames(java.lang.String elem) { + if (this.partNames == null) { + this.partNames = new java.util.ArrayList(); + } + this.partNames.add(elem); + } + + @org.apache.thrift.annotation.Nullable + public java.util.List getPartNames() { + return this.partNames; + } + + public void setPartNames(@org.apache.thrift.annotation.Nullable java.util.List partNames) { + this.partNames = partNames; + } + + public void unsetPartNames() { + this.partNames = null; + } + + /** Returns true if field partNames is set (has been assigned a value) and false otherwise */ + public boolean isSetPartNames() { + return this.partNames != null; + } + + public void setPartNamesIsSet(boolean value) { + if (!value) { + this.partNames = null; + } + } + public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) { switch (field) { case CAT_NAME: @@ -662,6 +715,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable } break; + case PART_NAMES: + if (value == null) { + unsetPartNames(); + } else { + setPartNames((java.util.List)value); + } + break; + } } @@ -704,6 +765,9 @@ public java.lang.Object getFieldValue(_Fields field) { case EXCLUDE_PARAM_KEY_PATTERN: return getExcludeParamKeyPattern(); + case PART_NAMES: + return getPartNames(); + } throw new java.lang.IllegalStateException(); } @@ -739,6 +803,8 @@ public boolean isSet(_Fields field) { return isSetIncludeParamKeyPattern(); case EXCLUDE_PARAM_KEY_PATTERN: return isSetExcludeParamKeyPattern(); + case PART_NAMES: + return isSetPartNames(); } throw new java.lang.IllegalStateException(); } @@ -864,6 +930,15 @@ public boolean equals(GetPartitionsPsWithAuthRequest that) { return false; } + boolean this_present_partNames = true && this.isSetPartNames(); + boolean that_present_partNames = true && that.isSetPartNames(); + if (this_present_partNames || that_present_partNames) { + if (!(this_present_partNames && that_present_partNames)) + return false; + if (!this.partNames.equals(that.partNames)) + return false; + } + return true; } @@ -919,6 +994,10 @@ public int hashCode() { if (isSetExcludeParamKeyPattern()) hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode(); + hashCode = hashCode * 8191 + ((isSetPartNames()) ? 131071 : 524287); + if (isSetPartNames()) + hashCode = hashCode * 8191 + partNames.hashCode(); + return hashCode; } @@ -1050,6 +1129,16 @@ public int compareTo(GetPartitionsPsWithAuthRequest other) { return lastComparison; } } + lastComparison = java.lang.Boolean.compare(isSetPartNames(), other.isSetPartNames()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartNames()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partNames, other.partNames); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1174,6 +1263,16 @@ public java.lang.String toString() { } first = false; } + if (isSetPartNames()) { + if (!first) sb.append(", "); + sb.append("partNames:"); + if (this.partNames == null) { + sb.append("null"); + } else { + sb.append(this.partNames); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -1343,6 +1442,24 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsPsWith org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 13: // PART_NAMES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list1494 = iprot.readListBegin(); + struct.partNames = new java.util.ArrayList(_list1494.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1495; + for (int _i1496 = 0; _i1496 < _list1494.size; ++_i1496) + { + _elem1495 = iprot.readString(); + struct.partNames.add(_elem1495); + } + iprot.readListEnd(); + } + struct.setPartNamesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -1378,9 +1495,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partVals.size())); - for (java.lang.String _iter1494 : struct.partVals) + for (java.lang.String _iter1497 : struct.partVals) { - oprot.writeString(_iter1494); + oprot.writeString(_iter1497); } oprot.writeListEnd(); } @@ -1404,9 +1521,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groupNames.size())); - for (java.lang.String _iter1495 : struct.groupNames) + for (java.lang.String _iter1498 : struct.groupNames) { - oprot.writeString(_iter1495); + oprot.writeString(_iter1498); } oprot.writeListEnd(); } @@ -1444,6 +1561,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldEnd(); } } + if (struct.partNames != null) { + if (struct.isSetPartNames()) { + oprot.writeFieldBegin(PART_NAMES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size())); + for (java.lang.String _iter1499 : struct.partNames) + { + oprot.writeString(_iter1499); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -1494,16 +1625,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetExcludeParamKeyPattern()) { optionals.set(9); } - oprot.writeBitSet(optionals, 10); + if (struct.isSetPartNames()) { + optionals.set(10); + } + oprot.writeBitSet(optionals, 11); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } if (struct.isSetPartVals()) { { oprot.writeI32(struct.partVals.size()); - for (java.lang.String _iter1496 : struct.partVals) + for (java.lang.String _iter1500 : struct.partVals) { - oprot.writeString(_iter1496); + oprot.writeString(_iter1500); } } } @@ -1516,9 +1650,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetGroupNames()) { { oprot.writeI32(struct.groupNames.size()); - for (java.lang.String _iter1497 : struct.groupNames) + for (java.lang.String _iter1501 : struct.groupNames) { - oprot.writeString(_iter1497); + oprot.writeString(_iter1501); } } } @@ -1537,6 +1671,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith if (struct.isSetExcludeParamKeyPattern()) { oprot.writeString(struct.excludeParamKeyPattern); } + if (struct.isSetPartNames()) { + { + oprot.writeI32(struct.partNames.size()); + for (java.lang.String _iter1502 : struct.partNames) + { + oprot.writeString(_iter1502); + } + } + } } @Override @@ -1546,20 +1689,20 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA struct.setDbNameIsSet(true); struct.tblName = iprot.readString(); struct.setTblNameIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(10); + java.util.BitSet incoming = iprot.readBitSet(11); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1498 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partVals = new java.util.ArrayList(_list1498.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1499; - for (int _i1500 = 0; _i1500 < _list1498.size; ++_i1500) + org.apache.thrift.protocol.TList _list1503 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partVals = new java.util.ArrayList(_list1503.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1504; + for (int _i1505 = 0; _i1505 < _list1503.size; ++_i1505) { - _elem1499 = iprot.readString(); - struct.partVals.add(_elem1499); + _elem1504 = iprot.readString(); + struct.partVals.add(_elem1504); } } struct.setPartValsIsSet(true); @@ -1574,13 +1717,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1501 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.groupNames = new java.util.ArrayList(_list1501.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1502; - for (int _i1503 = 0; _i1503 < _list1501.size; ++_i1503) + org.apache.thrift.protocol.TList _list1506 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.groupNames = new java.util.ArrayList(_list1506.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1507; + for (int _i1508 = 0; _i1508 < _list1506.size; ++_i1508) { - _elem1502 = iprot.readString(); - struct.groupNames.add(_elem1502); + _elem1507 = iprot.readString(); + struct.groupNames.add(_elem1507); } } struct.setGroupNamesIsSet(true); @@ -1605,6 +1748,19 @@ public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithA struct.excludeParamKeyPattern = iprot.readString(); struct.setExcludeParamKeyPatternIsSet(true); } + if (incoming.get(10)) { + { + org.apache.thrift.protocol.TList _list1509 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partNames = new java.util.ArrayList(_list1509.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1510; + for (int _i1511 = 0; _i1511 < _list1509.size; ++_i1511) + { + _elem1510 = iprot.readString(); + struct.partNames.add(_elem1510); + } + } + struct.setPartNamesIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java index fa839ad0470d..6984966a98d8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsPsWith case 1: // PARTITIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1504 = iprot.readListBegin(); - struct.partitions = new java.util.ArrayList(_list1504.size); - @org.apache.thrift.annotation.Nullable Partition _elem1505; - for (int _i1506 = 0; _i1506 < _list1504.size; ++_i1506) + org.apache.thrift.protocol.TList _list1512 = iprot.readListBegin(); + struct.partitions = new java.util.ArrayList(_list1512.size); + @org.apache.thrift.annotation.Nullable Partition _elem1513; + for (int _i1514 = 0; _i1514 < _list1512.size; ++_i1514) { - _elem1505 = new Partition(); - _elem1505.read(iprot); - struct.partitions.add(_elem1505); + _elem1513 = new Partition(); + _elem1513.read(iprot); + struct.partitions.add(_elem1513); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsPsWit oprot.writeFieldBegin(PARTITIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size())); - for (Partition _iter1507 : struct.partitions) + for (Partition _iter1515 : struct.partitions) { - _iter1507.write(oprot); + _iter1515.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.partitions.size()); - for (Partition _iter1508 : struct.partitions) + for (Partition _iter1516 : struct.partitions) { - _iter1508.write(oprot); + _iter1516.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWith public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithAuthResponse struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1509 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.partitions = new java.util.ArrayList(_list1509.size); - @org.apache.thrift.annotation.Nullable Partition _elem1510; - for (int _i1511 = 0; _i1511 < _list1509.size; ++_i1511) + org.apache.thrift.protocol.TList _list1517 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.partitions = new java.util.ArrayList(_list1517.size); + @org.apache.thrift.annotation.Nullable Partition _elem1518; + for (int _i1519 = 0; _i1519 < _list1517.size; ++_i1519) { - _elem1510 = new Partition(); - _elem1510.read(iprot); - struct.partitions.add(_elem1510); + _elem1518 = new Partition(); + _elem1518.read(iprot); + struct.partitions.add(_elem1518); } } struct.setPartitionsIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java index 891fa2969a98..cbfdcbc01c4f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java @@ -329,14 +329,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ReplicationMetricLi case 1: // REPLICATION_METRIC_LIST if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1512 = iprot.readListBegin(); - struct.replicationMetricList = new java.util.ArrayList(_list1512.size); - @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1513; - for (int _i1514 = 0; _i1514 < _list1512.size; ++_i1514) + org.apache.thrift.protocol.TList _list1520 = iprot.readListBegin(); + struct.replicationMetricList = new java.util.ArrayList(_list1520.size); + @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1521; + for (int _i1522 = 0; _i1522 < _list1520.size; ++_i1522) { - _elem1513 = new ReplicationMetrics(); - _elem1513.read(iprot); - struct.replicationMetricList.add(_elem1513); + _elem1521 = new ReplicationMetrics(); + _elem1521.read(iprot); + struct.replicationMetricList.add(_elem1521); } iprot.readListEnd(); } @@ -362,9 +362,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ReplicationMetricL oprot.writeFieldBegin(REPLICATION_METRIC_LIST_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.replicationMetricList.size())); - for (ReplicationMetrics _iter1515 : struct.replicationMetricList) + for (ReplicationMetrics _iter1523 : struct.replicationMetricList) { - _iter1515.write(oprot); + _iter1523.write(oprot); } oprot.writeListEnd(); } @@ -389,9 +389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricLi org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { oprot.writeI32(struct.replicationMetricList.size()); - for (ReplicationMetrics _iter1516 : struct.replicationMetricList) + for (ReplicationMetrics _iter1524 : struct.replicationMetricList) { - _iter1516.write(oprot); + _iter1524.write(oprot); } } } @@ -400,14 +400,14 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricLi public void read(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricList struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot; { - org.apache.thrift.protocol.TList _list1517 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.replicationMetricList = new java.util.ArrayList(_list1517.size); - @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1518; - for (int _i1519 = 0; _i1519 < _list1517.size; ++_i1519) + org.apache.thrift.protocol.TList _list1525 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.replicationMetricList = new java.util.ArrayList(_list1525.size); + @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1526; + for (int _i1527 = 0; _i1527 < _list1525.size; ++_i1527) { - _elem1518 = new ReplicationMetrics(); - _elem1518.read(iprot); - struct.replicationMetricList.add(_elem1518); + _elem1526 = new ReplicationMetrics(); + _elem1526.read(iprot); + struct.replicationMetricList.add(_elem1526); } } struct.setReplicationMetricListIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java index eb9b6dc553de..57799851969b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java @@ -58665,13 +58665,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_databases_resul case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1528 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1528.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1529; - for (int _i1530 = 0; _i1530 < _list1528.size; ++_i1530) + org.apache.thrift.protocol.TList _list1536 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1536.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1537; + for (int _i1538 = 0; _i1538 < _list1536.size; ++_i1538) { - _elem1529 = iprot.readString(); - struct.success.add(_elem1529); + _elem1537 = iprot.readString(); + struct.success.add(_elem1537); } iprot.readListEnd(); } @@ -58706,9 +58706,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_databases_resu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1531 : struct.success) + for (java.lang.String _iter1539 : struct.success) { - oprot.writeString(_iter1531); + oprot.writeString(_iter1539); } oprot.writeListEnd(); } @@ -58747,9 +58747,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_databases_resul if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1532 : struct.success) + for (java.lang.String _iter1540 : struct.success) { - oprot.writeString(_iter1532); + oprot.writeString(_iter1540); } } } @@ -58764,13 +58764,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_databases_result java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1533 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1533.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1534; - for (int _i1535 = 0; _i1535 < _list1533.size; ++_i1535) + org.apache.thrift.protocol.TList _list1541 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1541.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1542; + for (int _i1543 = 0; _i1543 < _list1541.size; ++_i1543) { - _elem1534 = iprot.readString(); - struct.success.add(_elem1534); + _elem1542 = iprot.readString(); + struct.success.add(_elem1542); } } struct.setSuccessIsSet(true); @@ -59433,13 +59433,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_databases_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1536 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1536.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1537; - for (int _i1538 = 0; _i1538 < _list1536.size; ++_i1538) + org.apache.thrift.protocol.TList _list1544 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1544.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1545; + for (int _i1546 = 0; _i1546 < _list1544.size; ++_i1546) { - _elem1537 = iprot.readString(); - struct.success.add(_elem1537); + _elem1545 = iprot.readString(); + struct.success.add(_elem1545); } iprot.readListEnd(); } @@ -59474,9 +59474,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_databases_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1539 : struct.success) + for (java.lang.String _iter1547 : struct.success) { - oprot.writeString(_iter1539); + oprot.writeString(_iter1547); } oprot.writeListEnd(); } @@ -59515,9 +59515,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_databases_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1540 : struct.success) + for (java.lang.String _iter1548 : struct.success) { - oprot.writeString(_iter1540); + oprot.writeString(_iter1548); } } } @@ -59532,13 +59532,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_databases_re java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1541 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1541.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1542; - for (int _i1543 = 0; _i1543 < _list1541.size; ++_i1543) + org.apache.thrift.protocol.TList _list1549 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1549.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1550; + for (int _i1551 = 0; _i1551 < _list1549.size; ++_i1551) { - _elem1542 = iprot.readString(); - struct.success.add(_elem1542); + _elem1550 = iprot.readString(); + struct.success.add(_elem1550); } } struct.setSuccessIsSet(true); @@ -64163,13 +64163,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_dataconnectors_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1544 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1544.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1545; - for (int _i1546 = 0; _i1546 < _list1544.size; ++_i1546) + org.apache.thrift.protocol.TList _list1552 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1552.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1553; + for (int _i1554 = 0; _i1554 < _list1552.size; ++_i1554) { - _elem1545 = iprot.readString(); - struct.success.add(_elem1545); + _elem1553 = iprot.readString(); + struct.success.add(_elem1553); } iprot.readListEnd(); } @@ -64204,9 +64204,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_dataconnectors oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1547 : struct.success) + for (java.lang.String _iter1555 : struct.success) { - oprot.writeString(_iter1547); + oprot.writeString(_iter1555); } oprot.writeListEnd(); } @@ -64245,9 +64245,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_dataconnectors_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1548 : struct.success) + for (java.lang.String _iter1556 : struct.success) { - oprot.writeString(_iter1548); + oprot.writeString(_iter1556); } } } @@ -64262,13 +64262,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_dataconnectors_r java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1549 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1549.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1550; - for (int _i1551 = 0; _i1551 < _list1549.size; ++_i1551) + org.apache.thrift.protocol.TList _list1557 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1557.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1558; + for (int _i1559 = 0; _i1559 < _list1557.size; ++_i1559) { - _elem1550 = iprot.readString(); - struct.success.add(_elem1550); + _elem1558 = iprot.readString(); + struct.success.add(_elem1558); } } struct.setSuccessIsSet(true); @@ -68909,16 +68909,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_type_all_result case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1552 = iprot.readMapBegin(); - struct.success = new java.util.HashMap(2*_map1552.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1553; - @org.apache.thrift.annotation.Nullable Type _val1554; - for (int _i1555 = 0; _i1555 < _map1552.size; ++_i1555) + org.apache.thrift.protocol.TMap _map1560 = iprot.readMapBegin(); + struct.success = new java.util.HashMap(2*_map1560.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1561; + @org.apache.thrift.annotation.Nullable Type _val1562; + for (int _i1563 = 0; _i1563 < _map1560.size; ++_i1563) { - _key1553 = iprot.readString(); - _val1554 = new Type(); - _val1554.read(iprot); - struct.success.put(_key1553, _val1554); + _key1561 = iprot.readString(); + _val1562 = new Type(); + _val1562.read(iprot); + struct.success.put(_key1561, _val1562); } iprot.readMapEnd(); } @@ -68953,10 +68953,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_type_all_resul oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (java.util.Map.Entry _iter1556 : struct.success.entrySet()) + for (java.util.Map.Entry _iter1564 : struct.success.entrySet()) { - oprot.writeString(_iter1556.getKey()); - _iter1556.getValue().write(oprot); + oprot.writeString(_iter1564.getKey()); + _iter1564.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -68995,10 +68995,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_type_all_result if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.util.Map.Entry _iter1557 : struct.success.entrySet()) + for (java.util.Map.Entry _iter1565 : struct.success.entrySet()) { - oprot.writeString(_iter1557.getKey()); - _iter1557.getValue().write(oprot); + oprot.writeString(_iter1565.getKey()); + _iter1565.getValue().write(oprot); } } } @@ -69013,16 +69013,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_type_all_result java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map1558 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.HashMap(2*_map1558.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1559; - @org.apache.thrift.annotation.Nullable Type _val1560; - for (int _i1561 = 0; _i1561 < _map1558.size; ++_i1561) + org.apache.thrift.protocol.TMap _map1566 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.HashMap(2*_map1566.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1567; + @org.apache.thrift.annotation.Nullable Type _val1568; + for (int _i1569 = 0; _i1569 < _map1566.size; ++_i1569) { - _key1559 = iprot.readString(); - _val1560 = new Type(); - _val1560.read(iprot); - struct.success.put(_key1559, _val1560); + _key1567 = iprot.readString(); + _val1568 = new Type(); + _val1568.read(iprot); + struct.success.put(_key1567, _val1568); } } struct.setSuccessIsSet(true); @@ -70066,14 +70066,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_fields_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1562 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1562.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1563; - for (int _i1564 = 0; _i1564 < _list1562.size; ++_i1564) + org.apache.thrift.protocol.TList _list1570 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1570.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1571; + for (int _i1572 = 0; _i1572 < _list1570.size; ++_i1572) { - _elem1563 = new FieldSchema(); - _elem1563.read(iprot); - struct.success.add(_elem1563); + _elem1571 = new FieldSchema(); + _elem1571.read(iprot); + struct.success.add(_elem1571); } iprot.readListEnd(); } @@ -70126,9 +70126,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_fields_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1565 : struct.success) + for (FieldSchema _iter1573 : struct.success) { - _iter1565.write(oprot); + _iter1573.write(oprot); } oprot.writeListEnd(); } @@ -70183,9 +70183,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_fields_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1566 : struct.success) + for (FieldSchema _iter1574 : struct.success) { - _iter1566.write(oprot); + _iter1574.write(oprot); } } } @@ -70206,14 +70206,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_fields_result st java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1567 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1567.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1568; - for (int _i1569 = 0; _i1569 < _list1567.size; ++_i1569) + org.apache.thrift.protocol.TList _list1575 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1575.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1576; + for (int _i1577 = 0; _i1577 < _list1575.size; ++_i1577) { - _elem1568 = new FieldSchema(); - _elem1568.read(iprot); - struct.success.add(_elem1568); + _elem1576 = new FieldSchema(); + _elem1576.read(iprot); + struct.success.add(_elem1576); } } struct.setSuccessIsSet(true); @@ -71376,14 +71376,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_fields_with_env case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1570 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1570.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1571; - for (int _i1572 = 0; _i1572 < _list1570.size; ++_i1572) + org.apache.thrift.protocol.TList _list1578 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1578.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1579; + for (int _i1580 = 0; _i1580 < _list1578.size; ++_i1580) { - _elem1571 = new FieldSchema(); - _elem1571.read(iprot); - struct.success.add(_elem1571); + _elem1579 = new FieldSchema(); + _elem1579.read(iprot); + struct.success.add(_elem1579); } iprot.readListEnd(); } @@ -71436,9 +71436,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_fields_with_en oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1573 : struct.success) + for (FieldSchema _iter1581 : struct.success) { - _iter1573.write(oprot); + _iter1581.write(oprot); } oprot.writeListEnd(); } @@ -71493,9 +71493,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_fields_with_env if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1574 : struct.success) + for (FieldSchema _iter1582 : struct.success) { - _iter1574.write(oprot); + _iter1582.write(oprot); } } } @@ -71516,14 +71516,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_fields_with_envi java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1575 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1575.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1576; - for (int _i1577 = 0; _i1577 < _list1575.size; ++_i1577) + org.apache.thrift.protocol.TList _list1583 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1583.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1584; + for (int _i1585 = 0; _i1585 < _list1583.size; ++_i1585) { - _elem1576 = new FieldSchema(); - _elem1576.read(iprot); - struct.success.add(_elem1576); + _elem1584 = new FieldSchema(); + _elem1584.read(iprot); + struct.success.add(_elem1584); } } struct.setSuccessIsSet(true); @@ -73629,14 +73629,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1578 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1578.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1579; - for (int _i1580 = 0; _i1580 < _list1578.size; ++_i1580) + org.apache.thrift.protocol.TList _list1586 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1586.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1587; + for (int _i1588 = 0; _i1588 < _list1586.size; ++_i1588) { - _elem1579 = new FieldSchema(); - _elem1579.read(iprot); - struct.success.add(_elem1579); + _elem1587 = new FieldSchema(); + _elem1587.read(iprot); + struct.success.add(_elem1587); } iprot.readListEnd(); } @@ -73689,9 +73689,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1581 : struct.success) + for (FieldSchema _iter1589 : struct.success) { - _iter1581.write(oprot); + _iter1589.write(oprot); } oprot.writeListEnd(); } @@ -73746,9 +73746,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1582 : struct.success) + for (FieldSchema _iter1590 : struct.success) { - _iter1582.write(oprot); + _iter1590.write(oprot); } } } @@ -73769,14 +73769,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_result st java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1583 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1583.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1584; - for (int _i1585 = 0; _i1585 < _list1583.size; ++_i1585) + org.apache.thrift.protocol.TList _list1591 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1591.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1592; + for (int _i1593 = 0; _i1593 < _list1591.size; ++_i1593) { - _elem1584 = new FieldSchema(); - _elem1584.read(iprot); - struct.success.add(_elem1584); + _elem1592 = new FieldSchema(); + _elem1592.read(iprot); + struct.success.add(_elem1592); } } struct.setSuccessIsSet(true); @@ -74939,14 +74939,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_with_env case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1586 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1586.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1587; - for (int _i1588 = 0; _i1588 < _list1586.size; ++_i1588) + org.apache.thrift.protocol.TList _list1594 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1594.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1595; + for (int _i1596 = 0; _i1596 < _list1594.size; ++_i1596) { - _elem1587 = new FieldSchema(); - _elem1587.read(iprot); - struct.success.add(_elem1587); + _elem1595 = new FieldSchema(); + _elem1595.read(iprot); + struct.success.add(_elem1595); } iprot.readListEnd(); } @@ -74999,9 +74999,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_with_en oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter1589 : struct.success) + for (FieldSchema _iter1597 : struct.success) { - _iter1589.write(oprot); + _iter1597.write(oprot); } oprot.writeListEnd(); } @@ -75056,9 +75056,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_with_env if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter1590 : struct.success) + for (FieldSchema _iter1598 : struct.success) { - _iter1590.write(oprot); + _iter1598.write(oprot); } } } @@ -75079,14 +75079,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_with_envi java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1591 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1591.size); - @org.apache.thrift.annotation.Nullable FieldSchema _elem1592; - for (int _i1593 = 0; _i1593 < _list1591.size; ++_i1593) + org.apache.thrift.protocol.TList _list1599 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1599.size); + @org.apache.thrift.annotation.Nullable FieldSchema _elem1600; + for (int _i1601 = 0; _i1601 < _list1599.size; ++_i1601) { - _elem1592 = new FieldSchema(); - _elem1592.read(iprot); - struct.success.add(_elem1592); + _elem1600 = new FieldSchema(); + _elem1600.read(iprot); + struct.success.add(_elem1600); } } struct.setSuccessIsSet(true); @@ -79293,14 +79293,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 2: // PRIMARY_KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1594 = iprot.readListBegin(); - struct.primaryKeys = new java.util.ArrayList(_list1594.size); - @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1595; - for (int _i1596 = 0; _i1596 < _list1594.size; ++_i1596) + org.apache.thrift.protocol.TList _list1602 = iprot.readListBegin(); + struct.primaryKeys = new java.util.ArrayList(_list1602.size); + @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1603; + for (int _i1604 = 0; _i1604 < _list1602.size; ++_i1604) { - _elem1595 = new SQLPrimaryKey(); - _elem1595.read(iprot); - struct.primaryKeys.add(_elem1595); + _elem1603 = new SQLPrimaryKey(); + _elem1603.read(iprot); + struct.primaryKeys.add(_elem1603); } iprot.readListEnd(); } @@ -79312,14 +79312,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 3: // FOREIGN_KEYS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1597 = iprot.readListBegin(); - struct.foreignKeys = new java.util.ArrayList(_list1597.size); - @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1598; - for (int _i1599 = 0; _i1599 < _list1597.size; ++_i1599) + org.apache.thrift.protocol.TList _list1605 = iprot.readListBegin(); + struct.foreignKeys = new java.util.ArrayList(_list1605.size); + @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1606; + for (int _i1607 = 0; _i1607 < _list1605.size; ++_i1607) { - _elem1598 = new SQLForeignKey(); - _elem1598.read(iprot); - struct.foreignKeys.add(_elem1598); + _elem1606 = new SQLForeignKey(); + _elem1606.read(iprot); + struct.foreignKeys.add(_elem1606); } iprot.readListEnd(); } @@ -79331,14 +79331,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 4: // UNIQUE_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1600 = iprot.readListBegin(); - struct.uniqueConstraints = new java.util.ArrayList(_list1600.size); - @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1601; - for (int _i1602 = 0; _i1602 < _list1600.size; ++_i1602) + org.apache.thrift.protocol.TList _list1608 = iprot.readListBegin(); + struct.uniqueConstraints = new java.util.ArrayList(_list1608.size); + @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1609; + for (int _i1610 = 0; _i1610 < _list1608.size; ++_i1610) { - _elem1601 = new SQLUniqueConstraint(); - _elem1601.read(iprot); - struct.uniqueConstraints.add(_elem1601); + _elem1609 = new SQLUniqueConstraint(); + _elem1609.read(iprot); + struct.uniqueConstraints.add(_elem1609); } iprot.readListEnd(); } @@ -79350,14 +79350,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 5: // NOT_NULL_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1603 = iprot.readListBegin(); - struct.notNullConstraints = new java.util.ArrayList(_list1603.size); - @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1604; - for (int _i1605 = 0; _i1605 < _list1603.size; ++_i1605) + org.apache.thrift.protocol.TList _list1611 = iprot.readListBegin(); + struct.notNullConstraints = new java.util.ArrayList(_list1611.size); + @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1612; + for (int _i1613 = 0; _i1613 < _list1611.size; ++_i1613) { - _elem1604 = new SQLNotNullConstraint(); - _elem1604.read(iprot); - struct.notNullConstraints.add(_elem1604); + _elem1612 = new SQLNotNullConstraint(); + _elem1612.read(iprot); + struct.notNullConstraints.add(_elem1612); } iprot.readListEnd(); } @@ -79369,14 +79369,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 6: // DEFAULT_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin(); - struct.defaultConstraints = new java.util.ArrayList(_list1606.size); - @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1607; - for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608) + org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin(); + struct.defaultConstraints = new java.util.ArrayList(_list1614.size); + @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1615; + for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616) { - _elem1607 = new SQLDefaultConstraint(); - _elem1607.read(iprot); - struct.defaultConstraints.add(_elem1607); + _elem1615 = new SQLDefaultConstraint(); + _elem1615.read(iprot); + struct.defaultConstraints.add(_elem1615); } iprot.readListEnd(); } @@ -79388,14 +79388,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, create_table_with_c case 7: // CHECK_CONSTRAINTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1609 = iprot.readListBegin(); - struct.checkConstraints = new java.util.ArrayList(_list1609.size); - @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1610; - for (int _i1611 = 0; _i1611 < _list1609.size; ++_i1611) + org.apache.thrift.protocol.TList _list1617 = iprot.readListBegin(); + struct.checkConstraints = new java.util.ArrayList(_list1617.size); + @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1618; + for (int _i1619 = 0; _i1619 < _list1617.size; ++_i1619) { - _elem1610 = new SQLCheckConstraint(); - _elem1610.read(iprot); - struct.checkConstraints.add(_elem1610); + _elem1618 = new SQLCheckConstraint(); + _elem1618.read(iprot); + struct.checkConstraints.add(_elem1618); } iprot.readListEnd(); } @@ -79426,9 +79426,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size())); - for (SQLPrimaryKey _iter1612 : struct.primaryKeys) + for (SQLPrimaryKey _iter1620 : struct.primaryKeys) { - _iter1612.write(oprot); + _iter1620.write(oprot); } oprot.writeListEnd(); } @@ -79438,9 +79438,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size())); - for (SQLForeignKey _iter1613 : struct.foreignKeys) + for (SQLForeignKey _iter1621 : struct.foreignKeys) { - _iter1613.write(oprot); + _iter1621.write(oprot); } oprot.writeListEnd(); } @@ -79450,9 +79450,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size())); - for (SQLUniqueConstraint _iter1614 : struct.uniqueConstraints) + for (SQLUniqueConstraint _iter1622 : struct.uniqueConstraints) { - _iter1614.write(oprot); + _iter1622.write(oprot); } oprot.writeListEnd(); } @@ -79462,9 +79462,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size())); - for (SQLNotNullConstraint _iter1615 : struct.notNullConstraints) + for (SQLNotNullConstraint _iter1623 : struct.notNullConstraints) { - _iter1615.write(oprot); + _iter1623.write(oprot); } oprot.writeListEnd(); } @@ -79474,9 +79474,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size())); - for (SQLDefaultConstraint _iter1616 : struct.defaultConstraints) + for (SQLDefaultConstraint _iter1624 : struct.defaultConstraints) { - _iter1616.write(oprot); + _iter1624.write(oprot); } oprot.writeListEnd(); } @@ -79486,9 +79486,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, create_table_with_ oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size())); - for (SQLCheckConstraint _iter1617 : struct.checkConstraints) + for (SQLCheckConstraint _iter1625 : struct.checkConstraints) { - _iter1617.write(oprot); + _iter1625.write(oprot); } oprot.writeListEnd(); } @@ -79540,54 +79540,54 @@ public void write(org.apache.thrift.protocol.TProtocol prot, create_table_with_c if (struct.isSetPrimaryKeys()) { { oprot.writeI32(struct.primaryKeys.size()); - for (SQLPrimaryKey _iter1618 : struct.primaryKeys) + for (SQLPrimaryKey _iter1626 : struct.primaryKeys) { - _iter1618.write(oprot); + _iter1626.write(oprot); } } } if (struct.isSetForeignKeys()) { { oprot.writeI32(struct.foreignKeys.size()); - for (SQLForeignKey _iter1619 : struct.foreignKeys) + for (SQLForeignKey _iter1627 : struct.foreignKeys) { - _iter1619.write(oprot); + _iter1627.write(oprot); } } } if (struct.isSetUniqueConstraints()) { { oprot.writeI32(struct.uniqueConstraints.size()); - for (SQLUniqueConstraint _iter1620 : struct.uniqueConstraints) + for (SQLUniqueConstraint _iter1628 : struct.uniqueConstraints) { - _iter1620.write(oprot); + _iter1628.write(oprot); } } } if (struct.isSetNotNullConstraints()) { { oprot.writeI32(struct.notNullConstraints.size()); - for (SQLNotNullConstraint _iter1621 : struct.notNullConstraints) + for (SQLNotNullConstraint _iter1629 : struct.notNullConstraints) { - _iter1621.write(oprot); + _iter1629.write(oprot); } } } if (struct.isSetDefaultConstraints()) { { oprot.writeI32(struct.defaultConstraints.size()); - for (SQLDefaultConstraint _iter1622 : struct.defaultConstraints) + for (SQLDefaultConstraint _iter1630 : struct.defaultConstraints) { - _iter1622.write(oprot); + _iter1630.write(oprot); } } } if (struct.isSetCheckConstraints()) { { oprot.writeI32(struct.checkConstraints.size()); - for (SQLCheckConstraint _iter1623 : struct.checkConstraints) + for (SQLCheckConstraint _iter1631 : struct.checkConstraints) { - _iter1623.write(oprot); + _iter1631.write(oprot); } } } @@ -79604,84 +79604,84 @@ public void read(org.apache.thrift.protocol.TProtocol prot, create_table_with_co } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1624 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.primaryKeys = new java.util.ArrayList(_list1624.size); - @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1625; - for (int _i1626 = 0; _i1626 < _list1624.size; ++_i1626) + org.apache.thrift.protocol.TList _list1632 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.primaryKeys = new java.util.ArrayList(_list1632.size); + @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1633; + for (int _i1634 = 0; _i1634 < _list1632.size; ++_i1634) { - _elem1625 = new SQLPrimaryKey(); - _elem1625.read(iprot); - struct.primaryKeys.add(_elem1625); + _elem1633 = new SQLPrimaryKey(); + _elem1633.read(iprot); + struct.primaryKeys.add(_elem1633); } } struct.setPrimaryKeysIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1627 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.foreignKeys = new java.util.ArrayList(_list1627.size); - @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1628; - for (int _i1629 = 0; _i1629 < _list1627.size; ++_i1629) + org.apache.thrift.protocol.TList _list1635 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.foreignKeys = new java.util.ArrayList(_list1635.size); + @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1636; + for (int _i1637 = 0; _i1637 < _list1635.size; ++_i1637) { - _elem1628 = new SQLForeignKey(); - _elem1628.read(iprot); - struct.foreignKeys.add(_elem1628); + _elem1636 = new SQLForeignKey(); + _elem1636.read(iprot); + struct.foreignKeys.add(_elem1636); } } struct.setForeignKeysIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list1630 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.uniqueConstraints = new java.util.ArrayList(_list1630.size); - @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1631; - for (int _i1632 = 0; _i1632 < _list1630.size; ++_i1632) + org.apache.thrift.protocol.TList _list1638 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.uniqueConstraints = new java.util.ArrayList(_list1638.size); + @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1639; + for (int _i1640 = 0; _i1640 < _list1638.size; ++_i1640) { - _elem1631 = new SQLUniqueConstraint(); - _elem1631.read(iprot); - struct.uniqueConstraints.add(_elem1631); + _elem1639 = new SQLUniqueConstraint(); + _elem1639.read(iprot); + struct.uniqueConstraints.add(_elem1639); } } struct.setUniqueConstraintsIsSet(true); } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1633 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.notNullConstraints = new java.util.ArrayList(_list1633.size); - @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1634; - for (int _i1635 = 0; _i1635 < _list1633.size; ++_i1635) + org.apache.thrift.protocol.TList _list1641 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.notNullConstraints = new java.util.ArrayList(_list1641.size); + @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1642; + for (int _i1643 = 0; _i1643 < _list1641.size; ++_i1643) { - _elem1634 = new SQLNotNullConstraint(); - _elem1634.read(iprot); - struct.notNullConstraints.add(_elem1634); + _elem1642 = new SQLNotNullConstraint(); + _elem1642.read(iprot); + struct.notNullConstraints.add(_elem1642); } } struct.setNotNullConstraintsIsSet(true); } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1636 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.defaultConstraints = new java.util.ArrayList(_list1636.size); - @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1637; - for (int _i1638 = 0; _i1638 < _list1636.size; ++_i1638) + org.apache.thrift.protocol.TList _list1644 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.defaultConstraints = new java.util.ArrayList(_list1644.size); + @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1645; + for (int _i1646 = 0; _i1646 < _list1644.size; ++_i1646) { - _elem1637 = new SQLDefaultConstraint(); - _elem1637.read(iprot); - struct.defaultConstraints.add(_elem1637); + _elem1645 = new SQLDefaultConstraint(); + _elem1645.read(iprot); + struct.defaultConstraints.add(_elem1645); } } struct.setDefaultConstraintsIsSet(true); } if (incoming.get(6)) { { - org.apache.thrift.protocol.TList _list1639 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.checkConstraints = new java.util.ArrayList(_list1639.size); - @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1640; - for (int _i1641 = 0; _i1641 < _list1639.size; ++_i1641) + org.apache.thrift.protocol.TList _list1647 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.checkConstraints = new java.util.ArrayList(_list1647.size); + @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1648; + for (int _i1649 = 0; _i1649 < _list1647.size; ++_i1649) { - _elem1640 = new SQLCheckConstraint(); - _elem1640.read(iprot); - struct.checkConstraints.add(_elem1640); + _elem1648 = new SQLCheckConstraint(); + _elem1648.read(iprot); + struct.checkConstraints.add(_elem1648); } } struct.setCheckConstraintsIsSet(true); @@ -91113,13 +91113,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, truncate_table_args case 3: // PART_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1642 = iprot.readListBegin(); - struct.partNames = new java.util.ArrayList(_list1642.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1643; - for (int _i1644 = 0; _i1644 < _list1642.size; ++_i1644) + org.apache.thrift.protocol.TList _list1650 = iprot.readListBegin(); + struct.partNames = new java.util.ArrayList(_list1650.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1651; + for (int _i1652 = 0; _i1652 < _list1650.size; ++_i1652) { - _elem1643 = iprot.readString(); - struct.partNames.add(_elem1643); + _elem1651 = iprot.readString(); + struct.partNames.add(_elem1651); } iprot.readListEnd(); } @@ -91155,9 +91155,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, truncate_table_arg oprot.writeFieldBegin(PART_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size())); - for (java.lang.String _iter1645 : struct.partNames) + for (java.lang.String _iter1653 : struct.partNames) { - oprot.writeString(_iter1645); + oprot.writeString(_iter1653); } oprot.writeListEnd(); } @@ -91200,9 +91200,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, truncate_table_args if (struct.isSetPartNames()) { { oprot.writeI32(struct.partNames.size()); - for (java.lang.String _iter1646 : struct.partNames) + for (java.lang.String _iter1654 : struct.partNames) { - oprot.writeString(_iter1646); + oprot.writeString(_iter1654); } } } @@ -91222,13 +91222,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, truncate_table_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1647 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.partNames = new java.util.ArrayList(_list1647.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1648; - for (int _i1649 = 0; _i1649 < _list1647.size; ++_i1649) + org.apache.thrift.protocol.TList _list1655 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.partNames = new java.util.ArrayList(_list1655.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1656; + for (int _i1657 = 0; _i1657 < _list1655.size; ++_i1657) { - _elem1648 = iprot.readString(); - struct.partNames.add(_elem1648); + _elem1656 = iprot.readString(); + struct.partNames.add(_elem1656); } } struct.setPartNamesIsSet(true); @@ -93306,13 +93306,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1650 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1650.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1651; - for (int _i1652 = 0; _i1652 < _list1650.size; ++_i1652) + org.apache.thrift.protocol.TList _list1658 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1658.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1659; + for (int _i1660 = 0; _i1660 < _list1658.size; ++_i1660) { - _elem1651 = iprot.readString(); - struct.success.add(_elem1651); + _elem1659 = iprot.readString(); + struct.success.add(_elem1659); } iprot.readListEnd(); } @@ -93347,9 +93347,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1653 : struct.success) + for (java.lang.String _iter1661 : struct.success) { - oprot.writeString(_iter1653); + oprot.writeString(_iter1661); } oprot.writeListEnd(); } @@ -93388,9 +93388,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1654 : struct.success) + for (java.lang.String _iter1662 : struct.success) { - oprot.writeString(_iter1654); + oprot.writeString(_iter1662); } } } @@ -93405,13 +93405,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result st java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1655 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1655.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1656; - for (int _i1657 = 0; _i1657 < _list1655.size; ++_i1657) + org.apache.thrift.protocol.TList _list1663 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1663.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1664; + for (int _i1665 = 0; _i1665 < _list1663.size; ++_i1665) { - _elem1656 = iprot.readString(); - struct.success.add(_elem1656); + _elem1664 = iprot.readString(); + struct.success.add(_elem1664); } } struct.setSuccessIsSet(true); @@ -94394,13 +94394,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_by_type_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1658 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1658.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1659; - for (int _i1660 = 0; _i1660 < _list1658.size; ++_i1660) + org.apache.thrift.protocol.TList _list1666 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1666.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1667; + for (int _i1668 = 0; _i1668 < _list1666.size; ++_i1668) { - _elem1659 = iprot.readString(); - struct.success.add(_elem1659); + _elem1667 = iprot.readString(); + struct.success.add(_elem1667); } iprot.readListEnd(); } @@ -94435,9 +94435,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_by_type oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1661 : struct.success) + for (java.lang.String _iter1669 : struct.success) { - oprot.writeString(_iter1661); + oprot.writeString(_iter1669); } oprot.writeListEnd(); } @@ -94476,9 +94476,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1662 : struct.success) + for (java.lang.String _iter1670 : struct.success) { - oprot.writeString(_iter1662); + oprot.writeString(_iter1670); } } } @@ -94493,13 +94493,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_by_type_r java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1663 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1663.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1664; - for (int _i1665 = 0; _i1665 < _list1663.size; ++_i1665) + org.apache.thrift.protocol.TList _list1671 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1671.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1672; + for (int _i1673 = 0; _i1673 < _list1671.size; ++_i1673) { - _elem1664 = iprot.readString(); - struct.success.add(_elem1664); + _elem1672 = iprot.readString(); + struct.success.add(_elem1672); } } struct.setSuccessIsSet(true); @@ -95165,14 +95165,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_materialize case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1666 = iprot.readListBegin(); - struct.success = new java.util.ArrayList
(_list1666.size); - @org.apache.thrift.annotation.Nullable Table _elem1667; - for (int _i1668 = 0; _i1668 < _list1666.size; ++_i1668) + org.apache.thrift.protocol.TList _list1674 = iprot.readListBegin(); + struct.success = new java.util.ArrayList
(_list1674.size); + @org.apache.thrift.annotation.Nullable Table _elem1675; + for (int _i1676 = 0; _i1676 < _list1674.size; ++_i1676) { - _elem1667 = new Table(); - _elem1667.read(iprot); - struct.success.add(_elem1667); + _elem1675 = new Table(); + _elem1675.read(iprot); + struct.success.add(_elem1675); } iprot.readListEnd(); } @@ -95207,9 +95207,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_materializ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Table _iter1669 : struct.success) + for (Table _iter1677 : struct.success) { - _iter1669.write(oprot); + _iter1677.write(oprot); } oprot.writeListEnd(); } @@ -95248,9 +95248,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_materialize if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Table _iter1670 : struct.success) + for (Table _iter1678 : struct.success) { - _iter1670.write(oprot); + _iter1678.write(oprot); } } } @@ -95265,14 +95265,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_materialized java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1671 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList
(_list1671.size); - @org.apache.thrift.annotation.Nullable Table _elem1672; - for (int _i1673 = 0; _i1673 < _list1671.size; ++_i1673) + org.apache.thrift.protocol.TList _list1679 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList
(_list1679.size); + @org.apache.thrift.annotation.Nullable Table _elem1680; + for (int _i1681 = 0; _i1681 < _list1679.size; ++_i1681) { - _elem1672 = new Table(); - _elem1672.read(iprot); - struct.success.add(_elem1672); + _elem1680 = new Table(); + _elem1680.read(iprot); + struct.success.add(_elem1680); } } struct.setSuccessIsSet(true); @@ -96047,13 +96047,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_materialized_vi case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1674 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1674.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1675; - for (int _i1676 = 0; _i1676 < _list1674.size; ++_i1676) + org.apache.thrift.protocol.TList _list1682 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1682.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1683; + for (int _i1684 = 0; _i1684 < _list1682.size; ++_i1684) { - _elem1675 = iprot.readString(); - struct.success.add(_elem1675); + _elem1683 = iprot.readString(); + struct.success.add(_elem1683); } iprot.readListEnd(); } @@ -96088,9 +96088,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_materialized_v oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1677 : struct.success) + for (java.lang.String _iter1685 : struct.success) { - oprot.writeString(_iter1677); + oprot.writeString(_iter1685); } oprot.writeListEnd(); } @@ -96129,9 +96129,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_materialized_vi if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1678 : struct.success) + for (java.lang.String _iter1686 : struct.success) { - oprot.writeString(_iter1678); + oprot.writeString(_iter1686); } } } @@ -96146,13 +96146,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_materialized_vie java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1679 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1679.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1680; - for (int _i1681 = 0; _i1681 < _list1679.size; ++_i1681) + org.apache.thrift.protocol.TList _list1687 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1687.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1688; + for (int _i1689 = 0; _i1689 < _list1687.size; ++_i1689) { - _elem1680 = iprot.readString(); - struct.success.add(_elem1680); + _elem1688 = iprot.readString(); + struct.success.add(_elem1688); } } struct.setSuccessIsSet(true); @@ -96662,13 +96662,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_args case 3: // TBL_TYPES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1682 = iprot.readListBegin(); - struct.tbl_types = new java.util.ArrayList(_list1682.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1683; - for (int _i1684 = 0; _i1684 < _list1682.size; ++_i1684) + org.apache.thrift.protocol.TList _list1690 = iprot.readListBegin(); + struct.tbl_types = new java.util.ArrayList(_list1690.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1691; + for (int _i1692 = 0; _i1692 < _list1690.size; ++_i1692) { - _elem1683 = iprot.readString(); - struct.tbl_types.add(_elem1683); + _elem1691 = iprot.readString(); + struct.tbl_types.add(_elem1691); } iprot.readListEnd(); } @@ -96704,9 +96704,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_arg oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size())); - for (java.lang.String _iter1685 : struct.tbl_types) + for (java.lang.String _iter1693 : struct.tbl_types) { - oprot.writeString(_iter1685); + oprot.writeString(_iter1693); } oprot.writeListEnd(); } @@ -96749,9 +96749,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args if (struct.isSetTbl_types()) { { oprot.writeI32(struct.tbl_types.size()); - for (java.lang.String _iter1686 : struct.tbl_types) + for (java.lang.String _iter1694 : struct.tbl_types) { - oprot.writeString(_iter1686); + oprot.writeString(_iter1694); } } } @@ -96771,13 +96771,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1687 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.tbl_types = new java.util.ArrayList(_list1687.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1688; - for (int _i1689 = 0; _i1689 < _list1687.size; ++_i1689) + org.apache.thrift.protocol.TList _list1695 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tbl_types = new java.util.ArrayList(_list1695.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1696; + for (int _i1697 = 0; _i1697 < _list1695.size; ++_i1697) { - _elem1688 = iprot.readString(); - struct.tbl_types.add(_elem1688); + _elem1696 = iprot.readString(); + struct.tbl_types.add(_elem1696); } } struct.setTbl_typesIsSet(true); @@ -97188,14 +97188,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_meta_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1690 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1690.size); - @org.apache.thrift.annotation.Nullable TableMeta _elem1691; - for (int _i1692 = 0; _i1692 < _list1690.size; ++_i1692) + org.apache.thrift.protocol.TList _list1698 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1698.size); + @org.apache.thrift.annotation.Nullable TableMeta _elem1699; + for (int _i1700 = 0; _i1700 < _list1698.size; ++_i1700) { - _elem1691 = new TableMeta(); - _elem1691.read(iprot); - struct.success.add(_elem1691); + _elem1699 = new TableMeta(); + _elem1699.read(iprot); + struct.success.add(_elem1699); } iprot.readListEnd(); } @@ -97230,9 +97230,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_meta_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TableMeta _iter1693 : struct.success) + for (TableMeta _iter1701 : struct.success) { - _iter1693.write(oprot); + _iter1701.write(oprot); } oprot.writeListEnd(); } @@ -97271,9 +97271,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_meta_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TableMeta _iter1694 : struct.success) + for (TableMeta _iter1702 : struct.success) { - _iter1694.write(oprot); + _iter1702.write(oprot); } } } @@ -97288,14 +97288,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_meta_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1695 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1695.size); - @org.apache.thrift.annotation.Nullable TableMeta _elem1696; - for (int _i1697 = 0; _i1697 < _list1695.size; ++_i1697) + org.apache.thrift.protocol.TList _list1703 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1703.size); + @org.apache.thrift.annotation.Nullable TableMeta _elem1704; + for (int _i1705 = 0; _i1705 < _list1703.size; ++_i1705) { - _elem1696 = new TableMeta(); - _elem1696.read(iprot); - struct.success.add(_elem1696); + _elem1704 = new TableMeta(); + _elem1704.read(iprot); + struct.success.add(_elem1704); } } struct.setSuccessIsSet(true); @@ -98070,13 +98070,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_tables_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1698 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1698.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1699; - for (int _i1700 = 0; _i1700 < _list1698.size; ++_i1700) + org.apache.thrift.protocol.TList _list1706 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1706.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1707; + for (int _i1708 = 0; _i1708 < _list1706.size; ++_i1708) { - _elem1699 = iprot.readString(); - struct.success.add(_elem1699); + _elem1707 = iprot.readString(); + struct.success.add(_elem1707); } iprot.readListEnd(); } @@ -98111,9 +98111,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_tables_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1701 : struct.success) + for (java.lang.String _iter1709 : struct.success) { - oprot.writeString(_iter1701); + oprot.writeString(_iter1709); } oprot.writeListEnd(); } @@ -98152,9 +98152,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_tables_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1702 : struct.success) + for (java.lang.String _iter1710 : struct.success) { - oprot.writeString(_iter1702); + oprot.writeString(_iter1710); } } } @@ -98169,13 +98169,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_tables_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1703 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1703.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1704; - for (int _i1705 = 0; _i1705 < _list1703.size; ++_i1705) + org.apache.thrift.protocol.TList _list1711 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1711.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1712; + for (int _i1713 = 0; _i1713 < _list1711.size; ++_i1713) { - _elem1704 = iprot.readString(); - struct.success.add(_elem1704); + _elem1712 = iprot.readString(); + struct.success.add(_elem1712); } } struct.setSuccessIsSet(true); @@ -99641,13 +99641,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_objects_b case 2: // TBL_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1706 = iprot.readListBegin(); - struct.tbl_names = new java.util.ArrayList(_list1706.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1707; - for (int _i1708 = 0; _i1708 < _list1706.size; ++_i1708) + org.apache.thrift.protocol.TList _list1714 = iprot.readListBegin(); + struct.tbl_names = new java.util.ArrayList(_list1714.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1715; + for (int _i1716 = 0; _i1716 < _list1714.size; ++_i1716) { - _elem1707 = iprot.readString(); - struct.tbl_names.add(_elem1707); + _elem1715 = iprot.readString(); + struct.tbl_names.add(_elem1715); } iprot.readListEnd(); } @@ -99678,9 +99678,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_objects_ oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size())); - for (java.lang.String _iter1709 : struct.tbl_names) + for (java.lang.String _iter1717 : struct.tbl_names) { - oprot.writeString(_iter1709); + oprot.writeString(_iter1717); } oprot.writeListEnd(); } @@ -99717,9 +99717,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_objects_b if (struct.isSetTbl_names()) { { oprot.writeI32(struct.tbl_names.size()); - for (java.lang.String _iter1710 : struct.tbl_names) + for (java.lang.String _iter1718 : struct.tbl_names) { - oprot.writeString(_iter1710); + oprot.writeString(_iter1718); } } } @@ -99735,13 +99735,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_objects_by } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list1711 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.tbl_names = new java.util.ArrayList(_list1711.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1712; - for (int _i1713 = 0; _i1713 < _list1711.size; ++_i1713) + org.apache.thrift.protocol.TList _list1719 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.tbl_names = new java.util.ArrayList(_list1719.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1720; + for (int _i1721 = 0; _i1721 < _list1719.size; ++_i1721) { - _elem1712 = iprot.readString(); - struct.tbl_names.add(_elem1712); + _elem1720 = iprot.readString(); + struct.tbl_names.add(_elem1720); } } struct.setTbl_namesIsSet(true); @@ -100071,14 +100071,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_objects_b case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1714 = iprot.readListBegin(); - struct.success = new java.util.ArrayList
(_list1714.size); - @org.apache.thrift.annotation.Nullable Table _elem1715; - for (int _i1716 = 0; _i1716 < _list1714.size; ++_i1716) + org.apache.thrift.protocol.TList _list1722 = iprot.readListBegin(); + struct.success = new java.util.ArrayList
(_list1722.size); + @org.apache.thrift.annotation.Nullable Table _elem1723; + for (int _i1724 = 0; _i1724 < _list1722.size; ++_i1724) { - _elem1715 = new Table(); - _elem1715.read(iprot); - struct.success.add(_elem1715); + _elem1723 = new Table(); + _elem1723.read(iprot); + struct.success.add(_elem1723); } iprot.readListEnd(); } @@ -100104,9 +100104,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_objects_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Table _iter1717 : struct.success) + for (Table _iter1725 : struct.success) { - _iter1717.write(oprot); + _iter1725.write(oprot); } oprot.writeListEnd(); } @@ -100137,9 +100137,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_objects_b if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Table _iter1718 : struct.success) + for (Table _iter1726 : struct.success) { - _iter1718.write(oprot); + _iter1726.write(oprot); } } } @@ -100151,14 +100151,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_objects_by java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1719 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList
(_list1719.size); - @org.apache.thrift.annotation.Nullable Table _elem1720; - for (int _i1721 = 0; _i1721 < _list1719.size; ++_i1721) + org.apache.thrift.protocol.TList _list1727 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList
(_list1727.size); + @org.apache.thrift.annotation.Nullable Table _elem1728; + for (int _i1729 = 0; _i1729 < _list1727.size; ++_i1729) { - _elem1720 = new Table(); - _elem1720.read(iprot); - struct.success.add(_elem1720); + _elem1728 = new Table(); + _elem1728.read(iprot); + struct.success.add(_elem1728); } } struct.setSuccessIsSet(true); @@ -100936,14 +100936,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_ext_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1722 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1722.size); - @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1723; - for (int _i1724 = 0; _i1724 < _list1722.size; ++_i1724) + org.apache.thrift.protocol.TList _list1730 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1730.size); + @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1731; + for (int _i1732 = 0; _i1732 < _list1730.size; ++_i1732) { - _elem1723 = new ExtendedTableInfo(); - _elem1723.read(iprot); - struct.success.add(_elem1723); + _elem1731 = new ExtendedTableInfo(); + _elem1731.read(iprot); + struct.success.add(_elem1731); } iprot.readListEnd(); } @@ -100978,9 +100978,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_ext_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (ExtendedTableInfo _iter1725 : struct.success) + for (ExtendedTableInfo _iter1733 : struct.success) { - _iter1725.write(oprot); + _iter1733.write(oprot); } oprot.writeListEnd(); } @@ -101019,9 +101019,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_ext_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (ExtendedTableInfo _iter1726 : struct.success) + for (ExtendedTableInfo _iter1734 : struct.success) { - _iter1726.write(oprot); + _iter1734.write(oprot); } } } @@ -101036,14 +101036,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_ext_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1727 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1727.size); - @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1728; - for (int _i1729 = 0; _i1729 < _list1727.size; ++_i1729) + org.apache.thrift.protocol.TList _list1735 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1735.size); + @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1736; + for (int _i1737 = 0; _i1737 < _list1735.size; ++_i1737) { - _elem1728 = new ExtendedTableInfo(); - _elem1728.read(iprot); - struct.success.add(_elem1728); + _elem1736 = new ExtendedTableInfo(); + _elem1736.read(iprot); + struct.success.add(_elem1736); } } struct.setSuccessIsSet(true); @@ -106594,13 +106594,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_names_by_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1730 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1730.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1731; - for (int _i1732 = 0; _i1732 < _list1730.size; ++_i1732) + org.apache.thrift.protocol.TList _list1738 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1738.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1739; + for (int _i1740 = 0; _i1740 < _list1738.size; ++_i1740) { - _elem1731 = iprot.readString(); - struct.success.add(_elem1731); + _elem1739 = iprot.readString(); + struct.success.add(_elem1739); } iprot.readListEnd(); } @@ -106653,9 +106653,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_names_by oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1733 : struct.success) + for (java.lang.String _iter1741 : struct.success) { - oprot.writeString(_iter1733); + oprot.writeString(_iter1741); } oprot.writeListEnd(); } @@ -106710,9 +106710,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_names_by_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1734 : struct.success) + for (java.lang.String _iter1742 : struct.success) { - oprot.writeString(_iter1734); + oprot.writeString(_iter1742); } } } @@ -106733,13 +106733,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_names_by_f java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1735 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1735.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1736; - for (int _i1737 = 0; _i1737 < _list1735.size; ++_i1737) + org.apache.thrift.protocol.TList _list1743 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1743.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1744; + for (int _i1745 = 0; _i1745 < _list1743.size; ++_i1745) { - _elem1736 = iprot.readString(); - struct.success.add(_elem1736); + _elem1744 = iprot.readString(); + struct.success.add(_elem1744); } } struct.setSuccessIsSet(true); @@ -113586,14 +113586,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_partitions_args case 1: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1738 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1738.size); - @org.apache.thrift.annotation.Nullable Partition _elem1739; - for (int _i1740 = 0; _i1740 < _list1738.size; ++_i1740) + org.apache.thrift.protocol.TList _list1746 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1746.size); + @org.apache.thrift.annotation.Nullable Partition _elem1747; + for (int _i1748 = 0; _i1748 < _list1746.size; ++_i1748) { - _elem1739 = new Partition(); - _elem1739.read(iprot); - struct.new_parts.add(_elem1739); + _elem1747 = new Partition(); + _elem1747.read(iprot); + struct.new_parts.add(_elem1747); } iprot.readListEnd(); } @@ -113619,9 +113619,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_partitions_arg oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter1741 : struct.new_parts) + for (Partition _iter1749 : struct.new_parts) { - _iter1741.write(oprot); + _iter1749.write(oprot); } oprot.writeListEnd(); } @@ -113652,9 +113652,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_partitions_args if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter1742 : struct.new_parts) + for (Partition _iter1750 : struct.new_parts) { - _iter1742.write(oprot); + _iter1750.write(oprot); } } } @@ -113666,14 +113666,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_partitions_args java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1743 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1743.size); - @org.apache.thrift.annotation.Nullable Partition _elem1744; - for (int _i1745 = 0; _i1745 < _list1743.size; ++_i1745) + org.apache.thrift.protocol.TList _list1751 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1751.size); + @org.apache.thrift.annotation.Nullable Partition _elem1752; + for (int _i1753 = 0; _i1753 < _list1751.size; ++_i1753) { - _elem1744 = new Partition(); - _elem1744.read(iprot); - struct.new_parts.add(_elem1744); + _elem1752 = new Partition(); + _elem1752.read(iprot); + struct.new_parts.add(_elem1752); } } struct.setNew_partsIsSet(true); @@ -114680,14 +114680,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_partitions_pspe case 1: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1746 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1746.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1747; - for (int _i1748 = 0; _i1748 < _list1746.size; ++_i1748) + org.apache.thrift.protocol.TList _list1754 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1754.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1755; + for (int _i1756 = 0; _i1756 < _list1754.size; ++_i1756) { - _elem1747 = new PartitionSpec(); - _elem1747.read(iprot); - struct.new_parts.add(_elem1747); + _elem1755 = new PartitionSpec(); + _elem1755.read(iprot); + struct.new_parts.add(_elem1755); } iprot.readListEnd(); } @@ -114713,9 +114713,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_partitions_psp oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (PartitionSpec _iter1749 : struct.new_parts) + for (PartitionSpec _iter1757 : struct.new_parts) { - _iter1749.write(oprot); + _iter1757.write(oprot); } oprot.writeListEnd(); } @@ -114746,9 +114746,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_partitions_pspe if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (PartitionSpec _iter1750 : struct.new_parts) + for (PartitionSpec _iter1758 : struct.new_parts) { - _iter1750.write(oprot); + _iter1758.write(oprot); } } } @@ -114760,14 +114760,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_partitions_pspec java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1751 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1751.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1752; - for (int _i1753 = 0; _i1753 < _list1751.size; ++_i1753) + org.apache.thrift.protocol.TList _list1759 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1759.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1760; + for (int _i1761 = 0; _i1761 < _list1759.size; ++_i1761) { - _elem1752 = new PartitionSpec(); - _elem1752.read(iprot); - struct.new_parts.add(_elem1752); + _elem1760 = new PartitionSpec(); + _elem1760.read(iprot); + struct.new_parts.add(_elem1760); } } struct.setNew_partsIsSet(true); @@ -115949,13 +115949,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, append_partition_ar case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1754 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1754.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1755; - for (int _i1756 = 0; _i1756 < _list1754.size; ++_i1756) + org.apache.thrift.protocol.TList _list1762 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1762.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1763; + for (int _i1764 = 0; _i1764 < _list1762.size; ++_i1764) { - _elem1755 = iprot.readString(); - struct.part_vals.add(_elem1755); + _elem1763 = iprot.readString(); + struct.part_vals.add(_elem1763); } iprot.readListEnd(); } @@ -115991,9 +115991,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, append_partition_a oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1757 : struct.part_vals) + for (java.lang.String _iter1765 : struct.part_vals) { - oprot.writeString(_iter1757); + oprot.writeString(_iter1765); } oprot.writeListEnd(); } @@ -116036,9 +116036,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, append_partition_ar if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1758 : struct.part_vals) + for (java.lang.String _iter1766 : struct.part_vals) { - oprot.writeString(_iter1758); + oprot.writeString(_iter1766); } } } @@ -116058,13 +116058,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, append_partition_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1759 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1759.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1760; - for (int _i1761 = 0; _i1761 < _list1759.size; ++_i1761) + org.apache.thrift.protocol.TList _list1767 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1767.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1768; + for (int _i1769 = 0; _i1769 < _list1767.size; ++_i1769) { - _elem1760 = iprot.readString(); - struct.part_vals.add(_elem1760); + _elem1768 = iprot.readString(); + struct.part_vals.add(_elem1768); } } struct.setPart_valsIsSet(true); @@ -118390,13 +118390,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, append_partition_wi case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1762 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1762.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1763; - for (int _i1764 = 0; _i1764 < _list1762.size; ++_i1764) + org.apache.thrift.protocol.TList _list1770 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1770.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1771; + for (int _i1772 = 0; _i1772 < _list1770.size; ++_i1772) { - _elem1763 = iprot.readString(); - struct.part_vals.add(_elem1763); + _elem1771 = iprot.readString(); + struct.part_vals.add(_elem1771); } iprot.readListEnd(); } @@ -118441,9 +118441,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, append_partition_w oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1765 : struct.part_vals) + for (java.lang.String _iter1773 : struct.part_vals) { - oprot.writeString(_iter1765); + oprot.writeString(_iter1773); } oprot.writeListEnd(); } @@ -118494,9 +118494,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, append_partition_wi if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1766 : struct.part_vals) + for (java.lang.String _iter1774 : struct.part_vals) { - oprot.writeString(_iter1766); + oprot.writeString(_iter1774); } } } @@ -118519,13 +118519,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, append_partition_wit } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1767 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1767.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1768; - for (int _i1769 = 0; _i1769 < _list1767.size; ++_i1769) + org.apache.thrift.protocol.TList _list1775 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1775.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1776; + for (int _i1777 = 0; _i1777 < _list1775.size; ++_i1777) { - _elem1768 = iprot.readString(); - struct.part_vals.add(_elem1768); + _elem1776 = iprot.readString(); + struct.part_vals.add(_elem1776); } } struct.setPart_valsIsSet(true); @@ -122417,13 +122417,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, drop_partition_args case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1770 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1770.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1771; - for (int _i1772 = 0; _i1772 < _list1770.size; ++_i1772) + org.apache.thrift.protocol.TList _list1778 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1778.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1779; + for (int _i1780 = 0; _i1780 < _list1778.size; ++_i1780) { - _elem1771 = iprot.readString(); - struct.part_vals.add(_elem1771); + _elem1779 = iprot.readString(); + struct.part_vals.add(_elem1779); } iprot.readListEnd(); } @@ -122467,9 +122467,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, drop_partition_arg oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1773 : struct.part_vals) + for (java.lang.String _iter1781 : struct.part_vals) { - oprot.writeString(_iter1773); + oprot.writeString(_iter1781); } oprot.writeListEnd(); } @@ -122518,9 +122518,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, drop_partition_args if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1774 : struct.part_vals) + for (java.lang.String _iter1782 : struct.part_vals) { - oprot.writeString(_iter1774); + oprot.writeString(_iter1782); } } } @@ -122543,13 +122543,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, drop_partition_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1775 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1775.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1776; - for (int _i1777 = 0; _i1777 < _list1775.size; ++_i1777) + org.apache.thrift.protocol.TList _list1783 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1783.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1784; + for (int _i1785 = 0; _i1785 < _list1783.size; ++_i1785) { - _elem1776 = iprot.readString(); - struct.part_vals.add(_elem1776); + _elem1784 = iprot.readString(); + struct.part_vals.add(_elem1784); } } struct.setPart_valsIsSet(true); @@ -123791,13 +123791,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, drop_partition_with case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1778 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1778.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1779; - for (int _i1780 = 0; _i1780 < _list1778.size; ++_i1780) + org.apache.thrift.protocol.TList _list1786 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1786.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1787; + for (int _i1788 = 0; _i1788 < _list1786.size; ++_i1788) { - _elem1779 = iprot.readString(); - struct.part_vals.add(_elem1779); + _elem1787 = iprot.readString(); + struct.part_vals.add(_elem1787); } iprot.readListEnd(); } @@ -123850,9 +123850,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, drop_partition_wit oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1781 : struct.part_vals) + for (java.lang.String _iter1789 : struct.part_vals) { - oprot.writeString(_iter1781); + oprot.writeString(_iter1789); } oprot.writeListEnd(); } @@ -123909,9 +123909,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, drop_partition_with if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1782 : struct.part_vals) + for (java.lang.String _iter1790 : struct.part_vals) { - oprot.writeString(_iter1782); + oprot.writeString(_iter1790); } } } @@ -123937,13 +123937,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, drop_partition_with_ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1783 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1783.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1784; - for (int _i1785 = 0; _i1785 < _list1783.size; ++_i1785) + org.apache.thrift.protocol.TList _list1791 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1791.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1792; + for (int _i1793 = 0; _i1793 < _list1791.size; ++_i1793) { - _elem1784 = iprot.readString(); - struct.part_vals.add(_elem1784); + _elem1792 = iprot.readString(); + struct.part_vals.add(_elem1792); } } struct.setPart_valsIsSet(true); @@ -128563,13 +128563,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_args case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1786 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1786.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1787; - for (int _i1788 = 0; _i1788 < _list1786.size; ++_i1788) + org.apache.thrift.protocol.TList _list1794 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1794.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1795; + for (int _i1796 = 0; _i1796 < _list1794.size; ++_i1796) { - _elem1787 = iprot.readString(); - struct.part_vals.add(_elem1787); + _elem1795 = iprot.readString(); + struct.part_vals.add(_elem1795); } iprot.readListEnd(); } @@ -128605,9 +128605,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_args oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1789 : struct.part_vals) + for (java.lang.String _iter1797 : struct.part_vals) { - oprot.writeString(_iter1789); + oprot.writeString(_iter1797); } oprot.writeListEnd(); } @@ -128650,9 +128650,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_args if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1790 : struct.part_vals) + for (java.lang.String _iter1798 : struct.part_vals) { - oprot.writeString(_iter1790); + oprot.writeString(_iter1798); } } } @@ -128672,13 +128672,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_args s } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1791 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1791.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1792; - for (int _i1793 = 0; _i1793 < _list1791.size; ++_i1793) + org.apache.thrift.protocol.TList _list1799 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1799.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1800; + for (int _i1801 = 0; _i1801 < _list1799.size; ++_i1801) { - _elem1792 = iprot.readString(); - struct.part_vals.add(_elem1792); + _elem1800 = iprot.readString(); + struct.part_vals.add(_elem1800); } } struct.setPart_valsIsSet(true); @@ -130850,15 +130850,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partition_ case 1: // PARTITION_SPECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1794 = iprot.readMapBegin(); - struct.partitionSpecs = new java.util.HashMap(2*_map1794.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1795; - @org.apache.thrift.annotation.Nullable java.lang.String _val1796; - for (int _i1797 = 0; _i1797 < _map1794.size; ++_i1797) + org.apache.thrift.protocol.TMap _map1802 = iprot.readMapBegin(); + struct.partitionSpecs = new java.util.HashMap(2*_map1802.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1803; + @org.apache.thrift.annotation.Nullable java.lang.String _val1804; + for (int _i1805 = 0; _i1805 < _map1802.size; ++_i1805) { - _key1795 = iprot.readString(); - _val1796 = iprot.readString(); - struct.partitionSpecs.put(_key1795, _val1796); + _key1803 = iprot.readString(); + _val1804 = iprot.readString(); + struct.partitionSpecs.put(_key1803, _val1804); } iprot.readMapEnd(); } @@ -130916,10 +130916,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size())); - for (java.util.Map.Entry _iter1798 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1806 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1798.getKey()); - oprot.writeString(_iter1798.getValue()); + oprot.writeString(_iter1806.getKey()); + oprot.writeString(_iter1806.getValue()); } oprot.writeMapEnd(); } @@ -130982,10 +130982,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partition_ if (struct.isSetPartitionSpecs()) { { oprot.writeI32(struct.partitionSpecs.size()); - for (java.util.Map.Entry _iter1799 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1807 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1799.getKey()); - oprot.writeString(_iter1799.getValue()); + oprot.writeString(_iter1807.getKey()); + oprot.writeString(_iter1807.getValue()); } } } @@ -131009,15 +131009,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partition_a java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map1800 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.partitionSpecs = new java.util.HashMap(2*_map1800.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1801; - @org.apache.thrift.annotation.Nullable java.lang.String _val1802; - for (int _i1803 = 0; _i1803 < _map1800.size; ++_i1803) + org.apache.thrift.protocol.TMap _map1808 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.partitionSpecs = new java.util.HashMap(2*_map1808.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1809; + @org.apache.thrift.annotation.Nullable java.lang.String _val1810; + for (int _i1811 = 0; _i1811 < _map1808.size; ++_i1811) { - _key1801 = iprot.readString(); - _val1802 = iprot.readString(); - struct.partitionSpecs.put(_key1801, _val1802); + _key1809 = iprot.readString(); + _val1810 = iprot.readString(); + struct.partitionSpecs.put(_key1809, _val1810); } } struct.setPartitionSpecsIsSet(true); @@ -132471,15 +132471,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partitions case 1: // PARTITION_SPECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map1804 = iprot.readMapBegin(); - struct.partitionSpecs = new java.util.HashMap(2*_map1804.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1805; - @org.apache.thrift.annotation.Nullable java.lang.String _val1806; - for (int _i1807 = 0; _i1807 < _map1804.size; ++_i1807) + org.apache.thrift.protocol.TMap _map1812 = iprot.readMapBegin(); + struct.partitionSpecs = new java.util.HashMap(2*_map1812.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1813; + @org.apache.thrift.annotation.Nullable java.lang.String _val1814; + for (int _i1815 = 0; _i1815 < _map1812.size; ++_i1815) { - _key1805 = iprot.readString(); - _val1806 = iprot.readString(); - struct.partitionSpecs.put(_key1805, _val1806); + _key1813 = iprot.readString(); + _val1814 = iprot.readString(); + struct.partitionSpecs.put(_key1813, _val1814); } iprot.readMapEnd(); } @@ -132537,10 +132537,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size())); - for (java.util.Map.Entry _iter1808 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1816 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1808.getKey()); - oprot.writeString(_iter1808.getValue()); + oprot.writeString(_iter1816.getKey()); + oprot.writeString(_iter1816.getValue()); } oprot.writeMapEnd(); } @@ -132603,10 +132603,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partitions if (struct.isSetPartitionSpecs()) { { oprot.writeI32(struct.partitionSpecs.size()); - for (java.util.Map.Entry _iter1809 : struct.partitionSpecs.entrySet()) + for (java.util.Map.Entry _iter1817 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter1809.getKey()); - oprot.writeString(_iter1809.getValue()); + oprot.writeString(_iter1817.getKey()); + oprot.writeString(_iter1817.getValue()); } } } @@ -132630,15 +132630,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partitions_ java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map1810 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.partitionSpecs = new java.util.HashMap(2*_map1810.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key1811; - @org.apache.thrift.annotation.Nullable java.lang.String _val1812; - for (int _i1813 = 0; _i1813 < _map1810.size; ++_i1813) + org.apache.thrift.protocol.TMap _map1818 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.partitionSpecs = new java.util.HashMap(2*_map1818.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key1819; + @org.apache.thrift.annotation.Nullable java.lang.String _val1820; + for (int _i1821 = 0; _i1821 < _map1818.size; ++_i1821) { - _key1811 = iprot.readString(); - _val1812 = iprot.readString(); - struct.partitionSpecs.put(_key1811, _val1812); + _key1819 = iprot.readString(); + _val1820 = iprot.readString(); + struct.partitionSpecs.put(_key1819, _val1820); } } struct.setPartitionSpecsIsSet(true); @@ -133308,14 +133308,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partitions case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1814 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1814.size); - @org.apache.thrift.annotation.Nullable Partition _elem1815; - for (int _i1816 = 0; _i1816 < _list1814.size; ++_i1816) + org.apache.thrift.protocol.TList _list1822 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1822.size); + @org.apache.thrift.annotation.Nullable Partition _elem1823; + for (int _i1824 = 0; _i1824 < _list1822.size; ++_i1824) { - _elem1815 = new Partition(); - _elem1815.read(iprot); - struct.success.add(_elem1815); + _elem1823 = new Partition(); + _elem1823.read(iprot); + struct.success.add(_elem1823); } iprot.readListEnd(); } @@ -133377,9 +133377,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1817 : struct.success) + for (Partition _iter1825 : struct.success) { - _iter1817.write(oprot); + _iter1825.write(oprot); } oprot.writeListEnd(); } @@ -133442,9 +133442,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partitions if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1818 : struct.success) + for (Partition _iter1826 : struct.success) { - _iter1818.write(oprot); + _iter1826.write(oprot); } } } @@ -133468,14 +133468,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partitions_ java.util.BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1819 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1819.size); - @org.apache.thrift.annotation.Nullable Partition _elem1820; - for (int _i1821 = 0; _i1821 < _list1819.size; ++_i1821) + org.apache.thrift.protocol.TList _list1827 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1827.size); + @org.apache.thrift.annotation.Nullable Partition _elem1828; + for (int _i1829 = 0; _i1829 < _list1827.size; ++_i1829) { - _elem1820 = new Partition(); - _elem1820.read(iprot); - struct.success.add(_elem1820); + _elem1828 = new Partition(); + _elem1828.read(iprot); + struct.success.add(_elem1828); } } struct.setSuccessIsSet(true); @@ -134180,13 +134180,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_with_ case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1822 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1822.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1823; - for (int _i1824 = 0; _i1824 < _list1822.size; ++_i1824) + org.apache.thrift.protocol.TList _list1830 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1830.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1831; + for (int _i1832 = 0; _i1832 < _list1830.size; ++_i1832) { - _elem1823 = iprot.readString(); - struct.part_vals.add(_elem1823); + _elem1831 = iprot.readString(); + struct.part_vals.add(_elem1831); } iprot.readListEnd(); } @@ -134206,13 +134206,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_with_ case 5: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1825 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list1825.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1826; - for (int _i1827 = 0; _i1827 < _list1825.size; ++_i1827) + org.apache.thrift.protocol.TList _list1833 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list1833.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1834; + for (int _i1835 = 0; _i1835 < _list1833.size; ++_i1835) { - _elem1826 = iprot.readString(); - struct.group_names.add(_elem1826); + _elem1834 = iprot.readString(); + struct.group_names.add(_elem1834); } iprot.readListEnd(); } @@ -134248,9 +134248,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_with oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1828 : struct.part_vals) + for (java.lang.String _iter1836 : struct.part_vals) { - oprot.writeString(_iter1828); + oprot.writeString(_iter1836); } oprot.writeListEnd(); } @@ -134265,9 +134265,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_with oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter1829 : struct.group_names) + for (java.lang.String _iter1837 : struct.group_names) { - oprot.writeString(_iter1829); + oprot.writeString(_iter1837); } oprot.writeListEnd(); } @@ -134316,9 +134316,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_with_ if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1830 : struct.part_vals) + for (java.lang.String _iter1838 : struct.part_vals) { - oprot.writeString(_iter1830); + oprot.writeString(_iter1838); } } } @@ -134328,9 +134328,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_with_ if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter1831 : struct.group_names) + for (java.lang.String _iter1839 : struct.group_names) { - oprot.writeString(_iter1831); + oprot.writeString(_iter1839); } } } @@ -134350,13 +134350,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_with_a } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1832 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1832.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1833; - for (int _i1834 = 0; _i1834 < _list1832.size; ++_i1834) + org.apache.thrift.protocol.TList _list1840 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1840.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1841; + for (int _i1842 = 0; _i1842 < _list1840.size; ++_i1842) { - _elem1833 = iprot.readString(); - struct.part_vals.add(_elem1833); + _elem1841 = iprot.readString(); + struct.part_vals.add(_elem1841); } } struct.setPart_valsIsSet(true); @@ -134367,13 +134367,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_with_a } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1835 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list1835.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1836; - for (int _i1837 = 0; _i1837 < _list1835.size; ++_i1837) + org.apache.thrift.protocol.TList _list1843 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list1843.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1844; + for (int _i1845 = 0; _i1845 < _list1843.size; ++_i1845) { - _elem1836 = iprot.readString(); - struct.group_names.add(_elem1836); + _elem1844 = iprot.readString(); + struct.group_names.add(_elem1844); } } struct.setGroup_namesIsSet(true); @@ -137160,14 +137160,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1838 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1838.size); - @org.apache.thrift.annotation.Nullable Partition _elem1839; - for (int _i1840 = 0; _i1840 < _list1838.size; ++_i1840) + org.apache.thrift.protocol.TList _list1846 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1846.size); + @org.apache.thrift.annotation.Nullable Partition _elem1847; + for (int _i1848 = 0; _i1848 < _list1846.size; ++_i1848) { - _elem1839 = new Partition(); - _elem1839.read(iprot); - struct.success.add(_elem1839); + _elem1847 = new Partition(); + _elem1847.read(iprot); + struct.success.add(_elem1847); } iprot.readListEnd(); } @@ -137211,9 +137211,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1841 : struct.success) + for (Partition _iter1849 : struct.success) { - _iter1841.write(oprot); + _iter1849.write(oprot); } oprot.writeListEnd(); } @@ -137260,9 +137260,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1842 : struct.success) + for (Partition _iter1850 : struct.success) { - _iter1842.write(oprot); + _iter1850.write(oprot); } } } @@ -137280,14 +137280,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_resul java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1843 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1843.size); - @org.apache.thrift.annotation.Nullable Partition _elem1844; - for (int _i1845 = 0; _i1845 < _list1843.size; ++_i1845) + org.apache.thrift.protocol.TList _list1851 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1851.size); + @org.apache.thrift.annotation.Nullable Partition _elem1852; + for (int _i1853 = 0; _i1853 < _list1851.size; ++_i1853) { - _elem1844 = new Partition(); - _elem1844.read(iprot); - struct.success.add(_elem1844); + _elem1852 = new Partition(); + _elem1852.read(iprot); + struct.success.add(_elem1852); } } struct.setSuccessIsSet(true); @@ -138925,13 +138925,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_with case 5: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1846 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list1846.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1847; - for (int _i1848 = 0; _i1848 < _list1846.size; ++_i1848) + org.apache.thrift.protocol.TList _list1854 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list1854.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1855; + for (int _i1856 = 0; _i1856 < _list1854.size; ++_i1856) { - _elem1847 = iprot.readString(); - struct.group_names.add(_elem1847); + _elem1855 = iprot.readString(); + struct.group_names.add(_elem1855); } iprot.readListEnd(); } @@ -138975,9 +138975,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_wit oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter1849 : struct.group_names) + for (java.lang.String _iter1857 : struct.group_names) { - oprot.writeString(_iter1849); + oprot.writeString(_iter1857); } oprot.writeListEnd(); } @@ -139032,9 +139032,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_with if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter1850 : struct.group_names) + for (java.lang.String _iter1858 : struct.group_names) { - oprot.writeString(_iter1850); + oprot.writeString(_iter1858); } } } @@ -139062,13 +139062,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_with_ } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list1851 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list1851.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1852; - for (int _i1853 = 0; _i1853 < _list1851.size; ++_i1853) + org.apache.thrift.protocol.TList _list1859 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list1859.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1860; + for (int _i1861 = 0; _i1861 < _list1859.size; ++_i1861) { - _elem1852 = iprot.readString(); - struct.group_names.add(_elem1852); + _elem1860 = iprot.readString(); + struct.group_names.add(_elem1860); } } struct.setGroup_namesIsSet(true); @@ -139560,14 +139560,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_with case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1854 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1854.size); - @org.apache.thrift.annotation.Nullable Partition _elem1855; - for (int _i1856 = 0; _i1856 < _list1854.size; ++_i1856) + org.apache.thrift.protocol.TList _list1862 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1862.size); + @org.apache.thrift.annotation.Nullable Partition _elem1863; + for (int _i1864 = 0; _i1864 < _list1862.size; ++_i1864) { - _elem1855 = new Partition(); - _elem1855.read(iprot); - struct.success.add(_elem1855); + _elem1863 = new Partition(); + _elem1863.read(iprot); + struct.success.add(_elem1863); } iprot.readListEnd(); } @@ -139611,9 +139611,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_wit oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1857 : struct.success) + for (Partition _iter1865 : struct.success) { - _iter1857.write(oprot); + _iter1865.write(oprot); } oprot.writeListEnd(); } @@ -139660,9 +139660,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_with if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1858 : struct.success) + for (Partition _iter1866 : struct.success) { - _iter1858.write(oprot); + _iter1866.write(oprot); } } } @@ -139680,14 +139680,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_with_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1859 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1859.size); - @org.apache.thrift.annotation.Nullable Partition _elem1860; - for (int _i1861 = 0; _i1861 < _list1859.size; ++_i1861) + org.apache.thrift.protocol.TList _list1867 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1867.size); + @org.apache.thrift.annotation.Nullable Partition _elem1868; + for (int _i1869 = 0; _i1869 < _list1867.size; ++_i1869) { - _elem1860 = new Partition(); - _elem1860.read(iprot); - struct.success.add(_elem1860); + _elem1868 = new Partition(); + _elem1868.read(iprot); + struct.success.add(_elem1868); } } struct.setSuccessIsSet(true); @@ -140756,14 +140756,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_pspe case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1862 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1862.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1863; - for (int _i1864 = 0; _i1864 < _list1862.size; ++_i1864) + org.apache.thrift.protocol.TList _list1870 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1870.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1871; + for (int _i1872 = 0; _i1872 < _list1870.size; ++_i1872) { - _elem1863 = new PartitionSpec(); - _elem1863.read(iprot); - struct.success.add(_elem1863); + _elem1871 = new PartitionSpec(); + _elem1871.read(iprot); + struct.success.add(_elem1871); } iprot.readListEnd(); } @@ -140807,9 +140807,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_psp oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (PartitionSpec _iter1865 : struct.success) + for (PartitionSpec _iter1873 : struct.success) { - _iter1865.write(oprot); + _iter1873.write(oprot); } oprot.writeListEnd(); } @@ -140856,9 +140856,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_pspe if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (PartitionSpec _iter1866 : struct.success) + for (PartitionSpec _iter1874 : struct.success) { - _iter1866.write(oprot); + _iter1874.write(oprot); } } } @@ -140876,14 +140876,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_pspec java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1867 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1867.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1868; - for (int _i1869 = 0; _i1869 < _list1867.size; ++_i1869) + org.apache.thrift.protocol.TList _list1875 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1875.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1876; + for (int _i1877 = 0; _i1877 < _list1875.size; ++_i1877) { - _elem1868 = new PartitionSpec(); - _elem1868.read(iprot); - struct.success.add(_elem1868); + _elem1876 = new PartitionSpec(); + _elem1876.read(iprot); + struct.success.add(_elem1876); } } struct.setSuccessIsSet(true); @@ -141949,13 +141949,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1870 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1870.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1871; - for (int _i1872 = 0; _i1872 < _list1870.size; ++_i1872) + org.apache.thrift.protocol.TList _list1878 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1878.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1879; + for (int _i1880 = 0; _i1880 < _list1878.size; ++_i1880) { - _elem1871 = iprot.readString(); - struct.success.add(_elem1871); + _elem1879 = iprot.readString(); + struct.success.add(_elem1879); } iprot.readListEnd(); } @@ -141999,9 +141999,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1873 : struct.success) + for (java.lang.String _iter1881 : struct.success) { - oprot.writeString(_iter1873); + oprot.writeString(_iter1881); } oprot.writeListEnd(); } @@ -142048,9 +142048,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1874 : struct.success) + for (java.lang.String _iter1882 : struct.success) { - oprot.writeString(_iter1874); + oprot.writeString(_iter1882); } } } @@ -142068,13 +142068,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1875 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1875.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1876; - for (int _i1877 = 0; _i1877 < _list1875.size; ++_i1877) + org.apache.thrift.protocol.TList _list1883 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1883.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1884; + for (int _i1885 = 0; _i1885 < _list1883.size; ++_i1885) { - _elem1876 = iprot.readString(); - struct.success.add(_elem1876); + _elem1884 = iprot.readString(); + struct.success.add(_elem1884); } } struct.setSuccessIsSet(true); @@ -143615,13 +143615,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_a case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1878 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1878.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1879; - for (int _i1880 = 0; _i1880 < _list1878.size; ++_i1880) + org.apache.thrift.protocol.TList _list1886 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1886.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1887; + for (int _i1888 = 0; _i1888 < _list1886.size; ++_i1888) { - _elem1879 = iprot.readString(); - struct.part_vals.add(_elem1879); + _elem1887 = iprot.readString(); + struct.part_vals.add(_elem1887); } iprot.readListEnd(); } @@ -143665,9 +143665,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1881 : struct.part_vals) + for (java.lang.String _iter1889 : struct.part_vals) { - oprot.writeString(_iter1881); + oprot.writeString(_iter1889); } oprot.writeListEnd(); } @@ -143716,9 +143716,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_a if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1882 : struct.part_vals) + for (java.lang.String _iter1890 : struct.part_vals) { - oprot.writeString(_iter1882); + oprot.writeString(_iter1890); } } } @@ -143741,13 +143741,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1883 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1883.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1884; - for (int _i1885 = 0; _i1885 < _list1883.size; ++_i1885) + org.apache.thrift.protocol.TList _list1891 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1891.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1892; + for (int _i1893 = 0; _i1893 < _list1891.size; ++_i1893) { - _elem1884 = iprot.readString(); - struct.part_vals.add(_elem1884); + _elem1892 = iprot.readString(); + struct.part_vals.add(_elem1892); } } struct.setPart_valsIsSet(true); @@ -144243,14 +144243,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1886 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1886.size); - @org.apache.thrift.annotation.Nullable Partition _elem1887; - for (int _i1888 = 0; _i1888 < _list1886.size; ++_i1888) + org.apache.thrift.protocol.TList _list1894 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1894.size); + @org.apache.thrift.annotation.Nullable Partition _elem1895; + for (int _i1896 = 0; _i1896 < _list1894.size; ++_i1896) { - _elem1887 = new Partition(); - _elem1887.read(iprot); - struct.success.add(_elem1887); + _elem1895 = new Partition(); + _elem1895.read(iprot); + struct.success.add(_elem1895); } iprot.readListEnd(); } @@ -144294,9 +144294,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1889 : struct.success) + for (Partition _iter1897 : struct.success) { - _iter1889.write(oprot); + _iter1897.write(oprot); } oprot.writeListEnd(); } @@ -144343,9 +144343,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1890 : struct.success) + for (Partition _iter1898 : struct.success) { - _iter1890.write(oprot); + _iter1898.write(oprot); } } } @@ -144363,14 +144363,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_re java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1891 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1891.size); - @org.apache.thrift.annotation.Nullable Partition _elem1892; - for (int _i1893 = 0; _i1893 < _list1891.size; ++_i1893) + org.apache.thrift.protocol.TList _list1899 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1899.size); + @org.apache.thrift.annotation.Nullable Partition _elem1900; + for (int _i1901 = 0; _i1901 < _list1899.size; ++_i1901) { - _elem1892 = new Partition(); - _elem1892.read(iprot); - struct.success.add(_elem1892); + _elem1900 = new Partition(); + _elem1900.read(iprot); + struct.success.add(_elem1900); } } struct.setSuccessIsSet(true); @@ -145145,13 +145145,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1894 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1894.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1895; - for (int _i1896 = 0; _i1896 < _list1894.size; ++_i1896) + org.apache.thrift.protocol.TList _list1902 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1902.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1903; + for (int _i1904 = 0; _i1904 < _list1902.size; ++_i1904) { - _elem1895 = iprot.readString(); - struct.part_vals.add(_elem1895); + _elem1903 = iprot.readString(); + struct.part_vals.add(_elem1903); } iprot.readListEnd(); } @@ -145179,13 +145179,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 6: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1897 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list1897.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1898; - for (int _i1899 = 0; _i1899 < _list1897.size; ++_i1899) + org.apache.thrift.protocol.TList _list1905 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list1905.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1906; + for (int _i1907 = 0; _i1907 < _list1905.size; ++_i1907) { - _elem1898 = iprot.readString(); - struct.group_names.add(_elem1898); + _elem1906 = iprot.readString(); + struct.group_names.add(_elem1906); } iprot.readListEnd(); } @@ -145221,9 +145221,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1900 : struct.part_vals) + for (java.lang.String _iter1908 : struct.part_vals) { - oprot.writeString(_iter1900); + oprot.writeString(_iter1908); } oprot.writeListEnd(); } @@ -145241,9 +145241,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter1901 : struct.group_names) + for (java.lang.String _iter1909 : struct.group_names) { - oprot.writeString(_iter1901); + oprot.writeString(_iter1909); } oprot.writeListEnd(); } @@ -145295,9 +145295,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1902 : struct.part_vals) + for (java.lang.String _iter1910 : struct.part_vals) { - oprot.writeString(_iter1902); + oprot.writeString(_iter1910); } } } @@ -145310,9 +145310,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter1903 : struct.group_names) + for (java.lang.String _iter1911 : struct.group_names) { - oprot.writeString(_iter1903); + oprot.writeString(_iter1911); } } } @@ -145332,13 +145332,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1904 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1904.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1905; - for (int _i1906 = 0; _i1906 < _list1904.size; ++_i1906) + org.apache.thrift.protocol.TList _list1912 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1912.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1913; + for (int _i1914 = 0; _i1914 < _list1912.size; ++_i1914) { - _elem1905 = iprot.readString(); - struct.part_vals.add(_elem1905); + _elem1913 = iprot.readString(); + struct.part_vals.add(_elem1913); } } struct.setPart_valsIsSet(true); @@ -145353,13 +145353,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list1907 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list1907.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1908; - for (int _i1909 = 0; _i1909 < _list1907.size; ++_i1909) + org.apache.thrift.protocol.TList _list1915 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list1915.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1916; + for (int _i1917 = 0; _i1917 < _list1915.size; ++_i1917) { - _elem1908 = iprot.readString(); - struct.group_names.add(_elem1908); + _elem1916 = iprot.readString(); + struct.group_names.add(_elem1916); } } struct.setGroup_namesIsSet(true); @@ -145851,14 +145851,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1910 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1910.size); - @org.apache.thrift.annotation.Nullable Partition _elem1911; - for (int _i1912 = 0; _i1912 < _list1910.size; ++_i1912) + org.apache.thrift.protocol.TList _list1918 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1918.size); + @org.apache.thrift.annotation.Nullable Partition _elem1919; + for (int _i1920 = 0; _i1920 < _list1918.size; ++_i1920) { - _elem1911 = new Partition(); - _elem1911.read(iprot); - struct.success.add(_elem1911); + _elem1919 = new Partition(); + _elem1919.read(iprot); + struct.success.add(_elem1919); } iprot.readListEnd(); } @@ -145902,9 +145902,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1913 : struct.success) + for (Partition _iter1921 : struct.success) { - _iter1913.write(oprot); + _iter1921.write(oprot); } oprot.writeListEnd(); } @@ -145951,9 +145951,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1914 : struct.success) + for (Partition _iter1922 : struct.success) { - _iter1914.write(oprot); + _iter1922.write(oprot); } } } @@ -145971,14 +145971,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1915 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1915.size); - @org.apache.thrift.annotation.Nullable Partition _elem1916; - for (int _i1917 = 0; _i1917 < _list1915.size; ++_i1917) + org.apache.thrift.protocol.TList _list1923 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1923.size); + @org.apache.thrift.annotation.Nullable Partition _elem1924; + for (int _i1925 = 0; _i1925 < _list1923.size; ++_i1925) { - _elem1916 = new Partition(); - _elem1916.read(iprot); - struct.success.add(_elem1916); + _elem1924 = new Partition(); + _elem1924.read(iprot); + struct.success.add(_elem1924); } } struct.setSuccessIsSet(true); @@ -147519,13 +147519,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1918 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1918.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1919; - for (int _i1920 = 0; _i1920 < _list1918.size; ++_i1920) + org.apache.thrift.protocol.TList _list1926 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list1926.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1927; + for (int _i1928 = 0; _i1928 < _list1926.size; ++_i1928) { - _elem1919 = iprot.readString(); - struct.part_vals.add(_elem1919); + _elem1927 = iprot.readString(); + struct.part_vals.add(_elem1927); } iprot.readListEnd(); } @@ -147569,9 +147569,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter1921 : struct.part_vals) + for (java.lang.String _iter1929 : struct.part_vals) { - oprot.writeString(_iter1921); + oprot.writeString(_iter1929); } oprot.writeListEnd(); } @@ -147620,9 +147620,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter1922 : struct.part_vals) + for (java.lang.String _iter1930 : struct.part_vals) { - oprot.writeString(_iter1922); + oprot.writeString(_iter1930); } } } @@ -147645,13 +147645,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1923 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list1923.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1924; - for (int _i1925 = 0; _i1925 < _list1923.size; ++_i1925) + org.apache.thrift.protocol.TList _list1931 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list1931.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1932; + for (int _i1933 = 0; _i1933 < _list1931.size; ++_i1933) { - _elem1924 = iprot.readString(); - struct.part_vals.add(_elem1924); + _elem1932 = iprot.readString(); + struct.part_vals.add(_elem1932); } } struct.setPart_valsIsSet(true); @@ -148144,13 +148144,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1926 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1926.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1927; - for (int _i1928 = 0; _i1928 < _list1926.size; ++_i1928) + org.apache.thrift.protocol.TList _list1934 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1934.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1935; + for (int _i1936 = 0; _i1936 < _list1934.size; ++_i1936) { - _elem1927 = iprot.readString(); - struct.success.add(_elem1927); + _elem1935 = iprot.readString(); + struct.success.add(_elem1935); } iprot.readListEnd(); } @@ -148194,9 +148194,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1929 : struct.success) + for (java.lang.String _iter1937 : struct.success) { - oprot.writeString(_iter1929); + oprot.writeString(_iter1937); } oprot.writeListEnd(); } @@ -148243,9 +148243,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1930 : struct.success) + for (java.lang.String _iter1938 : struct.success) { - oprot.writeString(_iter1930); + oprot.writeString(_iter1938); } } } @@ -148263,13 +148263,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1931 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1931.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1932; - for (int _i1933 = 0; _i1933 < _list1931.size; ++_i1933) + org.apache.thrift.protocol.TList _list1939 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1939.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1940; + for (int _i1941 = 0; _i1941 < _list1939.size; ++_i1941) { - _elem1932 = iprot.readString(); - struct.success.add(_elem1932); + _elem1940 = iprot.readString(); + struct.success.add(_elem1940); } } struct.setSuccessIsSet(true); @@ -150081,13 +150081,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1934 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1934.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1935; - for (int _i1936 = 0; _i1936 < _list1934.size; ++_i1936) + org.apache.thrift.protocol.TList _list1942 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1942.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1943; + for (int _i1944 = 0; _i1944 < _list1942.size; ++_i1944) { - _elem1935 = iprot.readString(); - struct.success.add(_elem1935); + _elem1943 = iprot.readString(); + struct.success.add(_elem1943); } iprot.readListEnd(); } @@ -150131,9 +150131,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter1937 : struct.success) + for (java.lang.String _iter1945 : struct.success) { - oprot.writeString(_iter1937); + oprot.writeString(_iter1945); } oprot.writeListEnd(); } @@ -150180,9 +150180,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter1938 : struct.success) + for (java.lang.String _iter1946 : struct.success) { - oprot.writeString(_iter1938); + oprot.writeString(_iter1946); } } } @@ -150200,13 +150200,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1939 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list1939.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1940; - for (int _i1941 = 0; _i1941 < _list1939.size; ++_i1941) + org.apache.thrift.protocol.TList _list1947 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list1947.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1948; + for (int _i1949 = 0; _i1949 < _list1947.size; ++_i1949) { - _elem1940 = iprot.readString(); - struct.success.add(_elem1940); + _elem1948 = iprot.readString(); + struct.success.add(_elem1948); } } struct.setSuccessIsSet(true); @@ -151379,14 +151379,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1942 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1942.size); - @org.apache.thrift.annotation.Nullable Partition _elem1943; - for (int _i1944 = 0; _i1944 < _list1942.size; ++_i1944) + org.apache.thrift.protocol.TList _list1950 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1950.size); + @org.apache.thrift.annotation.Nullable Partition _elem1951; + for (int _i1952 = 0; _i1952 < _list1950.size; ++_i1952) { - _elem1943 = new Partition(); - _elem1943.read(iprot); - struct.success.add(_elem1943); + _elem1951 = new Partition(); + _elem1951.read(iprot); + struct.success.add(_elem1951); } iprot.readListEnd(); } @@ -151430,9 +151430,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1945 : struct.success) + for (Partition _iter1953 : struct.success) { - _iter1945.write(oprot); + _iter1953.write(oprot); } oprot.writeListEnd(); } @@ -151479,9 +151479,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1946 : struct.success) + for (Partition _iter1954 : struct.success) { - _iter1946.write(oprot); + _iter1954.write(oprot); } } } @@ -151499,14 +151499,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_fi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1947 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1947.size); - @org.apache.thrift.annotation.Nullable Partition _elem1948; - for (int _i1949 = 0; _i1949 < _list1947.size; ++_i1949) + org.apache.thrift.protocol.TList _list1955 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1955.size); + @org.apache.thrift.annotation.Nullable Partition _elem1956; + for (int _i1957 = 0; _i1957 < _list1955.size; ++_i1957) { - _elem1948 = new Partition(); - _elem1948.read(iprot); - struct.success.add(_elem1948); + _elem1956 = new Partition(); + _elem1956.read(iprot); + struct.success.add(_elem1956); } } struct.setSuccessIsSet(true); @@ -152375,14 +152375,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1950 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1950.size); - @org.apache.thrift.annotation.Nullable Partition _elem1951; - for (int _i1952 = 0; _i1952 < _list1950.size; ++_i1952) + org.apache.thrift.protocol.TList _list1958 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1958.size); + @org.apache.thrift.annotation.Nullable Partition _elem1959; + for (int _i1960 = 0; _i1960 < _list1958.size; ++_i1960) { - _elem1951 = new Partition(); - _elem1951.read(iprot); - struct.success.add(_elem1951); + _elem1959 = new Partition(); + _elem1959.read(iprot); + struct.success.add(_elem1959); } iprot.readListEnd(); } @@ -152426,9 +152426,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1953 : struct.success) + for (Partition _iter1961 : struct.success) { - _iter1953.write(oprot); + _iter1961.write(oprot); } oprot.writeListEnd(); } @@ -152475,9 +152475,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1954 : struct.success) + for (Partition _iter1962 : struct.success) { - _iter1954.write(oprot); + _iter1962.write(oprot); } } } @@ -152495,14 +152495,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_fi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1955 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1955.size); - @org.apache.thrift.annotation.Nullable Partition _elem1956; - for (int _i1957 = 0; _i1957 < _list1955.size; ++_i1957) + org.apache.thrift.protocol.TList _list1963 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1963.size); + @org.apache.thrift.annotation.Nullable Partition _elem1964; + for (int _i1965 = 0; _i1965 < _list1963.size; ++_i1965) { - _elem1956 = new Partition(); - _elem1956.read(iprot); - struct.success.add(_elem1956); + _elem1964 = new Partition(); + _elem1964.read(iprot); + struct.success.add(_elem1964); } } struct.setSuccessIsSet(true); @@ -153675,14 +153675,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_part_specs_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1958 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1958.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1959; - for (int _i1960 = 0; _i1960 < _list1958.size; ++_i1960) + org.apache.thrift.protocol.TList _list1966 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1966.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1967; + for (int _i1968 = 0; _i1968 < _list1966.size; ++_i1968) { - _elem1959 = new PartitionSpec(); - _elem1959.read(iprot); - struct.success.add(_elem1959); + _elem1967 = new PartitionSpec(); + _elem1967.read(iprot); + struct.success.add(_elem1967); } iprot.readListEnd(); } @@ -153726,9 +153726,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_part_specs_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (PartitionSpec _iter1961 : struct.success) + for (PartitionSpec _iter1969 : struct.success) { - _iter1961.write(oprot); + _iter1969.write(oprot); } oprot.writeListEnd(); } @@ -153775,9 +153775,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_part_specs_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (PartitionSpec _iter1962 : struct.success) + for (PartitionSpec _iter1970 : struct.success) { - _iter1962.write(oprot); + _iter1970.write(oprot); } } } @@ -153795,14 +153795,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_part_specs_by_fi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1963 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1963.size); - @org.apache.thrift.annotation.Nullable PartitionSpec _elem1964; - for (int _i1965 = 0; _i1965 < _list1963.size; ++_i1965) + org.apache.thrift.protocol.TList _list1971 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1971.size); + @org.apache.thrift.annotation.Nullable PartitionSpec _elem1972; + for (int _i1973 = 0; _i1973 < _list1971.size; ++_i1973) { - _elem1964 = new PartitionSpec(); - _elem1964.read(iprot); - struct.success.add(_elem1964); + _elem1972 = new PartitionSpec(); + _elem1972.read(iprot); + struct.success.add(_elem1972); } } struct.setSuccessIsSet(true); @@ -157350,13 +157350,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_n case 3: // NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1966 = iprot.readListBegin(); - struct.names = new java.util.ArrayList(_list1966.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1967; - for (int _i1968 = 0; _i1968 < _list1966.size; ++_i1968) + org.apache.thrift.protocol.TList _list1974 = iprot.readListBegin(); + struct.names = new java.util.ArrayList(_list1974.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1975; + for (int _i1976 = 0; _i1976 < _list1974.size; ++_i1976) { - _elem1967 = iprot.readString(); - struct.names.add(_elem1967); + _elem1975 = iprot.readString(); + struct.names.add(_elem1975); } iprot.readListEnd(); } @@ -157392,9 +157392,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size())); - for (java.lang.String _iter1969 : struct.names) + for (java.lang.String _iter1977 : struct.names) { - oprot.writeString(_iter1969); + oprot.writeString(_iter1977); } oprot.writeListEnd(); } @@ -157437,9 +157437,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_n if (struct.isSetNames()) { { oprot.writeI32(struct.names.size()); - for (java.lang.String _iter1970 : struct.names) + for (java.lang.String _iter1978 : struct.names) { - oprot.writeString(_iter1970); + oprot.writeString(_iter1978); } } } @@ -157459,13 +157459,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_na } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1971 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.names = new java.util.ArrayList(_list1971.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1972; - for (int _i1973 = 0; _i1973 < _list1971.size; ++_i1973) + org.apache.thrift.protocol.TList _list1979 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.names = new java.util.ArrayList(_list1979.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem1980; + for (int _i1981 = 0; _i1981 < _list1979.size; ++_i1981) { - _elem1972 = iprot.readString(); - struct.names.add(_elem1972); + _elem1980 = iprot.readString(); + struct.names.add(_elem1980); } } struct.setNamesIsSet(true); @@ -158038,14 +158038,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_n case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1974 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list1974.size); - @org.apache.thrift.annotation.Nullable Partition _elem1975; - for (int _i1976 = 0; _i1976 < _list1974.size; ++_i1976) + org.apache.thrift.protocol.TList _list1982 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list1982.size); + @org.apache.thrift.annotation.Nullable Partition _elem1983; + for (int _i1984 = 0; _i1984 < _list1982.size; ++_i1984) { - _elem1975 = new Partition(); - _elem1975.read(iprot); - struct.success.add(_elem1975); + _elem1983 = new Partition(); + _elem1983.read(iprot); + struct.success.add(_elem1983); } iprot.readListEnd(); } @@ -158098,9 +158098,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter1977 : struct.success) + for (Partition _iter1985 : struct.success) { - _iter1977.write(oprot); + _iter1985.write(oprot); } oprot.writeListEnd(); } @@ -158155,9 +158155,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_n if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter1978 : struct.success) + for (Partition _iter1986 : struct.success) { - _iter1978.write(oprot); + _iter1986.write(oprot); } } } @@ -158178,14 +158178,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_na java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list1979 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list1979.size); - @org.apache.thrift.annotation.Nullable Partition _elem1980; - for (int _i1981 = 0; _i1981 < _list1979.size; ++_i1981) + org.apache.thrift.protocol.TList _list1987 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list1987.size); + @org.apache.thrift.annotation.Nullable Partition _elem1988; + for (int _i1989 = 0; _i1989 < _list1987.size; ++_i1989) { - _elem1980 = new Partition(); - _elem1980.read(iprot); - struct.success.add(_elem1980); + _elem1988 = new Partition(); + _elem1988.read(iprot); + struct.success.add(_elem1988); } } struct.setSuccessIsSet(true); @@ -162689,14 +162689,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, alter_partitions_ar case 3: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1982 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1982.size); - @org.apache.thrift.annotation.Nullable Partition _elem1983; - for (int _i1984 = 0; _i1984 < _list1982.size; ++_i1984) + org.apache.thrift.protocol.TList _list1990 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1990.size); + @org.apache.thrift.annotation.Nullable Partition _elem1991; + for (int _i1992 = 0; _i1992 < _list1990.size; ++_i1992) { - _elem1983 = new Partition(); - _elem1983.read(iprot); - struct.new_parts.add(_elem1983); + _elem1991 = new Partition(); + _elem1991.read(iprot); + struct.new_parts.add(_elem1991); } iprot.readListEnd(); } @@ -162732,9 +162732,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, alter_partitions_a oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter1985 : struct.new_parts) + for (Partition _iter1993 : struct.new_parts) { - _iter1985.write(oprot); + _iter1993.write(oprot); } oprot.writeListEnd(); } @@ -162777,9 +162777,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, alter_partitions_ar if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter1986 : struct.new_parts) + for (Partition _iter1994 : struct.new_parts) { - _iter1986.write(oprot); + _iter1994.write(oprot); } } } @@ -162799,14 +162799,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, alter_partitions_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1987 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1987.size); - @org.apache.thrift.annotation.Nullable Partition _elem1988; - for (int _i1989 = 0; _i1989 < _list1987.size; ++_i1989) + org.apache.thrift.protocol.TList _list1995 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list1995.size); + @org.apache.thrift.annotation.Nullable Partition _elem1996; + for (int _i1997 = 0; _i1997 < _list1995.size; ++_i1997) { - _elem1988 = new Partition(); - _elem1988.read(iprot); - struct.new_parts.add(_elem1988); + _elem1996 = new Partition(); + _elem1996.read(iprot); + struct.new_parts.add(_elem1996); } } struct.setNew_partsIsSet(true); @@ -163868,14 +163868,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, alter_partitions_wi case 3: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1990 = iprot.readListBegin(); - struct.new_parts = new java.util.ArrayList(_list1990.size); - @org.apache.thrift.annotation.Nullable Partition _elem1991; - for (int _i1992 = 0; _i1992 < _list1990.size; ++_i1992) + org.apache.thrift.protocol.TList _list1998 = iprot.readListBegin(); + struct.new_parts = new java.util.ArrayList(_list1998.size); + @org.apache.thrift.annotation.Nullable Partition _elem1999; + for (int _i2000 = 0; _i2000 < _list1998.size; ++_i2000) { - _elem1991 = new Partition(); - _elem1991.read(iprot); - struct.new_parts.add(_elem1991); + _elem1999 = new Partition(); + _elem1999.read(iprot); + struct.new_parts.add(_elem1999); } iprot.readListEnd(); } @@ -163920,9 +163920,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, alter_partitions_w oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter1993 : struct.new_parts) + for (Partition _iter2001 : struct.new_parts) { - _iter1993.write(oprot); + _iter2001.write(oprot); } oprot.writeListEnd(); } @@ -163973,9 +163973,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, alter_partitions_wi if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter1994 : struct.new_parts) + for (Partition _iter2002 : struct.new_parts) { - _iter1994.write(oprot); + _iter2002.write(oprot); } } } @@ -163998,14 +163998,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, alter_partitions_wit } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list1995 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.new_parts = new java.util.ArrayList(_list1995.size); - @org.apache.thrift.annotation.Nullable Partition _elem1996; - for (int _i1997 = 0; _i1997 < _list1995.size; ++_i1997) + org.apache.thrift.protocol.TList _list2003 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.new_parts = new java.util.ArrayList(_list2003.size); + @org.apache.thrift.annotation.Nullable Partition _elem2004; + for (int _i2005 = 0; _i2005 < _list2003.size; ++_i2005) { - _elem1996 = new Partition(); - _elem1996.read(iprot); - struct.new_parts.add(_elem1996); + _elem2004 = new Partition(); + _elem2004.read(iprot); + struct.new_parts.add(_elem2004); } } struct.setNew_partsIsSet(true); @@ -167169,13 +167169,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, rename_partition_ar case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list1998 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list1998.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem1999; - for (int _i2000 = 0; _i2000 < _list1998.size; ++_i2000) + org.apache.thrift.protocol.TList _list2006 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list2006.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2007; + for (int _i2008 = 0; _i2008 < _list2006.size; ++_i2008) { - _elem1999 = iprot.readString(); - struct.part_vals.add(_elem1999); + _elem2007 = iprot.readString(); + struct.part_vals.add(_elem2007); } iprot.readListEnd(); } @@ -167220,9 +167220,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, rename_partition_a oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter2001 : struct.part_vals) + for (java.lang.String _iter2009 : struct.part_vals) { - oprot.writeString(_iter2001); + oprot.writeString(_iter2009); } oprot.writeListEnd(); } @@ -167273,9 +167273,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, rename_partition_ar if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter2002 : struct.part_vals) + for (java.lang.String _iter2010 : struct.part_vals) { - oprot.writeString(_iter2002); + oprot.writeString(_iter2010); } } } @@ -167298,13 +167298,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, rename_partition_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list2003 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list2003.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2004; - for (int _i2005 = 0; _i2005 < _list2003.size; ++_i2005) + org.apache.thrift.protocol.TList _list2011 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list2011.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2012; + for (int _i2013 = 0; _i2013 < _list2011.size; ++_i2013) { - _elem2004 = iprot.readString(); - struct.part_vals.add(_elem2004); + _elem2012 = iprot.readString(); + struct.part_vals.add(_elem2012); } } struct.setPart_valsIsSet(true); @@ -169130,13 +169130,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_has_ case 1: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2006 = iprot.readListBegin(); - struct.part_vals = new java.util.ArrayList(_list2006.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2007; - for (int _i2008 = 0; _i2008 < _list2006.size; ++_i2008) + org.apache.thrift.protocol.TList _list2014 = iprot.readListBegin(); + struct.part_vals = new java.util.ArrayList(_list2014.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2015; + for (int _i2016 = 0; _i2016 < _list2014.size; ++_i2016) { - _elem2007 = iprot.readString(); - struct.part_vals.add(_elem2007); + _elem2015 = iprot.readString(); + struct.part_vals.add(_elem2015); } iprot.readListEnd(); } @@ -169170,9 +169170,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_has oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.lang.String _iter2009 : struct.part_vals) + for (java.lang.String _iter2017 : struct.part_vals) { - oprot.writeString(_iter2009); + oprot.writeString(_iter2017); } oprot.writeListEnd(); } @@ -169209,9 +169209,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_has_ if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.lang.String _iter2010 : struct.part_vals) + for (java.lang.String _iter2018 : struct.part_vals) { - oprot.writeString(_iter2010); + oprot.writeString(_iter2018); } } } @@ -169226,13 +169226,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_has_v java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2011 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.ArrayList(_list2011.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2012; - for (int _i2013 = 0; _i2013 < _list2011.size; ++_i2013) + org.apache.thrift.protocol.TList _list2019 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.ArrayList(_list2019.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2020; + for (int _i2021 = 0; _i2021 < _list2019.size; ++_i2021) { - _elem2012 = iprot.readString(); - struct.part_vals.add(_elem2012); + _elem2020 = iprot.readString(); + struct.part_vals.add(_elem2020); } } struct.setPart_valsIsSet(true); @@ -171405,13 +171405,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_to_v case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2014 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2014.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2015; - for (int _i2016 = 0; _i2016 < _list2014.size; ++_i2016) + org.apache.thrift.protocol.TList _list2022 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2022.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2023; + for (int _i2024 = 0; _i2024 < _list2022.size; ++_i2024) { - _elem2015 = iprot.readString(); - struct.success.add(_elem2015); + _elem2023 = iprot.readString(); + struct.success.add(_elem2023); } iprot.readListEnd(); } @@ -171446,9 +171446,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_to_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2017 : struct.success) + for (java.lang.String _iter2025 : struct.success) { - oprot.writeString(_iter2017); + oprot.writeString(_iter2025); } oprot.writeListEnd(); } @@ -171487,9 +171487,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_to_v if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2018 : struct.success) + for (java.lang.String _iter2026 : struct.success) { - oprot.writeString(_iter2018); + oprot.writeString(_iter2026); } } } @@ -171504,13 +171504,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_to_va java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2019 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2019.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2020; - for (int _i2021 = 0; _i2021 < _list2019.size; ++_i2021) + org.apache.thrift.protocol.TList _list2027 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2027.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2028; + for (int _i2029 = 0; _i2029 < _list2027.size; ++_i2029) { - _elem2020 = iprot.readString(); - struct.success.add(_elem2020); + _elem2028 = iprot.readString(); + struct.success.add(_elem2028); } } struct.setSuccessIsSet(true); @@ -172281,15 +172281,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_to_s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2022 = iprot.readMapBegin(); - struct.success = new java.util.HashMap(2*_map2022.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2023; - @org.apache.thrift.annotation.Nullable java.lang.String _val2024; - for (int _i2025 = 0; _i2025 < _map2022.size; ++_i2025) + org.apache.thrift.protocol.TMap _map2030 = iprot.readMapBegin(); + struct.success = new java.util.HashMap(2*_map2030.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2031; + @org.apache.thrift.annotation.Nullable java.lang.String _val2032; + for (int _i2033 = 0; _i2033 < _map2030.size; ++_i2033) { - _key2023 = iprot.readString(); - _val2024 = iprot.readString(); - struct.success.put(_key2023, _val2024); + _key2031 = iprot.readString(); + _val2032 = iprot.readString(); + struct.success.put(_key2031, _val2032); } iprot.readMapEnd(); } @@ -172324,10 +172324,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_to_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.util.Map.Entry _iter2026 : struct.success.entrySet()) + for (java.util.Map.Entry _iter2034 : struct.success.entrySet()) { - oprot.writeString(_iter2026.getKey()); - oprot.writeString(_iter2026.getValue()); + oprot.writeString(_iter2034.getKey()); + oprot.writeString(_iter2034.getValue()); } oprot.writeMapEnd(); } @@ -172366,10 +172366,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_to_s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.util.Map.Entry _iter2027 : struct.success.entrySet()) + for (java.util.Map.Entry _iter2035 : struct.success.entrySet()) { - oprot.writeString(_iter2027.getKey()); - oprot.writeString(_iter2027.getValue()); + oprot.writeString(_iter2035.getKey()); + oprot.writeString(_iter2035.getValue()); } } } @@ -172384,15 +172384,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_to_sp java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map2028 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.HashMap(2*_map2028.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2029; - @org.apache.thrift.annotation.Nullable java.lang.String _val2030; - for (int _i2031 = 0; _i2031 < _map2028.size; ++_i2031) + org.apache.thrift.protocol.TMap _map2036 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.HashMap(2*_map2036.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2037; + @org.apache.thrift.annotation.Nullable java.lang.String _val2038; + for (int _i2039 = 0; _i2039 < _map2036.size; ++_i2039) { - _key2029 = iprot.readString(); - _val2030 = iprot.readString(); - struct.success.put(_key2029, _val2030); + _key2037 = iprot.readString(); + _val2038 = iprot.readString(); + struct.success.put(_key2037, _val2038); } } struct.setSuccessIsSet(true); @@ -172991,15 +172991,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, markPartitionForEve case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2032 = iprot.readMapBegin(); - struct.part_vals = new java.util.HashMap(2*_map2032.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2033; - @org.apache.thrift.annotation.Nullable java.lang.String _val2034; - for (int _i2035 = 0; _i2035 < _map2032.size; ++_i2035) + org.apache.thrift.protocol.TMap _map2040 = iprot.readMapBegin(); + struct.part_vals = new java.util.HashMap(2*_map2040.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2041; + @org.apache.thrift.annotation.Nullable java.lang.String _val2042; + for (int _i2043 = 0; _i2043 < _map2040.size; ++_i2043) { - _key2033 = iprot.readString(); - _val2034 = iprot.readString(); - struct.part_vals.put(_key2033, _val2034); + _key2041 = iprot.readString(); + _val2042 = iprot.readString(); + struct.part_vals.put(_key2041, _val2042); } iprot.readMapEnd(); } @@ -173043,10 +173043,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, markPartitionForEv oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.util.Map.Entry _iter2036 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2044 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2036.getKey()); - oprot.writeString(_iter2036.getValue()); + oprot.writeString(_iter2044.getKey()); + oprot.writeString(_iter2044.getValue()); } oprot.writeMapEnd(); } @@ -173097,10 +173097,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, markPartitionForEve if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.util.Map.Entry _iter2037 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2045 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2037.getKey()); - oprot.writeString(_iter2037.getValue()); + oprot.writeString(_iter2045.getKey()); + oprot.writeString(_iter2045.getValue()); } } } @@ -173123,15 +173123,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, markPartitionForEven } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map2038 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.HashMap(2*_map2038.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2039; - @org.apache.thrift.annotation.Nullable java.lang.String _val2040; - for (int _i2041 = 0; _i2041 < _map2038.size; ++_i2041) + org.apache.thrift.protocol.TMap _map2046 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.HashMap(2*_map2046.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2047; + @org.apache.thrift.annotation.Nullable java.lang.String _val2048; + for (int _i2049 = 0; _i2049 < _map2046.size; ++_i2049) { - _key2039 = iprot.readString(); - _val2040 = iprot.readString(); - struct.part_vals.put(_key2039, _val2040); + _key2047 = iprot.readString(); + _val2048 = iprot.readString(); + struct.part_vals.put(_key2047, _val2048); } } struct.setPart_valsIsSet(true); @@ -174623,15 +174623,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isPartitionMarkedFo case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2042 = iprot.readMapBegin(); - struct.part_vals = new java.util.HashMap(2*_map2042.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2043; - @org.apache.thrift.annotation.Nullable java.lang.String _val2044; - for (int _i2045 = 0; _i2045 < _map2042.size; ++_i2045) + org.apache.thrift.protocol.TMap _map2050 = iprot.readMapBegin(); + struct.part_vals = new java.util.HashMap(2*_map2050.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2051; + @org.apache.thrift.annotation.Nullable java.lang.String _val2052; + for (int _i2053 = 0; _i2053 < _map2050.size; ++_i2053) { - _key2043 = iprot.readString(); - _val2044 = iprot.readString(); - struct.part_vals.put(_key2043, _val2044); + _key2051 = iprot.readString(); + _val2052 = iprot.readString(); + struct.part_vals.put(_key2051, _val2052); } iprot.readMapEnd(); } @@ -174675,10 +174675,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isPartitionMarkedF oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (java.util.Map.Entry _iter2046 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2054 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2046.getKey()); - oprot.writeString(_iter2046.getValue()); + oprot.writeString(_iter2054.getKey()); + oprot.writeString(_iter2054.getValue()); } oprot.writeMapEnd(); } @@ -174729,10 +174729,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, isPartitionMarkedFo if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (java.util.Map.Entry _iter2047 : struct.part_vals.entrySet()) + for (java.util.Map.Entry _iter2055 : struct.part_vals.entrySet()) { - oprot.writeString(_iter2047.getKey()); - oprot.writeString(_iter2047.getValue()); + oprot.writeString(_iter2055.getKey()); + oprot.writeString(_iter2055.getValue()); } } } @@ -174755,15 +174755,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, isPartitionMarkedFor } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map2048 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); - struct.part_vals = new java.util.HashMap(2*_map2048.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2049; - @org.apache.thrift.annotation.Nullable java.lang.String _val2050; - for (int _i2051 = 0; _i2051 < _map2048.size; ++_i2051) + org.apache.thrift.protocol.TMap _map2056 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); + struct.part_vals = new java.util.HashMap(2*_map2056.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2057; + @org.apache.thrift.annotation.Nullable java.lang.String _val2058; + for (int _i2059 = 0; _i2059 < _map2056.size; ++_i2059) { - _key2049 = iprot.readString(); - _val2050 = iprot.readString(); - struct.part_vals.put(_key2049, _val2050); + _key2057 = iprot.readString(); + _val2058 = iprot.readString(); + struct.part_vals.put(_key2057, _val2058); } } struct.setPart_valsIsSet(true); @@ -201467,13 +201467,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_functions_resul case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2052 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2052.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2053; - for (int _i2054 = 0; _i2054 < _list2052.size; ++_i2054) + org.apache.thrift.protocol.TList _list2060 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2060.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2061; + for (int _i2062 = 0; _i2062 < _list2060.size; ++_i2062) { - _elem2053 = iprot.readString(); - struct.success.add(_elem2053); + _elem2061 = iprot.readString(); + struct.success.add(_elem2061); } iprot.readListEnd(); } @@ -201508,9 +201508,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_functions_resu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2055 : struct.success) + for (java.lang.String _iter2063 : struct.success) { - oprot.writeString(_iter2055); + oprot.writeString(_iter2063); } oprot.writeListEnd(); } @@ -201549,9 +201549,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_functions_resul if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2056 : struct.success) + for (java.lang.String _iter2064 : struct.success) { - oprot.writeString(_iter2056); + oprot.writeString(_iter2064); } } } @@ -201566,13 +201566,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_functions_result java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2057 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2057.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2058; - for (int _i2059 = 0; _i2059 < _list2057.size; ++_i2059) + org.apache.thrift.protocol.TList _list2065 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2065.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2066; + for (int _i2067 = 0; _i2067 < _list2065.size; ++_i2067) { - _elem2058 = iprot.readString(); - struct.success.add(_elem2058); + _elem2066 = iprot.readString(); + struct.success.add(_elem2066); } } struct.setSuccessIsSet(true); @@ -205662,13 +205662,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_role_names_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2060 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2060.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2061; - for (int _i2062 = 0; _i2062 < _list2060.size; ++_i2062) + org.apache.thrift.protocol.TList _list2068 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2068.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2069; + for (int _i2070 = 0; _i2070 < _list2068.size; ++_i2070) { - _elem2061 = iprot.readString(); - struct.success.add(_elem2061); + _elem2069 = iprot.readString(); + struct.success.add(_elem2069); } iprot.readListEnd(); } @@ -205703,9 +205703,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_role_names_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2063 : struct.success) + for (java.lang.String _iter2071 : struct.success) { - oprot.writeString(_iter2063); + oprot.writeString(_iter2071); } oprot.writeListEnd(); } @@ -205744,9 +205744,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_role_names_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2064 : struct.success) + for (java.lang.String _iter2072 : struct.success) { - oprot.writeString(_iter2064); + oprot.writeString(_iter2072); } } } @@ -205761,13 +205761,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_role_names_resul java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2065 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2065.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2066; - for (int _i2067 = 0; _i2067 < _list2065.size; ++_i2067) + org.apache.thrift.protocol.TList _list2073 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2073.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2074; + for (int _i2075 = 0; _i2075 < _list2073.size; ++_i2075) { - _elem2066 = iprot.readString(); - struct.success.add(_elem2066); + _elem2074 = iprot.readString(); + struct.success.add(_elem2074); } } struct.setSuccessIsSet(true); @@ -209074,14 +209074,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_roles_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2068 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2068.size); - @org.apache.thrift.annotation.Nullable Role _elem2069; - for (int _i2070 = 0; _i2070 < _list2068.size; ++_i2070) + org.apache.thrift.protocol.TList _list2076 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2076.size); + @org.apache.thrift.annotation.Nullable Role _elem2077; + for (int _i2078 = 0; _i2078 < _list2076.size; ++_i2078) { - _elem2069 = new Role(); - _elem2069.read(iprot); - struct.success.add(_elem2069); + _elem2077 = new Role(); + _elem2077.read(iprot); + struct.success.add(_elem2077); } iprot.readListEnd(); } @@ -209116,9 +209116,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_roles_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Role _iter2071 : struct.success) + for (Role _iter2079 : struct.success) { - _iter2071.write(oprot); + _iter2079.write(oprot); } oprot.writeListEnd(); } @@ -209157,9 +209157,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_roles_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Role _iter2072 : struct.success) + for (Role _iter2080 : struct.success) { - _iter2072.write(oprot); + _iter2080.write(oprot); } } } @@ -209174,14 +209174,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_roles_result st java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2073 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2073.size); - @org.apache.thrift.annotation.Nullable Role _elem2074; - for (int _i2075 = 0; _i2075 < _list2073.size; ++_i2075) + org.apache.thrift.protocol.TList _list2081 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2081.size); + @org.apache.thrift.annotation.Nullable Role _elem2082; + for (int _i2083 = 0; _i2083 < _list2081.size; ++_i2083) { - _elem2074 = new Role(); - _elem2074.read(iprot); - struct.success.add(_elem2074); + _elem2082 = new Role(); + _elem2082.read(iprot); + struct.success.add(_elem2082); } } struct.setSuccessIsSet(true); @@ -212215,13 +212215,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_privilege_set_a case 3: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2076 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list2076.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2077; - for (int _i2078 = 0; _i2078 < _list2076.size; ++_i2078) + org.apache.thrift.protocol.TList _list2084 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list2084.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2085; + for (int _i2086 = 0; _i2086 < _list2084.size; ++_i2086) { - _elem2077 = iprot.readString(); - struct.group_names.add(_elem2077); + _elem2085 = iprot.readString(); + struct.group_names.add(_elem2085); } iprot.readListEnd(); } @@ -212257,9 +212257,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_privilege_set_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter2079 : struct.group_names) + for (java.lang.String _iter2087 : struct.group_names) { - oprot.writeString(_iter2079); + oprot.writeString(_iter2087); } oprot.writeListEnd(); } @@ -212302,9 +212302,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_a if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter2080 : struct.group_names) + for (java.lang.String _iter2088 : struct.group_names) { - oprot.writeString(_iter2080); + oprot.writeString(_iter2088); } } } @@ -212325,13 +212325,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list2081 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list2081.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2082; - for (int _i2083 = 0; _i2083 < _list2081.size; ++_i2083) + org.apache.thrift.protocol.TList _list2089 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list2089.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2090; + for (int _i2091 = 0; _i2091 < _list2089.size; ++_i2091) { - _elem2082 = iprot.readString(); - struct.group_names.add(_elem2082); + _elem2090 = iprot.readString(); + struct.group_names.add(_elem2090); } } struct.setGroup_namesIsSet(true); @@ -213802,14 +213802,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_privileges_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2084 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2084.size); - @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2085; - for (int _i2086 = 0; _i2086 < _list2084.size; ++_i2086) + org.apache.thrift.protocol.TList _list2092 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2092.size); + @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2093; + for (int _i2094 = 0; _i2094 < _list2092.size; ++_i2094) { - _elem2085 = new HiveObjectPrivilege(); - _elem2085.read(iprot); - struct.success.add(_elem2085); + _elem2093 = new HiveObjectPrivilege(); + _elem2093.read(iprot); + struct.success.add(_elem2093); } iprot.readListEnd(); } @@ -213844,9 +213844,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_privileges_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (HiveObjectPrivilege _iter2087 : struct.success) + for (HiveObjectPrivilege _iter2095 : struct.success) { - _iter2087.write(oprot); + _iter2095.write(oprot); } oprot.writeListEnd(); } @@ -213885,9 +213885,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_privileges_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (HiveObjectPrivilege _iter2088 : struct.success) + for (HiveObjectPrivilege _iter2096 : struct.success) { - _iter2088.write(oprot); + _iter2096.write(oprot); } } } @@ -213902,14 +213902,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_privileges_resu java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2089 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2089.size); - @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2090; - for (int _i2091 = 0; _i2091 < _list2089.size; ++_i2091) + org.apache.thrift.protocol.TList _list2097 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2097.size); + @org.apache.thrift.annotation.Nullable HiveObjectPrivilege _elem2098; + for (int _i2099 = 0; _i2099 < _list2097.size; ++_i2099) { - _elem2090 = new HiveObjectPrivilege(); - _elem2090.read(iprot); - struct.success.add(_elem2090); + _elem2098 = new HiveObjectPrivilege(); + _elem2098.read(iprot); + struct.success.add(_elem2098); } } struct.setSuccessIsSet(true); @@ -217887,13 +217887,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_args struct case 2: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2092 = iprot.readListBegin(); - struct.group_names = new java.util.ArrayList(_list2092.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2093; - for (int _i2094 = 0; _i2094 < _list2092.size; ++_i2094) + org.apache.thrift.protocol.TList _list2100 = iprot.readListBegin(); + struct.group_names = new java.util.ArrayList(_list2100.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2101; + for (int _i2102 = 0; _i2102 < _list2100.size; ++_i2102) { - _elem2093 = iprot.readString(); - struct.group_names.add(_elem2093); + _elem2101 = iprot.readString(); + struct.group_names.add(_elem2101); } iprot.readListEnd(); } @@ -217924,9 +217924,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_args struc oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (java.lang.String _iter2095 : struct.group_names) + for (java.lang.String _iter2103 : struct.group_names) { - oprot.writeString(_iter2095); + oprot.writeString(_iter2103); } oprot.writeListEnd(); } @@ -217963,9 +217963,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (java.lang.String _iter2096 : struct.group_names) + for (java.lang.String _iter2104 : struct.group_names) { - oprot.writeString(_iter2096); + oprot.writeString(_iter2104); } } } @@ -217981,13 +217981,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct) } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list2097 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.group_names = new java.util.ArrayList(_list2097.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2098; - for (int _i2099 = 0; _i2099 < _list2097.size; ++_i2099) + org.apache.thrift.protocol.TList _list2105 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.group_names = new java.util.ArrayList(_list2105.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2106; + for (int _i2107 = 0; _i2107 < _list2105.size; ++_i2107) { - _elem2098 = iprot.readString(); - struct.group_names.add(_elem2098); + _elem2106 = iprot.readString(); + struct.group_names.add(_elem2106); } } struct.setGroup_namesIsSet(true); @@ -218395,13 +218395,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_result stru case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2100 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2100.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2101; - for (int _i2102 = 0; _i2102 < _list2100.size; ++_i2102) + org.apache.thrift.protocol.TList _list2108 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2108.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2109; + for (int _i2110 = 0; _i2110 < _list2108.size; ++_i2110) { - _elem2101 = iprot.readString(); - struct.success.add(_elem2101); + _elem2109 = iprot.readString(); + struct.success.add(_elem2109); } iprot.readListEnd(); } @@ -218436,9 +218436,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_result str oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2103 : struct.success) + for (java.lang.String _iter2111 : struct.success) { - oprot.writeString(_iter2103); + oprot.writeString(_iter2111); } oprot.writeListEnd(); } @@ -218477,9 +218477,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_result stru if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2104 : struct.success) + for (java.lang.String _iter2112 : struct.success) { - oprot.writeString(_iter2104); + oprot.writeString(_iter2112); } } } @@ -218494,13 +218494,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_result struc java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2105 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2105.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2106; - for (int _i2107 = 0; _i2107 < _list2105.size; ++_i2107) + org.apache.thrift.protocol.TList _list2113 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2113.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2114; + for (int _i2115 = 0; _i2115 < _list2113.size; ++_i2115) { - _elem2106 = iprot.readString(); - struct.success.add(_elem2106); + _elem2114 = iprot.readString(); + struct.success.add(_elem2114); } } struct.setSuccessIsSet(true); @@ -223839,13 +223839,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_token_ident case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2108 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2108.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2109; - for (int _i2110 = 0; _i2110 < _list2108.size; ++_i2110) + org.apache.thrift.protocol.TList _list2116 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2116.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2117; + for (int _i2118 = 0; _i2118 < _list2116.size; ++_i2118) { - _elem2109 = iprot.readString(); - struct.success.add(_elem2109); + _elem2117 = iprot.readString(); + struct.success.add(_elem2117); } iprot.readListEnd(); } @@ -223871,9 +223871,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_token_iden oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2111 : struct.success) + for (java.lang.String _iter2119 : struct.success) { - oprot.writeString(_iter2111); + oprot.writeString(_iter2119); } oprot.writeListEnd(); } @@ -223904,9 +223904,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_token_ident if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2112 : struct.success) + for (java.lang.String _iter2120 : struct.success) { - oprot.writeString(_iter2112); + oprot.writeString(_iter2120); } } } @@ -223918,13 +223918,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_token_identi java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2113 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2113.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2114; - for (int _i2115 = 0; _i2115 < _list2113.size; ++_i2115) + org.apache.thrift.protocol.TList _list2121 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2121.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2122; + for (int _i2123 = 0; _i2123 < _list2121.size; ++_i2123) { - _elem2114 = iprot.readString(); - struct.success.add(_elem2114); + _elem2122 = iprot.readString(); + struct.success.add(_elem2122); } } struct.setSuccessIsSet(true); @@ -226975,13 +226975,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_master_keys_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2116 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2116.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2117; - for (int _i2118 = 0; _i2118 < _list2116.size; ++_i2118) + org.apache.thrift.protocol.TList _list2124 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2124.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2125; + for (int _i2126 = 0; _i2126 < _list2124.size; ++_i2126) { - _elem2117 = iprot.readString(); - struct.success.add(_elem2117); + _elem2125 = iprot.readString(); + struct.success.add(_elem2125); } iprot.readListEnd(); } @@ -227007,9 +227007,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_master_keys_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2119 : struct.success) + for (java.lang.String _iter2127 : struct.success) { - oprot.writeString(_iter2119); + oprot.writeString(_iter2127); } oprot.writeListEnd(); } @@ -227040,9 +227040,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_master_keys_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2120 : struct.success) + for (java.lang.String _iter2128 : struct.success) { - oprot.writeString(_iter2120); + oprot.writeString(_iter2128); } } } @@ -227054,13 +227054,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_master_keys_resu java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2121 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2121.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2122; - for (int _i2123 = 0; _i2123 < _list2121.size; ++_i2123) + org.apache.thrift.protocol.TList _list2129 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2129.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2130; + for (int _i2131 = 0; _i2131 < _list2129.size; ++_i2131) { - _elem2122 = iprot.readString(); - struct.success.add(_elem2122); + _elem2130 = iprot.readString(); + struct.success.add(_elem2130); } } struct.setSuccessIsSet(true); @@ -234121,15 +234121,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_write_ids_to_mi case 2: // WRITE_IDS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map2124 = iprot.readMapBegin(); - struct.writeIds = new java.util.HashMap(2*_map2124.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2125; - long _val2126; - for (int _i2127 = 0; _i2127 < _map2124.size; ++_i2127) + org.apache.thrift.protocol.TMap _map2132 = iprot.readMapBegin(); + struct.writeIds = new java.util.HashMap(2*_map2132.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2133; + long _val2134; + for (int _i2135 = 0; _i2135 < _map2132.size; ++_i2135) { - _key2125 = iprot.readString(); - _val2126 = iprot.readI64(); - struct.writeIds.put(_key2125, _val2126); + _key2133 = iprot.readString(); + _val2134 = iprot.readI64(); + struct.writeIds.put(_key2133, _val2134); } iprot.readMapEnd(); } @@ -234158,10 +234158,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_write_ids_to_m oprot.writeFieldBegin(WRITE_IDS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.writeIds.size())); - for (java.util.Map.Entry _iter2128 : struct.writeIds.entrySet()) + for (java.util.Map.Entry _iter2136 : struct.writeIds.entrySet()) { - oprot.writeString(_iter2128.getKey()); - oprot.writeI64(_iter2128.getValue()); + oprot.writeString(_iter2136.getKey()); + oprot.writeI64(_iter2136.getValue()); } oprot.writeMapEnd(); } @@ -234198,10 +234198,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_mi if (struct.isSetWriteIds()) { { oprot.writeI32(struct.writeIds.size()); - for (java.util.Map.Entry _iter2129 : struct.writeIds.entrySet()) + for (java.util.Map.Entry _iter2137 : struct.writeIds.entrySet()) { - oprot.writeString(_iter2129.getKey()); - oprot.writeI64(_iter2129.getValue()); + oprot.writeString(_iter2137.getKey()); + oprot.writeI64(_iter2137.getValue()); } } } @@ -234217,15 +234217,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_min } if (incoming.get(1)) { { - org.apache.thrift.protocol.TMap _map2130 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64); - struct.writeIds = new java.util.HashMap(2*_map2130.size); - @org.apache.thrift.annotation.Nullable java.lang.String _key2131; - long _val2132; - for (int _i2133 = 0; _i2133 < _map2130.size; ++_i2133) + org.apache.thrift.protocol.TMap _map2138 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64); + struct.writeIds = new java.util.HashMap(2*_map2138.size); + @org.apache.thrift.annotation.Nullable java.lang.String _key2139; + long _val2140; + for (int _i2141 = 0; _i2141 < _map2138.size; ++_i2141) { - _key2131 = iprot.readString(); - _val2132 = iprot.readI64(); - struct.writeIds.put(_key2131, _val2132); + _key2139 = iprot.readString(); + _val2140 = iprot.readI64(); + struct.writeIds.put(_key2139, _val2140); } } struct.setWriteIdsIsSet(true); @@ -250222,13 +250222,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, find_columns_with_s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2134 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2134.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2135; - for (int _i2136 = 0; _i2136 < _list2134.size; ++_i2136) + org.apache.thrift.protocol.TList _list2142 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2142.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2143; + for (int _i2144 = 0; _i2144 < _list2142.size; ++_i2144) { - _elem2135 = iprot.readString(); - struct.success.add(_elem2135); + _elem2143 = iprot.readString(); + struct.success.add(_elem2143); } iprot.readListEnd(); } @@ -250254,9 +250254,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, find_columns_with_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2137 : struct.success) + for (java.lang.String _iter2145 : struct.success) { - oprot.writeString(_iter2137); + oprot.writeString(_iter2145); } oprot.writeListEnd(); } @@ -250287,9 +250287,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, find_columns_with_s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2138 : struct.success) + for (java.lang.String _iter2146 : struct.success) { - oprot.writeString(_iter2138); + oprot.writeString(_iter2146); } } } @@ -250301,13 +250301,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, find_columns_with_st java.util.BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2139 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2139.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2140; - for (int _i2141 = 0; _i2141 < _list2139.size; ++_i2141) + org.apache.thrift.protocol.TList _list2147 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2147.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2148; + for (int _i2149 = 0; _i2149 < _list2147.size; ++_i2149) { - _elem2140 = iprot.readString(); - struct.success.add(_elem2140); + _elem2148 = iprot.readString(); + struct.success.add(_elem2148); } } struct.setSuccessIsSet(true); @@ -291289,14 +291289,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_all_vers case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2142 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2142.size); - @org.apache.thrift.annotation.Nullable SchemaVersion _elem2143; - for (int _i2144 = 0; _i2144 < _list2142.size; ++_i2144) + org.apache.thrift.protocol.TList _list2150 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2150.size); + @org.apache.thrift.annotation.Nullable SchemaVersion _elem2151; + for (int _i2152 = 0; _i2152 < _list2150.size; ++_i2152) { - _elem2143 = new SchemaVersion(); - _elem2143.read(iprot); - struct.success.add(_elem2143); + _elem2151 = new SchemaVersion(); + _elem2151.read(iprot); + struct.success.add(_elem2151); } iprot.readListEnd(); } @@ -291340,9 +291340,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_all_ver oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (SchemaVersion _iter2145 : struct.success) + for (SchemaVersion _iter2153 : struct.success) { - _iter2145.write(oprot); + _iter2153.write(oprot); } oprot.writeListEnd(); } @@ -291389,9 +291389,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_all_vers if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (SchemaVersion _iter2146 : struct.success) + for (SchemaVersion _iter2154 : struct.success) { - _iter2146.write(oprot); + _iter2154.write(oprot); } } } @@ -291409,14 +291409,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_all_versi java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2147 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2147.size); - @org.apache.thrift.annotation.Nullable SchemaVersion _elem2148; - for (int _i2149 = 0; _i2149 < _list2147.size; ++_i2149) + org.apache.thrift.protocol.TList _list2155 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2155.size); + @org.apache.thrift.annotation.Nullable SchemaVersion _elem2156; + for (int _i2157 = 0; _i2157 < _list2155.size; ++_i2157) { - _elem2148 = new SchemaVersion(); - _elem2148.read(iprot); - struct.success.add(_elem2148); + _elem2156 = new SchemaVersion(); + _elem2156.read(iprot); + struct.success.add(_elem2156); } } struct.setSuccessIsSet(true); @@ -300031,14 +300031,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_runtime_stats_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2150 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2150.size); - @org.apache.thrift.annotation.Nullable RuntimeStat _elem2151; - for (int _i2152 = 0; _i2152 < _list2150.size; ++_i2152) + org.apache.thrift.protocol.TList _list2158 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2158.size); + @org.apache.thrift.annotation.Nullable RuntimeStat _elem2159; + for (int _i2160 = 0; _i2160 < _list2158.size; ++_i2160) { - _elem2151 = new RuntimeStat(); - _elem2151.read(iprot); - struct.success.add(_elem2151); + _elem2159 = new RuntimeStat(); + _elem2159.read(iprot); + struct.success.add(_elem2159); } iprot.readListEnd(); } @@ -300073,9 +300073,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_runtime_stats_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (RuntimeStat _iter2153 : struct.success) + for (RuntimeStat _iter2161 : struct.success) { - _iter2153.write(oprot); + _iter2161.write(oprot); } oprot.writeListEnd(); } @@ -300114,9 +300114,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_runtime_stats_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (RuntimeStat _iter2154 : struct.success) + for (RuntimeStat _iter2162 : struct.success) { - _iter2154.write(oprot); + _iter2162.write(oprot); } } } @@ -300131,14 +300131,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_runtime_stats_re java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2155 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2155.size); - @org.apache.thrift.annotation.Nullable RuntimeStat _elem2156; - for (int _i2157 = 0; _i2157 < _list2155.size; ++_i2157) + org.apache.thrift.protocol.TList _list2163 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2163.size); + @org.apache.thrift.annotation.Nullable RuntimeStat _elem2164; + for (int _i2165 = 0; _i2165 < _list2163.size; ++_i2165) { - _elem2156 = new RuntimeStat(); - _elem2156.read(iprot); - struct.success.add(_elem2156); + _elem2164 = new RuntimeStat(); + _elem2164.read(iprot); + struct.success.add(_elem2164); } } struct.setSuccessIsSet(true); @@ -310249,13 +310249,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_stored_proc case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2158 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2158.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2159; - for (int _i2160 = 0; _i2160 < _list2158.size; ++_i2160) + org.apache.thrift.protocol.TList _list2166 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2166.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2167; + for (int _i2168 = 0; _i2168 < _list2166.size; ++_i2168) { - _elem2159 = iprot.readString(); - struct.success.add(_elem2159); + _elem2167 = iprot.readString(); + struct.success.add(_elem2167); } iprot.readListEnd(); } @@ -310290,9 +310290,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_stored_pro oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2161 : struct.success) + for (java.lang.String _iter2169 : struct.success) { - oprot.writeString(_iter2161); + oprot.writeString(_iter2169); } oprot.writeListEnd(); } @@ -310331,9 +310331,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_stored_proc if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2162 : struct.success) + for (java.lang.String _iter2170 : struct.success) { - oprot.writeString(_iter2162); + oprot.writeString(_iter2170); } } } @@ -310348,13 +310348,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_stored_proce java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2163 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2163.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2164; - for (int _i2165 = 0; _i2165 < _list2163.size; ++_i2165) + org.apache.thrift.protocol.TList _list2171 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2171.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2172; + for (int _i2173 = 0; _i2173 < _list2171.size; ++_i2173) { - _elem2164 = iprot.readString(); - struct.success.add(_elem2164); + _elem2172 = iprot.readString(); + struct.success.add(_elem2172); } } struct.setSuccessIsSet(true); @@ -312811,13 +312811,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_packages_re case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2166 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2166.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2167; - for (int _i2168 = 0; _i2168 < _list2166.size; ++_i2168) + org.apache.thrift.protocol.TList _list2174 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2174.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2175; + for (int _i2176 = 0; _i2176 < _list2174.size; ++_i2176) { - _elem2167 = iprot.readString(); - struct.success.add(_elem2167); + _elem2175 = iprot.readString(); + struct.success.add(_elem2175); } iprot.readListEnd(); } @@ -312852,9 +312852,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_packages_r oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (java.lang.String _iter2169 : struct.success) + for (java.lang.String _iter2177 : struct.success) { - oprot.writeString(_iter2169); + oprot.writeString(_iter2177); } oprot.writeListEnd(); } @@ -312893,9 +312893,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_packages_re if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (java.lang.String _iter2170 : struct.success) + for (java.lang.String _iter2178 : struct.success) { - oprot.writeString(_iter2170); + oprot.writeString(_iter2178); } } } @@ -312910,13 +312910,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_packages_res java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2171 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); - struct.success = new java.util.ArrayList(_list2171.size); - @org.apache.thrift.annotation.Nullable java.lang.String _elem2172; - for (int _i2173 = 0; _i2173 < _list2171.size; ++_i2173) + org.apache.thrift.protocol.TList _list2179 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING); + struct.success = new java.util.ArrayList(_list2179.size); + @org.apache.thrift.annotation.Nullable java.lang.String _elem2180; + for (int _i2181 = 0; _i2181 < _list2179.size; ++_i2181) { - _elem2172 = iprot.readString(); - struct.success.add(_elem2172); + _elem2180 = iprot.readString(); + struct.success.add(_elem2180); } } struct.setSuccessIsSet(true); @@ -314430,14 +314430,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_write_event case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list2174 = iprot.readListBegin(); - struct.success = new java.util.ArrayList(_list2174.size); - @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2175; - for (int _i2176 = 0; _i2176 < _list2174.size; ++_i2176) + org.apache.thrift.protocol.TList _list2182 = iprot.readListBegin(); + struct.success = new java.util.ArrayList(_list2182.size); + @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2183; + for (int _i2184 = 0; _i2184 < _list2182.size; ++_i2184) { - _elem2175 = new WriteEventInfo(); - _elem2175.read(iprot); - struct.success.add(_elem2175); + _elem2183 = new WriteEventInfo(); + _elem2183.read(iprot); + struct.success.add(_elem2183); } iprot.readListEnd(); } @@ -314472,9 +314472,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_write_even oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (WriteEventInfo _iter2177 : struct.success) + for (WriteEventInfo _iter2185 : struct.success) { - _iter2177.write(oprot); + _iter2185.write(oprot); } oprot.writeListEnd(); } @@ -314513,9 +314513,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_write_event if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (WriteEventInfo _iter2178 : struct.success) + for (WriteEventInfo _iter2186 : struct.success) { - _iter2178.write(oprot); + _iter2186.write(oprot); } } } @@ -314530,14 +314530,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_write_event_ java.util.BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list2179 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); - struct.success = new java.util.ArrayList(_list2179.size); - @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2180; - for (int _i2181 = 0; _i2181 < _list2179.size; ++_i2181) + org.apache.thrift.protocol.TList _list2187 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT); + struct.success = new java.util.ArrayList(_list2187.size); + @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2188; + for (int _i2189 = 0; _i2189 < _list2187.size; ++_i2189) { - _elem2180 = new WriteEventInfo(); - _elem2180.read(iprot); - struct.success.add(_elem2180); + _elem2188 = new WriteEventInfo(); + _elem2188.read(iprot); + struct.success.add(_elem2188); } } struct.setSuccessIsSet(true); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php index 9b2db717d0f4..f34820d0501a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetOpenTxnsRequest.php @@ -69,13 +69,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->excludeTxnTypes = array(); - $_size1339 = 0; - $_etype1342 = 0; - $xfer += $input->readListBegin($_etype1342, $_size1339); - for ($_i1343 = 0; $_i1343 < $_size1339; ++$_i1343) { - $elem1344 = null; - $xfer += $input->readI32($elem1344); - $this->excludeTxnTypes []= $elem1344; + $_size1346 = 0; + $_etype1349 = 0; + $xfer += $input->readListBegin($_etype1349, $_size1346); + for ($_i1350 = 0; $_i1350 < $_size1346; ++$_i1350) { + $elem1351 = null; + $xfer += $input->readI32($elem1351); + $this->excludeTxnTypes []= $elem1351; } $xfer += $input->readListEnd(); } else { @@ -102,8 +102,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('excludeTxnTypes', TType::LST, 1); $output->writeListBegin(TType::I32, count($this->excludeTxnTypes)); - foreach ($this->excludeTxnTypes as $iter1345) { - $xfer += $output->writeI32($iter1345); + foreach ($this->excludeTxnTypes as $iter1352) { + $xfer += $output->writeI32($iter1352); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php index 19c68368627e..80209821d954 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthRequest.php @@ -89,6 +89,15 @@ class GetPartitionsPsWithAuthRequest 'isRequired' => false, 'type' => TType::STRING, ), + 13 => array( + 'var' => 'partNames', + 'isRequired' => false, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); /** @@ -139,6 +148,10 @@ class GetPartitionsPsWithAuthRequest * @var string */ public $excludeParamKeyPattern = null; + /** + * @var string[] + */ + public $partNames = null; public function __construct($vals = null) { @@ -179,6 +192,9 @@ public function __construct($vals = null) if (isset($vals['excludeParamKeyPattern'])) { $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern']; } + if (isset($vals['partNames'])) { + $this->partNames = $vals['partNames']; + } } } @@ -303,6 +319,22 @@ public function read($input) $xfer += $input->skip($ftype); } break; + case 13: + if ($ftype == TType::LST) { + $this->partNames = array(); + $_size1323 = 0; + $_etype1326 = 0; + $xfer += $input->readListBegin($_etype1326, $_size1323); + for ($_i1327 = 0; $_i1327 < $_size1323; ++$_i1327) { + $elem1328 = null; + $xfer += $input->readString($elem1328); + $this->partNames []= $elem1328; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -338,8 +370,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partVals', TType::LST, 4); $output->writeListBegin(TType::STRING, count($this->partVals)); - foreach ($this->partVals as $iter1323) { - $xfer += $output->writeString($iter1323); + foreach ($this->partVals as $iter1329) { + $xfer += $output->writeString($iter1329); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -360,8 +392,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('groupNames', TType::LST, 7); $output->writeListBegin(TType::STRING, count($this->groupNames)); - foreach ($this->groupNames as $iter1324) { - $xfer += $output->writeString($iter1324); + foreach ($this->groupNames as $iter1330) { + $xfer += $output->writeString($iter1330); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -391,6 +423,18 @@ public function write($output) $xfer += $output->writeString($this->excludeParamKeyPattern); $xfer += $output->writeFieldEnd(); } + if ($this->partNames !== null) { + if (!is_array($this->partNames)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('partNames', TType::LST, 13); + $output->writeListBegin(TType::STRING, count($this->partNames)); + foreach ($this->partNames as $iter1331) { + $xfer += $output->writeString($iter1331); + } + $output->writeListEnd(); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php index 3c8a7092cd90..f08a4fc4ea35 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsPsWithAuthResponse.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->partitions = array(); - $_size1325 = 0; - $_etype1328 = 0; - $xfer += $input->readListBegin($_etype1328, $_size1325); - for ($_i1329 = 0; $_i1329 < $_size1325; ++$_i1329) { - $elem1330 = null; - $elem1330 = new \metastore\Partition(); - $xfer += $elem1330->read($input); - $this->partitions []= $elem1330; + $_size1332 = 0; + $_etype1335 = 0; + $xfer += $input->readListBegin($_etype1335, $_size1332); + for ($_i1336 = 0; $_i1336 < $_size1332; ++$_i1336) { + $elem1337 = null; + $elem1337 = new \metastore\Partition(); + $xfer += $elem1337->read($input); + $this->partitions []= $elem1337; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitions', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->partitions)); - foreach ($this->partitions as $iter1331) { - $xfer += $iter1331->write($output); + foreach ($this->partitions as $iter1338) { + $xfer += $iter1338->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php index 5c136ac01a8c..8926dcb1e65e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ReplicationMetricList.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->replicationMetricList = array(); - $_size1332 = 0; - $_etype1335 = 0; - $xfer += $input->readListBegin($_etype1335, $_size1332); - for ($_i1336 = 0; $_i1336 < $_size1332; ++$_i1336) { - $elem1337 = null; - $elem1337 = new \metastore\ReplicationMetrics(); - $xfer += $elem1337->read($input); - $this->replicationMetricList []= $elem1337; + $_size1339 = 0; + $_etype1342 = 0; + $xfer += $input->readListBegin($_etype1342, $_size1339); + for ($_i1343 = 0; $_i1343 < $_size1339; ++$_i1343) { + $elem1344 = null; + $elem1344 = new \metastore\ReplicationMetrics(); + $xfer += $elem1344->read($input); + $this->replicationMetricList []= $elem1344; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('replicationMetricList', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->replicationMetricList)); - foreach ($this->replicationMetricList as $iter1338) { - $xfer += $iter1338->write($output); + foreach ($this->replicationMetricList as $iter1345) { + $xfer += $iter1345->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php index a32966ffdc6f..1dc57b40b50a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_args.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1530 = 0; - $_etype1533 = 0; - $xfer += $input->readListBegin($_etype1533, $_size1530); - for ($_i1534 = 0; $_i1534 < $_size1530; ++$_i1534) { - $elem1535 = null; - $elem1535 = new \metastore\Partition(); - $xfer += $elem1535->read($input); - $this->new_parts []= $elem1535; + $_size1537 = 0; + $_etype1540 = 0; + $xfer += $input->readListBegin($_etype1540, $_size1537); + for ($_i1541 = 0; $_i1541 < $_size1537; ++$_i1541) { + $elem1542 = null; + $elem1542 = new \metastore\Partition(); + $xfer += $elem1542->read($input); + $this->new_parts []= $elem1542; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1536) { - $xfer += $iter1536->write($output); + foreach ($this->new_parts as $iter1543) { + $xfer += $iter1543->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php index 2bbda0ca9aee..80216cf42726 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_partitions_pspec_args.php @@ -69,14 +69,14 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1537 = 0; - $_etype1540 = 0; - $xfer += $input->readListBegin($_etype1540, $_size1537); - for ($_i1541 = 0; $_i1541 < $_size1537; ++$_i1541) { - $elem1542 = null; - $elem1542 = new \metastore\PartitionSpec(); - $xfer += $elem1542->read($input); - $this->new_parts []= $elem1542; + $_size1544 = 0; + $_etype1547 = 0; + $xfer += $input->readListBegin($_etype1547, $_size1544); + for ($_i1548 = 0; $_i1548 < $_size1544; ++$_i1548) { + $elem1549 = null; + $elem1549 = new \metastore\PartitionSpec(); + $xfer += $elem1549->read($input); + $this->new_parts []= $elem1549; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1543) { - $xfer += $iter1543->write($output); + foreach ($this->new_parts as $iter1550) { + $xfer += $iter1550->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php index 8589337a8431..05df1f2cabdb 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php @@ -91,16 +91,16 @@ public function read($input) case 2: if ($ftype == TType::MAP) { $this->writeIds = array(); - $_size1869 = 0; - $_ktype1870 = 0; - $_vtype1871 = 0; - $xfer += $input->readMapBegin($_ktype1870, $_vtype1871, $_size1869); - for ($_i1873 = 0; $_i1873 < $_size1869; ++$_i1873) { - $key1874 = ''; - $val1875 = 0; - $xfer += $input->readString($key1874); - $xfer += $input->readI64($val1875); - $this->writeIds[$key1874] = $val1875; + $_size1876 = 0; + $_ktype1877 = 0; + $_vtype1878 = 0; + $xfer += $input->readMapBegin($_ktype1877, $_vtype1878, $_size1876); + for ($_i1880 = 0; $_i1880 < $_size1876; ++$_i1880) { + $key1881 = ''; + $val1882 = 0; + $xfer += $input->readString($key1881); + $xfer += $input->readI64($val1882); + $this->writeIds[$key1881] = $val1882; } $xfer += $input->readMapEnd(); } else { @@ -132,9 +132,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('writeIds', TType::MAP, 2); $output->writeMapBegin(TType::STRING, TType::I64, count($this->writeIds)); - foreach ($this->writeIds as $kiter1876 => $viter1877) { - $xfer += $output->writeString($kiter1876); - $xfer += $output->writeI64($viter1877); + foreach ($this->writeIds as $kiter1883 => $viter1884) { + $xfer += $output->writeString($kiter1883); + $xfer += $output->writeI64($viter1884); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php index 2dae7ba57dcd..c5b5c2a76af8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_args.php @@ -107,14 +107,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1744 = 0; - $_etype1747 = 0; - $xfer += $input->readListBegin($_etype1747, $_size1744); - for ($_i1748 = 0; $_i1748 < $_size1744; ++$_i1748) { - $elem1749 = null; - $elem1749 = new \metastore\Partition(); - $xfer += $elem1749->read($input); - $this->new_parts []= $elem1749; + $_size1751 = 0; + $_etype1754 = 0; + $xfer += $input->readListBegin($_etype1754, $_size1751); + for ($_i1755 = 0; $_i1755 < $_size1751; ++$_i1755) { + $elem1756 = null; + $elem1756 = new \metastore\Partition(); + $xfer += $elem1756->read($input); + $this->new_parts []= $elem1756; } $xfer += $input->readListEnd(); } else { @@ -151,8 +151,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1750) { - $xfer += $iter1750->write($output); + foreach ($this->new_parts as $iter1757) { + $xfer += $iter1757->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php index b8129f56cb9d..42f1fa4248a3 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_alter_partitions_with_environment_context_args.php @@ -120,14 +120,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->new_parts = array(); - $_size1751 = 0; - $_etype1754 = 0; - $xfer += $input->readListBegin($_etype1754, $_size1751); - for ($_i1755 = 0; $_i1755 < $_size1751; ++$_i1755) { - $elem1756 = null; - $elem1756 = new \metastore\Partition(); - $xfer += $elem1756->read($input); - $this->new_parts []= $elem1756; + $_size1758 = 0; + $_etype1761 = 0; + $xfer += $input->readListBegin($_etype1761, $_size1758); + for ($_i1762 = 0; $_i1762 < $_size1758; ++$_i1762) { + $elem1763 = null; + $elem1763 = new \metastore\Partition(); + $xfer += $elem1763->read($input); + $this->new_parts []= $elem1763; } $xfer += $input->readListEnd(); } else { @@ -172,8 +172,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - foreach ($this->new_parts as $iter1757) { - $xfer += $iter1757->write($output); + foreach ($this->new_parts as $iter1764) { + $xfer += $iter1764->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php index f20598fc6701..d8c58f4e318b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1544 = 0; - $_etype1547 = 0; - $xfer += $input->readListBegin($_etype1547, $_size1544); - for ($_i1548 = 0; $_i1548 < $_size1544; ++$_i1548) { - $elem1549 = null; - $xfer += $input->readString($elem1549); - $this->part_vals []= $elem1549; + $_size1551 = 0; + $_etype1554 = 0; + $xfer += $input->readListBegin($_etype1554, $_size1551); + for ($_i1555 = 0; $_i1555 < $_size1551; ++$_i1555) { + $elem1556 = null; + $xfer += $input->readString($elem1556); + $this->part_vals []= $elem1556; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1550) { - $xfer += $output->writeString($iter1550); + foreach ($this->part_vals as $iter1557) { + $xfer += $output->writeString($iter1557); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php index b2e1ffbe9e34..c447067cc283 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_append_partition_with_environment_context_args.php @@ -119,13 +119,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1551 = 0; - $_etype1554 = 0; - $xfer += $input->readListBegin($_etype1554, $_size1551); - for ($_i1555 = 0; $_i1555 < $_size1551; ++$_i1555) { - $elem1556 = null; - $xfer += $input->readString($elem1556); - $this->part_vals []= $elem1556; + $_size1558 = 0; + $_etype1561 = 0; + $xfer += $input->readListBegin($_etype1561, $_size1558); + for ($_i1562 = 0; $_i1562 < $_size1558; ++$_i1562) { + $elem1563 = null; + $xfer += $input->readString($elem1563); + $this->part_vals []= $elem1563; } $xfer += $input->readListEnd(); } else { @@ -170,8 +170,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1557) { - $xfer += $output->writeString($iter1557); + foreach ($this->part_vals as $iter1564) { + $xfer += $output->writeString($iter1564); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php index 6a77f79145d1..dace4a5a191c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_create_table_with_constraints_args.php @@ -175,14 +175,14 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->primaryKeys = array(); - $_size1404 = 0; - $_etype1407 = 0; - $xfer += $input->readListBegin($_etype1407, $_size1404); - for ($_i1408 = 0; $_i1408 < $_size1404; ++$_i1408) { - $elem1409 = null; - $elem1409 = new \metastore\SQLPrimaryKey(); - $xfer += $elem1409->read($input); - $this->primaryKeys []= $elem1409; + $_size1411 = 0; + $_etype1414 = 0; + $xfer += $input->readListBegin($_etype1414, $_size1411); + for ($_i1415 = 0; $_i1415 < $_size1411; ++$_i1415) { + $elem1416 = null; + $elem1416 = new \metastore\SQLPrimaryKey(); + $xfer += $elem1416->read($input); + $this->primaryKeys []= $elem1416; } $xfer += $input->readListEnd(); } else { @@ -192,14 +192,14 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->foreignKeys = array(); - $_size1410 = 0; - $_etype1413 = 0; - $xfer += $input->readListBegin($_etype1413, $_size1410); - for ($_i1414 = 0; $_i1414 < $_size1410; ++$_i1414) { - $elem1415 = null; - $elem1415 = new \metastore\SQLForeignKey(); - $xfer += $elem1415->read($input); - $this->foreignKeys []= $elem1415; + $_size1417 = 0; + $_etype1420 = 0; + $xfer += $input->readListBegin($_etype1420, $_size1417); + for ($_i1421 = 0; $_i1421 < $_size1417; ++$_i1421) { + $elem1422 = null; + $elem1422 = new \metastore\SQLForeignKey(); + $xfer += $elem1422->read($input); + $this->foreignKeys []= $elem1422; } $xfer += $input->readListEnd(); } else { @@ -209,14 +209,14 @@ public function read($input) case 4: if ($ftype == TType::LST) { $this->uniqueConstraints = array(); - $_size1416 = 0; - $_etype1419 = 0; - $xfer += $input->readListBegin($_etype1419, $_size1416); - for ($_i1420 = 0; $_i1420 < $_size1416; ++$_i1420) { - $elem1421 = null; - $elem1421 = new \metastore\SQLUniqueConstraint(); - $xfer += $elem1421->read($input); - $this->uniqueConstraints []= $elem1421; + $_size1423 = 0; + $_etype1426 = 0; + $xfer += $input->readListBegin($_etype1426, $_size1423); + for ($_i1427 = 0; $_i1427 < $_size1423; ++$_i1427) { + $elem1428 = null; + $elem1428 = new \metastore\SQLUniqueConstraint(); + $xfer += $elem1428->read($input); + $this->uniqueConstraints []= $elem1428; } $xfer += $input->readListEnd(); } else { @@ -226,14 +226,14 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->notNullConstraints = array(); - $_size1422 = 0; - $_etype1425 = 0; - $xfer += $input->readListBegin($_etype1425, $_size1422); - for ($_i1426 = 0; $_i1426 < $_size1422; ++$_i1426) { - $elem1427 = null; - $elem1427 = new \metastore\SQLNotNullConstraint(); - $xfer += $elem1427->read($input); - $this->notNullConstraints []= $elem1427; + $_size1429 = 0; + $_etype1432 = 0; + $xfer += $input->readListBegin($_etype1432, $_size1429); + for ($_i1433 = 0; $_i1433 < $_size1429; ++$_i1433) { + $elem1434 = null; + $elem1434 = new \metastore\SQLNotNullConstraint(); + $xfer += $elem1434->read($input); + $this->notNullConstraints []= $elem1434; } $xfer += $input->readListEnd(); } else { @@ -243,14 +243,14 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->defaultConstraints = array(); - $_size1428 = 0; - $_etype1431 = 0; - $xfer += $input->readListBegin($_etype1431, $_size1428); - for ($_i1432 = 0; $_i1432 < $_size1428; ++$_i1432) { - $elem1433 = null; - $elem1433 = new \metastore\SQLDefaultConstraint(); - $xfer += $elem1433->read($input); - $this->defaultConstraints []= $elem1433; + $_size1435 = 0; + $_etype1438 = 0; + $xfer += $input->readListBegin($_etype1438, $_size1435); + for ($_i1439 = 0; $_i1439 < $_size1435; ++$_i1439) { + $elem1440 = null; + $elem1440 = new \metastore\SQLDefaultConstraint(); + $xfer += $elem1440->read($input); + $this->defaultConstraints []= $elem1440; } $xfer += $input->readListEnd(); } else { @@ -260,14 +260,14 @@ public function read($input) case 7: if ($ftype == TType::LST) { $this->checkConstraints = array(); - $_size1434 = 0; - $_etype1437 = 0; - $xfer += $input->readListBegin($_etype1437, $_size1434); - for ($_i1438 = 0; $_i1438 < $_size1434; ++$_i1438) { - $elem1439 = null; - $elem1439 = new \metastore\SQLCheckConstraint(); - $xfer += $elem1439->read($input); - $this->checkConstraints []= $elem1439; + $_size1441 = 0; + $_etype1444 = 0; + $xfer += $input->readListBegin($_etype1444, $_size1441); + for ($_i1445 = 0; $_i1445 < $_size1441; ++$_i1445) { + $elem1446 = null; + $elem1446 = new \metastore\SQLCheckConstraint(); + $xfer += $elem1446->read($input); + $this->checkConstraints []= $elem1446; } $xfer += $input->readListEnd(); } else { @@ -302,8 +302,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('primaryKeys', TType::LST, 2); $output->writeListBegin(TType::STRUCT, count($this->primaryKeys)); - foreach ($this->primaryKeys as $iter1440) { - $xfer += $iter1440->write($output); + foreach ($this->primaryKeys as $iter1447) { + $xfer += $iter1447->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -314,8 +314,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('foreignKeys', TType::LST, 3); $output->writeListBegin(TType::STRUCT, count($this->foreignKeys)); - foreach ($this->foreignKeys as $iter1441) { - $xfer += $iter1441->write($output); + foreach ($this->foreignKeys as $iter1448) { + $xfer += $iter1448->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -326,8 +326,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('uniqueConstraints', TType::LST, 4); $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints)); - foreach ($this->uniqueConstraints as $iter1442) { - $xfer += $iter1442->write($output); + foreach ($this->uniqueConstraints as $iter1449) { + $xfer += $iter1449->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -338,8 +338,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('notNullConstraints', TType::LST, 5); $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints)); - foreach ($this->notNullConstraints as $iter1443) { - $xfer += $iter1443->write($output); + foreach ($this->notNullConstraints as $iter1450) { + $xfer += $iter1450->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -350,8 +350,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('defaultConstraints', TType::LST, 6); $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints)); - foreach ($this->defaultConstraints as $iter1444) { - $xfer += $iter1444->write($output); + foreach ($this->defaultConstraints as $iter1451) { + $xfer += $iter1451->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -362,8 +362,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('checkConstraints', TType::LST, 7); $output->writeListBegin(TType::STRUCT, count($this->checkConstraints)); - foreach ($this->checkConstraints as $iter1445) { - $xfer += $iter1445->write($output); + foreach ($this->checkConstraints as $iter1452) { + $xfer += $iter1452->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php index af48d9f1470d..f34eaae559d2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_args.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1558 = 0; - $_etype1561 = 0; - $xfer += $input->readListBegin($_etype1561, $_size1558); - for ($_i1562 = 0; $_i1562 < $_size1558; ++$_i1562) { - $elem1563 = null; - $xfer += $input->readString($elem1563); - $this->part_vals []= $elem1563; + $_size1565 = 0; + $_etype1568 = 0; + $xfer += $input->readListBegin($_etype1568, $_size1565); + for ($_i1569 = 0; $_i1569 < $_size1565; ++$_i1569) { + $elem1570 = null; + $xfer += $input->readString($elem1570); + $this->part_vals []= $elem1570; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1564) { - $xfer += $output->writeString($iter1564); + foreach ($this->part_vals as $iter1571) { + $xfer += $output->writeString($iter1571); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php index 6d436e6d5d2f..9049d55a59a6 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_drop_partition_with_environment_context_args.php @@ -131,13 +131,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1565 = 0; - $_etype1568 = 0; - $xfer += $input->readListBegin($_etype1568, $_size1565); - for ($_i1569 = 0; $_i1569 < $_size1565; ++$_i1569) { - $elem1570 = null; - $xfer += $input->readString($elem1570); - $this->part_vals []= $elem1570; + $_size1572 = 0; + $_etype1575 = 0; + $xfer += $input->readListBegin($_etype1575, $_size1572); + for ($_i1576 = 0; $_i1576 < $_size1572; ++$_i1576) { + $elem1577 = null; + $xfer += $input->readString($elem1577); + $this->part_vals []= $elem1577; } $xfer += $input->readListEnd(); } else { @@ -189,8 +189,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1571) { - $xfer += $output->writeString($iter1571); + foreach ($this->part_vals as $iter1578) { + $xfer += $output->writeString($iter1578); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php index 226f156cb6f4..5b38432184e8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partition_args.php @@ -120,16 +120,16 @@ public function read($input) case 1: if ($ftype == TType::MAP) { $this->partitionSpecs = array(); - $_size1579 = 0; - $_ktype1580 = 0; - $_vtype1581 = 0; - $xfer += $input->readMapBegin($_ktype1580, $_vtype1581, $_size1579); - for ($_i1583 = 0; $_i1583 < $_size1579; ++$_i1583) { - $key1584 = ''; - $val1585 = ''; - $xfer += $input->readString($key1584); - $xfer += $input->readString($val1585); - $this->partitionSpecs[$key1584] = $val1585; + $_size1586 = 0; + $_ktype1587 = 0; + $_vtype1588 = 0; + $xfer += $input->readMapBegin($_ktype1587, $_vtype1588, $_size1586); + for ($_i1590 = 0; $_i1590 < $_size1586; ++$_i1590) { + $key1591 = ''; + $val1592 = ''; + $xfer += $input->readString($key1591); + $xfer += $input->readString($val1592); + $this->partitionSpecs[$key1591] = $val1592; } $xfer += $input->readMapEnd(); } else { @@ -184,9 +184,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); - foreach ($this->partitionSpecs as $kiter1586 => $viter1587) { - $xfer += $output->writeString($kiter1586); - $xfer += $output->writeString($viter1587); + foreach ($this->partitionSpecs as $kiter1593 => $viter1594) { + $xfer += $output->writeString($kiter1593); + $xfer += $output->writeString($viter1594); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php index 582a0668db2d..75f568fb8f8d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_args.php @@ -120,16 +120,16 @@ public function read($input) case 1: if ($ftype == TType::MAP) { $this->partitionSpecs = array(); - $_size1588 = 0; - $_ktype1589 = 0; - $_vtype1590 = 0; - $xfer += $input->readMapBegin($_ktype1589, $_vtype1590, $_size1588); - for ($_i1592 = 0; $_i1592 < $_size1588; ++$_i1592) { - $key1593 = ''; - $val1594 = ''; - $xfer += $input->readString($key1593); - $xfer += $input->readString($val1594); - $this->partitionSpecs[$key1593] = $val1594; + $_size1595 = 0; + $_ktype1596 = 0; + $_vtype1597 = 0; + $xfer += $input->readMapBegin($_ktype1596, $_vtype1597, $_size1595); + for ($_i1599 = 0; $_i1599 < $_size1595; ++$_i1599) { + $key1600 = ''; + $val1601 = ''; + $xfer += $input->readString($key1600); + $xfer += $input->readString($val1601); + $this->partitionSpecs[$key1600] = $val1601; } $xfer += $input->readMapEnd(); } else { @@ -184,9 +184,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); - foreach ($this->partitionSpecs as $kiter1595 => $viter1596) { - $xfer += $output->writeString($kiter1595); - $xfer += $output->writeString($viter1596); + foreach ($this->partitionSpecs as $kiter1602 => $viter1603) { + $xfer += $output->writeString($kiter1602); + $xfer += $output->writeString($viter1603); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php index aa6f6904c5d6..95e90156f504 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_exchange_partitions_result.php @@ -121,14 +121,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1597 = 0; - $_etype1600 = 0; - $xfer += $input->readListBegin($_etype1600, $_size1597); - for ($_i1601 = 0; $_i1601 < $_size1597; ++$_i1601) { - $elem1602 = null; - $elem1602 = new \metastore\Partition(); - $xfer += $elem1602->read($input); - $this->success []= $elem1602; + $_size1604 = 0; + $_etype1607 = 0; + $xfer += $input->readListBegin($_etype1607, $_size1604); + for ($_i1608 = 0; $_i1608 < $_size1604; ++$_i1608) { + $elem1609 = null; + $elem1609 = new \metastore\Partition(); + $xfer += $elem1609->read($input); + $this->success []= $elem1609; } $xfer += $input->readListEnd(); } else { @@ -187,8 +187,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1603) { - $xfer += $iter1603->write($output); + foreach ($this->success as $iter1610) { + $xfer += $iter1610->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php index bf71f8bad539..6d627c4a27a8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php @@ -68,13 +68,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1878 = 0; - $_etype1881 = 0; - $xfer += $input->readListBegin($_etype1881, $_size1878); - for ($_i1882 = 0; $_i1882 < $_size1878; ++$_i1882) { - $elem1883 = null; - $xfer += $input->readString($elem1883); - $this->success []= $elem1883; + $_size1885 = 0; + $_etype1888 = 0; + $xfer += $input->readListBegin($_etype1888, $_size1885); + for ($_i1889 = 0; $_i1889 < $_size1885; ++$_i1889) { + $elem1890 = null; + $xfer += $input->readString($elem1890); + $this->success []= $elem1890; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1884) { - $xfer += $output->writeString($iter1884); + foreach ($this->success as $iter1891) { + $xfer += $output->writeString($iter1891); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php index e52138309bef..a5860860e68a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_databases_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1353 = 0; - $_etype1356 = 0; - $xfer += $input->readListBegin($_etype1356, $_size1353); - for ($_i1357 = 0; $_i1357 < $_size1353; ++$_i1357) { - $elem1358 = null; - $xfer += $input->readString($elem1358); - $this->success []= $elem1358; + $_size1360 = 0; + $_etype1363 = 0; + $xfer += $input->readListBegin($_etype1363, $_size1360); + for ($_i1364 = 0; $_i1364 < $_size1360; ++$_i1364) { + $elem1365 = null; + $xfer += $input->readString($elem1365); + $this->success []= $elem1365; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1359) { - $xfer += $output->writeString($iter1359); + foreach ($this->success as $iter1366) { + $xfer += $output->writeString($iter1366); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php index 464a804f21d9..ab6bc80c5919 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1467 = 0; - $_etype1470 = 0; - $xfer += $input->readListBegin($_etype1470, $_size1467); - for ($_i1471 = 0; $_i1471 < $_size1467; ++$_i1471) { - $elem1472 = null; - $elem1472 = new \metastore\Table(); - $xfer += $elem1472->read($input); - $this->success []= $elem1472; + $_size1474 = 0; + $_etype1477 = 0; + $xfer += $input->readListBegin($_etype1477, $_size1474); + for ($_i1478 = 0; $_i1478 < $_size1474; ++$_i1478) { + $elem1479 = null; + $elem1479 = new \metastore\Table(); + $xfer += $elem1479->read($input); + $this->success []= $elem1479; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1473) { - $xfer += $iter1473->write($output); + foreach ($this->success as $iter1480) { + $xfer += $iter1480->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php index c14aeda7dc91..0be3936c19e4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1906 = 0; - $_etype1909 = 0; - $xfer += $input->readListBegin($_etype1909, $_size1906); - for ($_i1910 = 0; $_i1910 < $_size1906; ++$_i1910) { - $elem1911 = null; - $xfer += $input->readString($elem1911); - $this->success []= $elem1911; + $_size1913 = 0; + $_etype1916 = 0; + $xfer += $input->readListBegin($_etype1916, $_size1913); + for ($_i1917 = 0; $_i1917 < $_size1913; ++$_i1917) { + $elem1918 = null; + $xfer += $input->readString($elem1918); + $this->success []= $elem1918; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1912) { - $xfer += $output->writeString($iter1912); + foreach ($this->success as $iter1919) { + $xfer += $output->writeString($iter1919); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php index b6c42bf25aea..3bf12bef1c9a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1899 = 0; - $_etype1902 = 0; - $xfer += $input->readListBegin($_etype1902, $_size1899); - for ($_i1903 = 0; $_i1903 < $_size1899; ++$_i1903) { - $elem1904 = null; - $xfer += $input->readString($elem1904); - $this->success []= $elem1904; + $_size1906 = 0; + $_etype1909 = 0; + $xfer += $input->readListBegin($_etype1909, $_size1906); + for ($_i1910 = 0; $_i1910 < $_size1906; ++$_i1910) { + $elem1911 = null; + $xfer += $input->readString($elem1911); + $this->success []= $elem1911; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1905) { - $xfer += $output->writeString($iter1905); + foreach ($this->success as $iter1912) { + $xfer += $output->writeString($iter1912); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php index 8a9a245ee734..220515f5d1d2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_tables_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1495 = 0; - $_etype1498 = 0; - $xfer += $input->readListBegin($_etype1498, $_size1495); - for ($_i1499 = 0; $_i1499 < $_size1495; ++$_i1499) { - $elem1500 = null; - $xfer += $input->readString($elem1500); - $this->success []= $elem1500; + $_size1502 = 0; + $_etype1505 = 0; + $xfer += $input->readListBegin($_etype1505, $_size1502); + for ($_i1506 = 0; $_i1506 < $_size1502; ++$_i1506) { + $elem1507 = null; + $xfer += $input->readString($elem1507); + $this->success []= $elem1507; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1501) { - $xfer += $output->writeString($iter1501); + foreach ($this->success as $iter1508) { + $xfer += $output->writeString($iter1508); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php index fd3bdbf40935..09590993f1fd 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_token_identifiers_result.php @@ -68,13 +68,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1855 = 0; - $_etype1858 = 0; - $xfer += $input->readListBegin($_etype1858, $_size1855); - for ($_i1859 = 0; $_i1859 < $_size1855; ++$_i1859) { - $elem1860 = null; - $xfer += $input->readString($elem1860); - $this->success []= $elem1860; + $_size1862 = 0; + $_etype1865 = 0; + $xfer += $input->readListBegin($_etype1865, $_size1862); + for ($_i1866 = 0; $_i1866 < $_size1862; ++$_i1866) { + $elem1867 = null; + $xfer += $input->readString($elem1867); + $this->success []= $elem1867; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1861) { - $xfer += $output->writeString($iter1861); + foreach ($this->success as $iter1868) { + $xfer += $output->writeString($iter1868); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php index aa956ce7c99c..343de3e5128f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1913 = 0; - $_etype1916 = 0; - $xfer += $input->readListBegin($_etype1916, $_size1913); - for ($_i1917 = 0; $_i1917 < $_size1913; ++$_i1917) { - $elem1918 = null; - $elem1918 = new \metastore\WriteEventInfo(); - $xfer += $elem1918->read($input); - $this->success []= $elem1918; + $_size1920 = 0; + $_etype1923 = 0; + $xfer += $input->readListBegin($_etype1923, $_size1920); + for ($_i1924 = 0; $_i1924 < $_size1920; ++$_i1924) { + $elem1925 = null; + $elem1925 = new \metastore\WriteEventInfo(); + $xfer += $elem1925->read($input); + $this->success []= $elem1925; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1919) { - $xfer += $iter1919->write($output); + foreach ($this->success as $iter1926) { + $xfer += $iter1926->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php index cf6afd2c2395..a50a97c3be3c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_databases_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1346 = 0; - $_etype1349 = 0; - $xfer += $input->readListBegin($_etype1349, $_size1346); - for ($_i1350 = 0; $_i1350 < $_size1346; ++$_i1350) { - $elem1351 = null; - $xfer += $input->readString($elem1351); - $this->success []= $elem1351; + $_size1353 = 0; + $_etype1356 = 0; + $xfer += $input->readListBegin($_etype1356, $_size1353); + for ($_i1357 = 0; $_i1357 < $_size1353; ++$_i1357) { + $elem1358 = null; + $xfer += $input->readString($elem1358); + $this->success []= $elem1358; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1352) { - $xfer += $output->writeString($iter1352); + foreach ($this->success as $iter1359) { + $xfer += $output->writeString($iter1359); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php index 41b8faec9aac..9b3dcd5b1f5d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_dataconnectors_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1360 = 0; - $_etype1363 = 0; - $xfer += $input->readListBegin($_etype1363, $_size1360); - for ($_i1364 = 0; $_i1364 < $_size1360; ++$_i1364) { - $elem1365 = null; - $xfer += $input->readString($elem1365); - $this->success []= $elem1365; + $_size1367 = 0; + $_etype1370 = 0; + $xfer += $input->readListBegin($_etype1370, $_size1367); + for ($_i1371 = 0; $_i1371 < $_size1367; ++$_i1371) { + $elem1372 = null; + $xfer += $input->readString($elem1372); + $this->success []= $elem1372; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1366) { - $xfer += $output->writeString($iter1366); + foreach ($this->success as $iter1373) { + $xfer += $output->writeString($iter1373); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php index 548467e4810e..a1d3bf985fc8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1376 = 0; - $_etype1379 = 0; - $xfer += $input->readListBegin($_etype1379, $_size1376); - for ($_i1380 = 0; $_i1380 < $_size1376; ++$_i1380) { - $elem1381 = null; - $elem1381 = new \metastore\FieldSchema(); - $xfer += $elem1381->read($input); - $this->success []= $elem1381; + $_size1383 = 0; + $_etype1386 = 0; + $xfer += $input->readListBegin($_etype1386, $_size1383); + for ($_i1387 = 0; $_i1387 < $_size1383; ++$_i1387) { + $elem1388 = null; + $elem1388 = new \metastore\FieldSchema(); + $xfer += $elem1388->read($input); + $this->success []= $elem1388; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1382) { - $xfer += $iter1382->write($output); + foreach ($this->success as $iter1389) { + $xfer += $iter1389->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php index 6e8331417d50..ac940fd26b67 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_fields_with_environment_context_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1383 = 0; - $_etype1386 = 0; - $xfer += $input->readListBegin($_etype1386, $_size1383); - for ($_i1387 = 0; $_i1387 < $_size1383; ++$_i1387) { - $elem1388 = null; - $elem1388 = new \metastore\FieldSchema(); - $xfer += $elem1388->read($input); - $this->success []= $elem1388; + $_size1390 = 0; + $_etype1393 = 0; + $xfer += $input->readListBegin($_etype1393, $_size1390); + for ($_i1394 = 0; $_i1394 < $_size1390; ++$_i1394) { + $elem1395 = null; + $elem1395 = new \metastore\FieldSchema(); + $xfer += $elem1395->read($input); + $this->success []= $elem1395; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1389) { - $xfer += $iter1389->write($output); + foreach ($this->success as $iter1396) { + $xfer += $iter1396->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php index 7515fcd01b3c..72548b7497d8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_functions_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1806 = 0; - $_etype1809 = 0; - $xfer += $input->readListBegin($_etype1809, $_size1806); - for ($_i1810 = 0; $_i1810 < $_size1806; ++$_i1810) { - $elem1811 = null; - $xfer += $input->readString($elem1811); - $this->success []= $elem1811; + $_size1813 = 0; + $_etype1816 = 0; + $xfer += $input->readListBegin($_etype1816, $_size1813); + for ($_i1817 = 0; $_i1817 < $_size1813; ++$_i1817) { + $elem1818 = null; + $xfer += $input->readString($elem1818); + $this->success []= $elem1818; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1812) { - $xfer += $output->writeString($iter1812); + foreach ($this->success as $iter1819) { + $xfer += $output->writeString($iter1819); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php index da066c60534b..f05f0168b04c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_master_keys_result.php @@ -68,13 +68,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1862 = 0; - $_etype1865 = 0; - $xfer += $input->readListBegin($_etype1865, $_size1862); - for ($_i1866 = 0; $_i1866 < $_size1862; ++$_i1866) { - $elem1867 = null; - $xfer += $input->readString($elem1867); - $this->success []= $elem1867; + $_size1869 = 0; + $_etype1872 = 0; + $xfer += $input->readListBegin($_etype1872, $_size1869); + for ($_i1873 = 0; $_i1873 < $_size1869; ++$_i1873) { + $elem1874 = null; + $xfer += $input->readString($elem1874); + $this->success []= $elem1874; } $xfer += $input->readListEnd(); } else { @@ -101,8 +101,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1868) { - $xfer += $output->writeString($iter1868); + foreach ($this->success as $iter1875) { + $xfer += $output->writeString($iter1875); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php index e9c2964bc6a6..0e4bd2a7454b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_materialized_views_for_rewriting_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1474 = 0; - $_etype1477 = 0; - $xfer += $input->readListBegin($_etype1477, $_size1474); - for ($_i1478 = 0; $_i1478 < $_size1474; ++$_i1478) { - $elem1479 = null; - $xfer += $input->readString($elem1479); - $this->success []= $elem1479; + $_size1481 = 0; + $_etype1484 = 0; + $xfer += $input->readListBegin($_etype1484, $_size1481); + for ($_i1485 = 0; $_i1485 < $_size1481; ++$_i1485) { + $elem1486 = null; + $xfer += $input->readString($elem1486); + $this->success []= $elem1486; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1480) { - $xfer += $output->writeString($iter1480); + foreach ($this->success as $iter1487) { + $xfer += $output->writeString($iter1487); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php index db79d88731f6..8b7e9b7c095e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_part_specs_by_filter_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1723 = 0; - $_etype1726 = 0; - $xfer += $input->readListBegin($_etype1726, $_size1723); - for ($_i1727 = 0; $_i1727 < $_size1723; ++$_i1727) { - $elem1728 = null; - $elem1728 = new \metastore\PartitionSpec(); - $xfer += $elem1728->read($input); - $this->success []= $elem1728; + $_size1730 = 0; + $_etype1733 = 0; + $xfer += $input->readListBegin($_etype1733, $_size1730); + for ($_i1734 = 0; $_i1734 < $_size1730; ++$_i1734) { + $elem1735 = null; + $elem1735 = new \metastore\PartitionSpec(); + $xfer += $elem1735->read($input); + $this->success []= $elem1735; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1729) { - $xfer += $iter1729->write($output); + foreach ($this->success as $iter1736) { + $xfer += $iter1736->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php index 1d43357ad595..f989a977cd45 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1572 = 0; - $_etype1575 = 0; - $xfer += $input->readListBegin($_etype1575, $_size1572); - for ($_i1576 = 0; $_i1576 < $_size1572; ++$_i1576) { - $elem1577 = null; - $xfer += $input->readString($elem1577); - $this->part_vals []= $elem1577; + $_size1579 = 0; + $_etype1582 = 0; + $xfer += $input->readListBegin($_etype1582, $_size1579); + for ($_i1583 = 0; $_i1583 < $_size1579; ++$_i1583) { + $elem1584 = null; + $xfer += $input->readString($elem1584); + $this->part_vals []= $elem1584; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1578) { - $xfer += $output->writeString($iter1578); + foreach ($this->part_vals as $iter1585) { + $xfer += $output->writeString($iter1585); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php index 5dfca92a5356..08591f2cb016 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_args.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1688 = 0; - $_etype1691 = 0; - $xfer += $input->readListBegin($_etype1691, $_size1688); - for ($_i1692 = 0; $_i1692 < $_size1688; ++$_i1692) { - $elem1693 = null; - $xfer += $input->readString($elem1693); - $this->part_vals []= $elem1693; + $_size1695 = 0; + $_etype1698 = 0; + $xfer += $input->readListBegin($_etype1698, $_size1695); + for ($_i1699 = 0; $_i1699 < $_size1695; ++$_i1699) { + $elem1700 = null; + $xfer += $input->readString($elem1700); + $this->part_vals []= $elem1700; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1694) { - $xfer += $output->writeString($iter1694); + foreach ($this->part_vals as $iter1701) { + $xfer += $output->writeString($iter1701); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php index f6cac7f439f5..ec1b1405abc2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_ps_result.php @@ -94,13 +94,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1695 = 0; - $_etype1698 = 0; - $xfer += $input->readListBegin($_etype1698, $_size1695); - for ($_i1699 = 0; $_i1699 < $_size1695; ++$_i1699) { - $elem1700 = null; - $xfer += $input->readString($elem1700); - $this->success []= $elem1700; + $_size1702 = 0; + $_etype1705 = 0; + $xfer += $input->readListBegin($_etype1705, $_size1702); + for ($_i1706 = 0; $_i1706 < $_size1702; ++$_i1706) { + $elem1707 = null; + $xfer += $input->readString($elem1707); + $this->success []= $elem1707; } $xfer += $input->readListEnd(); } else { @@ -143,8 +143,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1701) { - $xfer += $output->writeString($iter1701); + foreach ($this->success as $iter1708) { + $xfer += $output->writeString($iter1708); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php index 927ddd467d7b..4ff05c25c732 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_req_result.php @@ -94,13 +94,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1702 = 0; - $_etype1705 = 0; - $xfer += $input->readListBegin($_etype1705, $_size1702); - for ($_i1706 = 0; $_i1706 < $_size1702; ++$_i1706) { - $elem1707 = null; - $xfer += $input->readString($elem1707); - $this->success []= $elem1707; + $_size1709 = 0; + $_etype1712 = 0; + $xfer += $input->readListBegin($_etype1712, $_size1709); + for ($_i1713 = 0; $_i1713 < $_size1709; ++$_i1713) { + $elem1714 = null; + $xfer += $input->readString($elem1714); + $this->success []= $elem1714; } $xfer += $input->readListEnd(); } else { @@ -143,8 +143,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1708) { - $xfer += $output->writeString($iter1708); + foreach ($this->success as $iter1715) { + $xfer += $output->writeString($iter1715); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php index a3f8299255ba..cdc873884469 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_names_result.php @@ -94,13 +94,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1646 = 0; - $_etype1649 = 0; - $xfer += $input->readListBegin($_etype1649, $_size1646); - for ($_i1650 = 0; $_i1650 < $_size1646; ++$_i1650) { - $elem1651 = null; - $xfer += $input->readString($elem1651); - $this->success []= $elem1651; + $_size1653 = 0; + $_etype1656 = 0; + $xfer += $input->readListBegin($_etype1656, $_size1653); + for ($_i1657 = 0; $_i1657 < $_size1653; ++$_i1657) { + $elem1658 = null; + $xfer += $input->readString($elem1658); + $this->success []= $elem1658; } $xfer += $input->readListEnd(); } else { @@ -143,8 +143,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1652) { - $xfer += $output->writeString($iter1652); + foreach ($this->success as $iter1659) { + $xfer += $output->writeString($iter1659); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php index 8e79331510ae..8fb33e53d09f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partition_with_auth_args.php @@ -134,13 +134,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1604 = 0; - $_etype1607 = 0; - $xfer += $input->readListBegin($_etype1607, $_size1604); - for ($_i1608 = 0; $_i1608 < $_size1604; ++$_i1608) { - $elem1609 = null; - $xfer += $input->readString($elem1609); - $this->part_vals []= $elem1609; + $_size1611 = 0; + $_etype1614 = 0; + $xfer += $input->readListBegin($_etype1614, $_size1611); + for ($_i1615 = 0; $_i1615 < $_size1611; ++$_i1615) { + $elem1616 = null; + $xfer += $input->readString($elem1616); + $this->part_vals []= $elem1616; } $xfer += $input->readListEnd(); } else { @@ -157,13 +157,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1610 = 0; - $_etype1613 = 0; - $xfer += $input->readListBegin($_etype1613, $_size1610); - for ($_i1614 = 0; $_i1614 < $_size1610; ++$_i1614) { - $elem1615 = null; - $xfer += $input->readString($elem1615); - $this->group_names []= $elem1615; + $_size1617 = 0; + $_etype1620 = 0; + $xfer += $input->readListBegin($_etype1620, $_size1617); + for ($_i1621 = 0; $_i1621 < $_size1617; ++$_i1621) { + $elem1622 = null; + $xfer += $input->readString($elem1622); + $this->group_names []= $elem1622; } $xfer += $input->readListEnd(); } else { @@ -200,8 +200,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1616) { - $xfer += $output->writeString($iter1616); + foreach ($this->part_vals as $iter1623) { + $xfer += $output->writeString($iter1623); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -217,8 +217,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1617) { - $xfer += $output->writeString($iter1617); + foreach ($this->group_names as $iter1624) { + $xfer += $output->writeString($iter1624); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php index accd7d41558b..e88cd7eb667f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_req_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1716 = 0; - $_etype1719 = 0; - $xfer += $input->readListBegin($_etype1719, $_size1716); - for ($_i1720 = 0; $_i1720 < $_size1716; ++$_i1720) { - $elem1721 = null; - $elem1721 = new \metastore\Partition(); - $xfer += $elem1721->read($input); - $this->success []= $elem1721; + $_size1723 = 0; + $_etype1726 = 0; + $xfer += $input->readListBegin($_etype1726, $_size1723); + for ($_i1727 = 0; $_i1727 < $_size1723; ++$_i1727) { + $elem1728 = null; + $elem1728 = new \metastore\Partition(); + $xfer += $elem1728->read($input); + $this->success []= $elem1728; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1722) { - $xfer += $iter1722->write($output); + foreach ($this->success as $iter1729) { + $xfer += $iter1729->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php index 3479e5db697f..538238196fd2 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_filter_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1709 = 0; - $_etype1712 = 0; - $xfer += $input->readListBegin($_etype1712, $_size1709); - for ($_i1713 = 0; $_i1713 < $_size1709; ++$_i1713) { - $elem1714 = null; - $elem1714 = new \metastore\Partition(); - $xfer += $elem1714->read($input); - $this->success []= $elem1714; + $_size1716 = 0; + $_etype1719 = 0; + $xfer += $input->readListBegin($_etype1719, $_size1716); + for ($_i1720 = 0; $_i1720 < $_size1716; ++$_i1720) { + $elem1721 = null; + $elem1721 = new \metastore\Partition(); + $xfer += $elem1721->read($input); + $this->success []= $elem1721; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1715) { - $xfer += $iter1715->write($output); + foreach ($this->success as $iter1722) { + $xfer += $iter1722->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php index e49950967509..e13c697443a4 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->names = array(); - $_size1730 = 0; - $_etype1733 = 0; - $xfer += $input->readListBegin($_etype1733, $_size1730); - for ($_i1734 = 0; $_i1734 < $_size1730; ++$_i1734) { - $elem1735 = null; - $xfer += $input->readString($elem1735); - $this->names []= $elem1735; + $_size1737 = 0; + $_etype1740 = 0; + $xfer += $input->readListBegin($_etype1740, $_size1737); + for ($_i1741 = 0; $_i1741 < $_size1737; ++$_i1741) { + $elem1742 = null; + $xfer += $input->readString($elem1742); + $this->names []= $elem1742; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('names', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->names)); - foreach ($this->names as $iter1736) { - $xfer += $output->writeString($iter1736); + foreach ($this->names as $iter1743) { + $xfer += $output->writeString($iter1743); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php index 616ae5afaac1..3094d1016d23 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_by_names_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1737 = 0; - $_etype1740 = 0; - $xfer += $input->readListBegin($_etype1740, $_size1737); - for ($_i1741 = 0; $_i1741 < $_size1737; ++$_i1741) { - $elem1742 = null; - $elem1742 = new \metastore\Partition(); - $xfer += $elem1742->read($input); - $this->success []= $elem1742; + $_size1744 = 0; + $_etype1747 = 0; + $xfer += $input->readListBegin($_etype1747, $_size1744); + for ($_i1748 = 0; $_i1748 < $_size1744; ++$_i1748) { + $elem1749 = null; + $elem1749 = new \metastore\Partition(); + $xfer += $elem1749->read($input); + $this->success []= $elem1749; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1743) { - $xfer += $iter1743->write($output); + foreach ($this->success as $iter1750) { + $xfer += $iter1750->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php index b51b393bf67c..e1e17f875ea1 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_args.php @@ -118,13 +118,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1653 = 0; - $_etype1656 = 0; - $xfer += $input->readListBegin($_etype1656, $_size1653); - for ($_i1657 = 0; $_i1657 < $_size1653; ++$_i1657) { - $elem1658 = null; - $xfer += $input->readString($elem1658); - $this->part_vals []= $elem1658; + $_size1660 = 0; + $_etype1663 = 0; + $xfer += $input->readListBegin($_etype1663, $_size1660); + for ($_i1664 = 0; $_i1664 < $_size1660; ++$_i1664) { + $elem1665 = null; + $xfer += $input->readString($elem1665); + $this->part_vals []= $elem1665; } $xfer += $input->readListEnd(); } else { @@ -168,8 +168,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1659) { - $xfer += $output->writeString($iter1659); + foreach ($this->part_vals as $iter1666) { + $xfer += $output->writeString($iter1666); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php index a4751d3ba9f0..685bd5a914da 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1660 = 0; - $_etype1663 = 0; - $xfer += $input->readListBegin($_etype1663, $_size1660); - for ($_i1664 = 0; $_i1664 < $_size1660; ++$_i1664) { - $elem1665 = null; - $elem1665 = new \metastore\Partition(); - $xfer += $elem1665->read($input); - $this->success []= $elem1665; + $_size1667 = 0; + $_etype1670 = 0; + $xfer += $input->readListBegin($_etype1670, $_size1667); + for ($_i1671 = 0; $_i1671 < $_size1667; ++$_i1671) { + $elem1672 = null; + $elem1672 = new \metastore\Partition(); + $xfer += $elem1672->read($input); + $this->success []= $elem1672; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1666) { - $xfer += $iter1666->write($output); + foreach ($this->success as $iter1673) { + $xfer += $iter1673->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php index fe977507604b..3b63b0f437c0 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_args.php @@ -146,13 +146,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1667 = 0; - $_etype1670 = 0; - $xfer += $input->readListBegin($_etype1670, $_size1667); - for ($_i1671 = 0; $_i1671 < $_size1667; ++$_i1671) { - $elem1672 = null; - $xfer += $input->readString($elem1672); - $this->part_vals []= $elem1672; + $_size1674 = 0; + $_etype1677 = 0; + $xfer += $input->readListBegin($_etype1677, $_size1674); + for ($_i1678 = 0; $_i1678 < $_size1674; ++$_i1678) { + $elem1679 = null; + $xfer += $input->readString($elem1679); + $this->part_vals []= $elem1679; } $xfer += $input->readListEnd(); } else { @@ -176,13 +176,13 @@ public function read($input) case 6: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1673 = 0; - $_etype1676 = 0; - $xfer += $input->readListBegin($_etype1676, $_size1673); - for ($_i1677 = 0; $_i1677 < $_size1673; ++$_i1677) { - $elem1678 = null; - $xfer += $input->readString($elem1678); - $this->group_names []= $elem1678; + $_size1680 = 0; + $_etype1683 = 0; + $xfer += $input->readListBegin($_etype1683, $_size1680); + for ($_i1684 = 0; $_i1684 < $_size1680; ++$_i1684) { + $elem1685 = null; + $xfer += $input->readString($elem1685); + $this->group_names []= $elem1685; } $xfer += $input->readListEnd(); } else { @@ -219,8 +219,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1679) { - $xfer += $output->writeString($iter1679); + foreach ($this->part_vals as $iter1686) { + $xfer += $output->writeString($iter1686); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); @@ -241,8 +241,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 6); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1680) { - $xfer += $output->writeString($iter1680); + foreach ($this->group_names as $iter1687) { + $xfer += $output->writeString($iter1687); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php index cd3ea604870b..f19d8923c62a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_ps_with_auth_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1681 = 0; - $_etype1684 = 0; - $xfer += $input->readListBegin($_etype1684, $_size1681); - for ($_i1685 = 0; $_i1685 < $_size1681; ++$_i1685) { - $elem1686 = null; - $elem1686 = new \metastore\Partition(); - $xfer += $elem1686->read($input); - $this->success []= $elem1686; + $_size1688 = 0; + $_etype1691 = 0; + $xfer += $input->readListBegin($_etype1691, $_size1688); + for ($_i1692 = 0; $_i1692 < $_size1688; ++$_i1692) { + $elem1693 = null; + $elem1693 = new \metastore\Partition(); + $xfer += $elem1693->read($input); + $this->success []= $elem1693; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1687) { - $xfer += $iter1687->write($output); + foreach ($this->success as $iter1694) { + $xfer += $iter1694->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php index 7e25f59da25c..0ab3c0c1e525 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_pspec_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1639 = 0; - $_etype1642 = 0; - $xfer += $input->readListBegin($_etype1642, $_size1639); - for ($_i1643 = 0; $_i1643 < $_size1639; ++$_i1643) { - $elem1644 = null; - $elem1644 = new \metastore\PartitionSpec(); - $xfer += $elem1644->read($input); - $this->success []= $elem1644; + $_size1646 = 0; + $_etype1649 = 0; + $xfer += $input->readListBegin($_etype1649, $_size1646); + for ($_i1650 = 0; $_i1650 < $_size1646; ++$_i1650) { + $elem1651 = null; + $elem1651 = new \metastore\PartitionSpec(); + $xfer += $elem1651->read($input); + $this->success []= $elem1651; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1645) { - $xfer += $iter1645->write($output); + foreach ($this->success as $iter1652) { + $xfer += $iter1652->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php index 42c9a65c494e..6566002f3a3f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1618 = 0; - $_etype1621 = 0; - $xfer += $input->readListBegin($_etype1621, $_size1618); - for ($_i1622 = 0; $_i1622 < $_size1618; ++$_i1622) { - $elem1623 = null; - $elem1623 = new \metastore\Partition(); - $xfer += $elem1623->read($input); - $this->success []= $elem1623; + $_size1625 = 0; + $_etype1628 = 0; + $xfer += $input->readListBegin($_etype1628, $_size1625); + for ($_i1629 = 0; $_i1629 < $_size1625; ++$_i1629) { + $elem1630 = null; + $elem1630 = new \metastore\Partition(); + $xfer += $elem1630->read($input); + $this->success []= $elem1630; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1624) { - $xfer += $iter1624->write($output); + foreach ($this->success as $iter1631) { + $xfer += $iter1631->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php index b54451581882..ebd29d773a67 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_args.php @@ -144,13 +144,13 @@ public function read($input) case 5: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1625 = 0; - $_etype1628 = 0; - $xfer += $input->readListBegin($_etype1628, $_size1625); - for ($_i1629 = 0; $_i1629 < $_size1625; ++$_i1629) { - $elem1630 = null; - $xfer += $input->readString($elem1630); - $this->group_names []= $elem1630; + $_size1632 = 0; + $_etype1635 = 0; + $xfer += $input->readListBegin($_etype1635, $_size1632); + for ($_i1636 = 0; $_i1636 < $_size1632; ++$_i1636) { + $elem1637 = null; + $xfer += $input->readString($elem1637); + $this->group_names []= $elem1637; } $xfer += $input->readListEnd(); } else { @@ -197,8 +197,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1631) { - $xfer += $output->writeString($iter1631); + foreach ($this->group_names as $iter1638) { + $xfer += $output->writeString($iter1638); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php index 9f06b78f393e..27169e09b5f8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_partitions_with_auth_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1632 = 0; - $_etype1635 = 0; - $xfer += $input->readListBegin($_etype1635, $_size1632); - for ($_i1636 = 0; $_i1636 < $_size1632; ++$_i1636) { - $elem1637 = null; - $elem1637 = new \metastore\Partition(); - $xfer += $elem1637->read($input); - $this->success []= $elem1637; + $_size1639 = 0; + $_etype1642 = 0; + $xfer += $input->readListBegin($_etype1642, $_size1639); + for ($_i1643 = 0; $_i1643 < $_size1639; ++$_i1643) { + $elem1644 = null; + $elem1644 = new \metastore\Partition(); + $xfer += $elem1644->read($input); + $this->success []= $elem1644; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1638) { - $xfer += $iter1638->write($output); + foreach ($this->success as $iter1645) { + $xfer += $iter1645->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php index ab1303b1f8ca..3e701a3cd77e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_privilege_set_args.php @@ -108,13 +108,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1827 = 0; - $_etype1830 = 0; - $xfer += $input->readListBegin($_etype1830, $_size1827); - for ($_i1831 = 0; $_i1831 < $_size1827; ++$_i1831) { - $elem1832 = null; - $xfer += $input->readString($elem1832); - $this->group_names []= $elem1832; + $_size1834 = 0; + $_etype1837 = 0; + $xfer += $input->readListBegin($_etype1837, $_size1834); + for ($_i1838 = 0; $_i1838 < $_size1834; ++$_i1838) { + $elem1839 = null; + $xfer += $input->readString($elem1839); + $this->group_names []= $elem1839; } $xfer += $input->readListEnd(); } else { @@ -154,8 +154,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1833) { - $xfer += $output->writeString($iter1833); + foreach ($this->group_names as $iter1840) { + $xfer += $output->writeString($iter1840); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php index 7353970ddf4c..ee595f86e948 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_role_names_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1813 = 0; - $_etype1816 = 0; - $xfer += $input->readListBegin($_etype1816, $_size1813); - for ($_i1817 = 0; $_i1817 < $_size1813; ++$_i1817) { - $elem1818 = null; - $xfer += $input->readString($elem1818); - $this->success []= $elem1818; + $_size1820 = 0; + $_etype1823 = 0; + $xfer += $input->readListBegin($_etype1823, $_size1820); + for ($_i1824 = 0; $_i1824 < $_size1820; ++$_i1824) { + $elem1825 = null; + $xfer += $input->readString($elem1825); + $this->success []= $elem1825; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1819) { - $xfer += $output->writeString($iter1819); + foreach ($this->success as $iter1826) { + $xfer += $output->writeString($iter1826); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php index 051d4230459a..24d692ee6fa1 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1892 = 0; - $_etype1895 = 0; - $xfer += $input->readListBegin($_etype1895, $_size1892); - for ($_i1896 = 0; $_i1896 < $_size1892; ++$_i1896) { - $elem1897 = null; - $elem1897 = new \metastore\RuntimeStat(); - $xfer += $elem1897->read($input); - $this->success []= $elem1897; + $_size1899 = 0; + $_etype1902 = 0; + $xfer += $input->readListBegin($_etype1902, $_size1899); + for ($_i1903 = 0; $_i1903 < $_size1899; ++$_i1903) { + $elem1904 = null; + $elem1904 = new \metastore\RuntimeStat(); + $xfer += $elem1904->read($input); + $this->success []= $elem1904; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1898) { - $xfer += $iter1898->write($output); + foreach ($this->success as $iter1905) { + $xfer += $iter1905->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php index b3afca738165..b532ad412a52 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php @@ -95,14 +95,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1885 = 0; - $_etype1888 = 0; - $xfer += $input->readListBegin($_etype1888, $_size1885); - for ($_i1889 = 0; $_i1889 < $_size1885; ++$_i1889) { - $elem1890 = null; - $elem1890 = new \metastore\SchemaVersion(); - $xfer += $elem1890->read($input); - $this->success []= $elem1890; + $_size1892 = 0; + $_etype1895 = 0; + $xfer += $input->readListBegin($_etype1895, $_size1892); + for ($_i1896 = 0; $_i1896 < $_size1892; ++$_i1896) { + $elem1897 = null; + $elem1897 = new \metastore\SchemaVersion(); + $xfer += $elem1897->read($input); + $this->success []= $elem1897; } $xfer += $input->readListEnd(); } else { @@ -145,8 +145,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1891) { - $xfer += $iter1891->write($output); + foreach ($this->success as $iter1898) { + $xfer += $iter1898->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php index f6ae9424cb74..70ed35fa2bc9 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1390 = 0; - $_etype1393 = 0; - $xfer += $input->readListBegin($_etype1393, $_size1390); - for ($_i1394 = 0; $_i1394 < $_size1390; ++$_i1394) { - $elem1395 = null; - $elem1395 = new \metastore\FieldSchema(); - $xfer += $elem1395->read($input); - $this->success []= $elem1395; + $_size1397 = 0; + $_etype1400 = 0; + $xfer += $input->readListBegin($_etype1400, $_size1397); + for ($_i1401 = 0; $_i1401 < $_size1397; ++$_i1401) { + $elem1402 = null; + $elem1402 = new \metastore\FieldSchema(); + $xfer += $elem1402->read($input); + $this->success []= $elem1402; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1396) { - $xfer += $iter1396->write($output); + foreach ($this->success as $iter1403) { + $xfer += $iter1403->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php index adc09cc0e92b..234e74394ebe 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_with_environment_context_result.php @@ -108,14 +108,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1397 = 0; - $_etype1400 = 0; - $xfer += $input->readListBegin($_etype1400, $_size1397); - for ($_i1401 = 0; $_i1401 < $_size1397; ++$_i1401) { - $elem1402 = null; - $elem1402 = new \metastore\FieldSchema(); - $xfer += $elem1402->read($input); - $this->success []= $elem1402; + $_size1404 = 0; + $_etype1407 = 0; + $xfer += $input->readListBegin($_etype1407, $_size1404); + for ($_i1408 = 0; $_i1408 < $_size1404; ++$_i1408) { + $elem1409 = null; + $elem1409 = new \metastore\FieldSchema(); + $xfer += $elem1409->read($input); + $this->success []= $elem1409; } $xfer += $input->readListEnd(); } else { @@ -166,8 +166,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1403) { - $xfer += $iter1403->write($output); + foreach ($this->success as $iter1410) { + $xfer += $iter1410->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php index eb41013a39aa..adf3a47ff11a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->tbl_types = array(); - $_size1481 = 0; - $_etype1484 = 0; - $xfer += $input->readListBegin($_etype1484, $_size1481); - for ($_i1485 = 0; $_i1485 < $_size1481; ++$_i1485) { - $elem1486 = null; - $xfer += $input->readString($elem1486); - $this->tbl_types []= $elem1486; + $_size1488 = 0; + $_etype1491 = 0; + $xfer += $input->readListBegin($_etype1491, $_size1488); + for ($_i1492 = 0; $_i1492 < $_size1488; ++$_i1492) { + $elem1493 = null; + $xfer += $input->readString($elem1493); + $this->tbl_types []= $elem1493; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('tbl_types', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->tbl_types)); - foreach ($this->tbl_types as $iter1487) { - $xfer += $output->writeString($iter1487); + foreach ($this->tbl_types as $iter1494) { + $xfer += $output->writeString($iter1494); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php index fa85fda009df..a991856ebb7a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_meta_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1488 = 0; - $_etype1491 = 0; - $xfer += $input->readListBegin($_etype1491, $_size1488); - for ($_i1492 = 0; $_i1492 < $_size1488; ++$_i1492) { - $elem1493 = null; - $elem1493 = new \metastore\TableMeta(); - $xfer += $elem1493->read($input); - $this->success []= $elem1493; + $_size1495 = 0; + $_etype1498 = 0; + $xfer += $input->readListBegin($_etype1498, $_size1495); + for ($_i1499 = 0; $_i1499 < $_size1495; ++$_i1499) { + $elem1500 = null; + $elem1500 = new \metastore\TableMeta(); + $xfer += $elem1500->read($input); + $this->success []= $elem1500; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1494) { - $xfer += $iter1494->write($output); + foreach ($this->success as $iter1501) { + $xfer += $iter1501->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php index e125030e775a..f0490e007289 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_names_by_filter_result.php @@ -107,13 +107,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1523 = 0; - $_etype1526 = 0; - $xfer += $input->readListBegin($_etype1526, $_size1523); - for ($_i1527 = 0; $_i1527 < $_size1523; ++$_i1527) { - $elem1528 = null; - $xfer += $input->readString($elem1528); - $this->success []= $elem1528; + $_size1530 = 0; + $_etype1533 = 0; + $xfer += $input->readListBegin($_etype1533, $_size1530); + for ($_i1534 = 0; $_i1534 < $_size1530; ++$_i1534) { + $elem1535 = null; + $xfer += $input->readString($elem1535); + $this->success []= $elem1535; } $xfer += $input->readListEnd(); } else { @@ -164,8 +164,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1529) { - $xfer += $output->writeString($iter1529); + foreach ($this->success as $iter1536) { + $xfer += $output->writeString($iter1536); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php index 8bdb9756a4b3..c8b8d72e0229 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_args.php @@ -87,13 +87,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->tbl_names = array(); - $_size1502 = 0; - $_etype1505 = 0; - $xfer += $input->readListBegin($_etype1505, $_size1502); - for ($_i1506 = 0; $_i1506 < $_size1502; ++$_i1506) { - $elem1507 = null; - $xfer += $input->readString($elem1507); - $this->tbl_names []= $elem1507; + $_size1509 = 0; + $_etype1512 = 0; + $xfer += $input->readListBegin($_etype1512, $_size1509); + for ($_i1513 = 0; $_i1513 < $_size1509; ++$_i1513) { + $elem1514 = null; + $xfer += $input->readString($elem1514); + $this->tbl_names []= $elem1514; } $xfer += $input->readListEnd(); } else { @@ -125,8 +125,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->tbl_names)); - foreach ($this->tbl_names as $iter1508) { - $xfer += $output->writeString($iter1508); + foreach ($this->tbl_names as $iter1515) { + $xfer += $output->writeString($iter1515); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php index fccbc17d1db8..b1da98ed7ad8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_table_objects_by_name_result.php @@ -69,14 +69,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1509 = 0; - $_etype1512 = 0; - $xfer += $input->readListBegin($_etype1512, $_size1509); - for ($_i1513 = 0; $_i1513 < $_size1509; ++$_i1513) { - $elem1514 = null; - $elem1514 = new \metastore\Table(); - $xfer += $elem1514->read($input); - $this->success []= $elem1514; + $_size1516 = 0; + $_etype1519 = 0; + $xfer += $input->readListBegin($_etype1519, $_size1516); + for ($_i1520 = 0; $_i1520 < $_size1516; ++$_i1520) { + $elem1521 = null; + $elem1521 = new \metastore\Table(); + $xfer += $elem1521->read($input); + $this->success []= $elem1521; } $xfer += $input->readListEnd(); } else { @@ -103,8 +103,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1515) { - $xfer += $iter1515->write($output); + foreach ($this->success as $iter1522) { + $xfer += $iter1522->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php index f5819a715114..3c10e66ead77 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_by_type_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1460 = 0; - $_etype1463 = 0; - $xfer += $input->readListBegin($_etype1463, $_size1460); - for ($_i1464 = 0; $_i1464 < $_size1460; ++$_i1464) { - $elem1465 = null; - $xfer += $input->readString($elem1465); - $this->success []= $elem1465; + $_size1467 = 0; + $_etype1470 = 0; + $xfer += $input->readListBegin($_etype1470, $_size1467); + for ($_i1471 = 0; $_i1471 < $_size1467; ++$_i1471) { + $elem1472 = null; + $xfer += $input->readString($elem1472); + $this->success []= $elem1472; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1466) { - $xfer += $output->writeString($iter1466); + foreach ($this->success as $iter1473) { + $xfer += $output->writeString($iter1473); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php index 6d23127f2712..6ca1744e9029 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_ext_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1516 = 0; - $_etype1519 = 0; - $xfer += $input->readListBegin($_etype1519, $_size1516); - for ($_i1520 = 0; $_i1520 < $_size1516; ++$_i1520) { - $elem1521 = null; - $elem1521 = new \metastore\ExtendedTableInfo(); - $xfer += $elem1521->read($input); - $this->success []= $elem1521; + $_size1523 = 0; + $_etype1526 = 0; + $xfer += $input->readListBegin($_etype1526, $_size1523); + for ($_i1527 = 0; $_i1527 < $_size1523; ++$_i1527) { + $elem1528 = null; + $elem1528 = new \metastore\ExtendedTableInfo(); + $xfer += $elem1528->read($input); + $this->success []= $elem1528; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1522) { - $xfer += $iter1522->write($output); + foreach ($this->success as $iter1529) { + $xfer += $iter1529->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php index 1e24c0bae93a..7124e0f5b987 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_tables_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1453 = 0; - $_etype1456 = 0; - $xfer += $input->readListBegin($_etype1456, $_size1453); - for ($_i1457 = 0; $_i1457 < $_size1453; ++$_i1457) { - $elem1458 = null; - $xfer += $input->readString($elem1458); - $this->success []= $elem1458; + $_size1460 = 0; + $_etype1463 = 0; + $xfer += $input->readListBegin($_etype1463, $_size1460); + for ($_i1464 = 0; $_i1464 < $_size1460; ++$_i1464) { + $elem1465 = null; + $xfer += $input->readString($elem1465); + $this->success []= $elem1465; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1459) { - $xfer += $output->writeString($iter1459); + foreach ($this->success as $iter1466) { + $xfer += $output->writeString($iter1466); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php index 45daa41ab670..4ba3c2b1fff1 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_type_all_result.php @@ -86,17 +86,17 @@ public function read($input) case 0: if ($ftype == TType::MAP) { $this->success = array(); - $_size1367 = 0; - $_ktype1368 = 0; - $_vtype1369 = 0; - $xfer += $input->readMapBegin($_ktype1368, $_vtype1369, $_size1367); - for ($_i1371 = 0; $_i1371 < $_size1367; ++$_i1371) { - $key1372 = ''; - $val1373 = new \metastore\Type(); - $xfer += $input->readString($key1372); - $val1373 = new \metastore\Type(); - $xfer += $val1373->read($input); - $this->success[$key1372] = $val1373; + $_size1374 = 0; + $_ktype1375 = 0; + $_vtype1376 = 0; + $xfer += $input->readMapBegin($_ktype1375, $_vtype1376, $_size1374); + for ($_i1378 = 0; $_i1378 < $_size1374; ++$_i1378) { + $key1379 = ''; + $val1380 = new \metastore\Type(); + $xfer += $input->readString($key1379); + $val1380 = new \metastore\Type(); + $xfer += $val1380->read($input); + $this->success[$key1379] = $val1380; } $xfer += $input->readMapEnd(); } else { @@ -131,9 +131,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::MAP, 0); $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success)); - foreach ($this->success as $kiter1374 => $viter1375) { - $xfer += $output->writeString($kiter1374); - $xfer += $viter1375->write($output); + foreach ($this->success as $kiter1381 => $viter1382) { + $xfer += $output->writeString($kiter1381); + $xfer += $viter1382->write($output); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php index 38c1c98781de..e0bc6a0013ea 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_isPartitionMarkedForEvent_args.php @@ -123,16 +123,16 @@ public function read($input) case 3: if ($ftype == TType::MAP) { $this->part_vals = array(); - $_size1797 = 0; - $_ktype1798 = 0; - $_vtype1799 = 0; - $xfer += $input->readMapBegin($_ktype1798, $_vtype1799, $_size1797); - for ($_i1801 = 0; $_i1801 < $_size1797; ++$_i1801) { - $key1802 = ''; - $val1803 = ''; - $xfer += $input->readString($key1802); - $xfer += $input->readString($val1803); - $this->part_vals[$key1802] = $val1803; + $_size1804 = 0; + $_ktype1805 = 0; + $_vtype1806 = 0; + $xfer += $input->readMapBegin($_ktype1805, $_vtype1806, $_size1804); + for ($_i1808 = 0; $_i1808 < $_size1804; ++$_i1808) { + $key1809 = ''; + $val1810 = ''; + $xfer += $input->readString($key1809); + $xfer += $input->readString($val1810); + $this->part_vals[$key1809] = $val1810; } $xfer += $input->readMapEnd(); } else { @@ -176,9 +176,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $kiter1804 => $viter1805) { - $xfer += $output->writeString($kiter1804); - $xfer += $output->writeString($viter1805); + foreach ($this->part_vals as $kiter1811 => $viter1812) { + $xfer += $output->writeString($kiter1811); + $xfer += $output->writeString($viter1812); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php index 0c04dbcb0aea..74058fdfaf54 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_privileges_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1834 = 0; - $_etype1837 = 0; - $xfer += $input->readListBegin($_etype1837, $_size1834); - for ($_i1838 = 0; $_i1838 < $_size1834; ++$_i1838) { - $elem1839 = null; - $elem1839 = new \metastore\HiveObjectPrivilege(); - $xfer += $elem1839->read($input); - $this->success []= $elem1839; + $_size1841 = 0; + $_etype1844 = 0; + $xfer += $input->readListBegin($_etype1844, $_size1841); + for ($_i1845 = 0; $_i1845 < $_size1841; ++$_i1845) { + $elem1846 = null; + $elem1846 = new \metastore\HiveObjectPrivilege(); + $xfer += $elem1846->read($input); + $this->success []= $elem1846; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1840) { - $xfer += $iter1840->write($output); + foreach ($this->success as $iter1847) { + $xfer += $iter1847->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php index 860a92a023cc..26d19cd9fc87 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_list_roles_result.php @@ -82,14 +82,14 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1820 = 0; - $_etype1823 = 0; - $xfer += $input->readListBegin($_etype1823, $_size1820); - for ($_i1824 = 0; $_i1824 < $_size1820; ++$_i1824) { - $elem1825 = null; - $elem1825 = new \metastore\Role(); - $xfer += $elem1825->read($input); - $this->success []= $elem1825; + $_size1827 = 0; + $_etype1830 = 0; + $xfer += $input->readListBegin($_etype1830, $_size1827); + for ($_i1831 = 0; $_i1831 < $_size1827; ++$_i1831) { + $elem1832 = null; + $elem1832 = new \metastore\Role(); + $xfer += $elem1832->read($input); + $this->success []= $elem1832; } $xfer += $input->readListEnd(); } else { @@ -124,8 +124,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRUCT, count($this->success)); - foreach ($this->success as $iter1826) { - $xfer += $iter1826->write($output); + foreach ($this->success as $iter1833) { + $xfer += $iter1833->write($output); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php index a335b9c9d02f..4391b3a0f4ba 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_markPartitionForEvent_args.php @@ -123,16 +123,16 @@ public function read($input) case 3: if ($ftype == TType::MAP) { $this->part_vals = array(); - $_size1788 = 0; - $_ktype1789 = 0; - $_vtype1790 = 0; - $xfer += $input->readMapBegin($_ktype1789, $_vtype1790, $_size1788); - for ($_i1792 = 0; $_i1792 < $_size1788; ++$_i1792) { - $key1793 = ''; - $val1794 = ''; - $xfer += $input->readString($key1793); - $xfer += $input->readString($val1794); - $this->part_vals[$key1793] = $val1794; + $_size1795 = 0; + $_ktype1796 = 0; + $_vtype1797 = 0; + $xfer += $input->readMapBegin($_ktype1796, $_vtype1797, $_size1795); + for ($_i1799 = 0; $_i1799 < $_size1795; ++$_i1799) { + $key1800 = ''; + $val1801 = ''; + $xfer += $input->readString($key1800); + $xfer += $input->readString($val1801); + $this->part_vals[$key1800] = $val1801; } $xfer += $input->readMapEnd(); } else { @@ -176,9 +176,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $kiter1795 => $viter1796) { - $xfer += $output->writeString($kiter1795); - $xfer += $output->writeString($viter1796); + foreach ($this->part_vals as $kiter1802 => $viter1803) { + $xfer += $output->writeString($kiter1802); + $xfer += $output->writeString($viter1803); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php index 32a792c46d56..130c0fe3cd77 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_has_valid_characters_args.php @@ -80,13 +80,13 @@ public function read($input) case 1: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1765 = 0; - $_etype1768 = 0; - $xfer += $input->readListBegin($_etype1768, $_size1765); - for ($_i1769 = 0; $_i1769 < $_size1765; ++$_i1769) { - $elem1770 = null; - $xfer += $input->readString($elem1770); - $this->part_vals []= $elem1770; + $_size1772 = 0; + $_etype1775 = 0; + $xfer += $input->readListBegin($_etype1775, $_size1772); + for ($_i1776 = 0; $_i1776 < $_size1772; ++$_i1776) { + $elem1777 = null; + $xfer += $input->readString($elem1777); + $this->part_vals []= $elem1777; } $xfer += $input->readListEnd(); } else { @@ -120,8 +120,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 1); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1771) { - $xfer += $output->writeString($iter1771); + foreach ($this->part_vals as $iter1778) { + $xfer += $output->writeString($iter1778); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php index 18263b34051d..db788f95e942 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_spec_result.php @@ -85,16 +85,16 @@ public function read($input) case 0: if ($ftype == TType::MAP) { $this->success = array(); - $_size1779 = 0; - $_ktype1780 = 0; - $_vtype1781 = 0; - $xfer += $input->readMapBegin($_ktype1780, $_vtype1781, $_size1779); - for ($_i1783 = 0; $_i1783 < $_size1779; ++$_i1783) { - $key1784 = ''; - $val1785 = ''; - $xfer += $input->readString($key1784); - $xfer += $input->readString($val1785); - $this->success[$key1784] = $val1785; + $_size1786 = 0; + $_ktype1787 = 0; + $_vtype1788 = 0; + $xfer += $input->readMapBegin($_ktype1787, $_vtype1788, $_size1786); + for ($_i1790 = 0; $_i1790 < $_size1786; ++$_i1790) { + $key1791 = ''; + $val1792 = ''; + $xfer += $input->readString($key1791); + $xfer += $input->readString($val1792); + $this->success[$key1791] = $val1792; } $xfer += $input->readMapEnd(); } else { @@ -129,9 +129,9 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::MAP, 0); $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); - foreach ($this->success as $kiter1786 => $viter1787) { - $xfer += $output->writeString($kiter1786); - $xfer += $output->writeString($viter1787); + foreach ($this->success as $kiter1793 => $viter1794) { + $xfer += $output->writeString($kiter1793); + $xfer += $output->writeString($viter1794); } $output->writeMapEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php index bd6f8e46f7d0..eae456401469 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_partition_name_to_vals_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1772 = 0; - $_etype1775 = 0; - $xfer += $input->readListBegin($_etype1775, $_size1772); - for ($_i1776 = 0; $_i1776 < $_size1772; ++$_i1776) { - $elem1777 = null; - $xfer += $input->readString($elem1777); - $this->success []= $elem1777; + $_size1779 = 0; + $_etype1782 = 0; + $xfer += $input->readListBegin($_etype1782, $_size1779); + for ($_i1783 = 0; $_i1783 < $_size1779; ++$_i1783) { + $elem1784 = null; + $xfer += $input->readString($elem1784); + $this->success []= $elem1784; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1778) { - $xfer += $output->writeString($iter1778); + foreach ($this->success as $iter1785) { + $xfer += $output->writeString($iter1785); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php index a0ebcef960cd..dfb455df16fa 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_rename_partition_args.php @@ -119,13 +119,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size1758 = 0; - $_etype1761 = 0; - $xfer += $input->readListBegin($_etype1761, $_size1758); - for ($_i1762 = 0; $_i1762 < $_size1758; ++$_i1762) { - $elem1763 = null; - $xfer += $input->readString($elem1763); - $this->part_vals []= $elem1763; + $_size1765 = 0; + $_etype1768 = 0; + $xfer += $input->readListBegin($_etype1768, $_size1765); + for ($_i1769 = 0; $_i1769 < $_size1765; ++$_i1769) { + $elem1770 = null; + $xfer += $input->readString($elem1770); + $this->part_vals []= $elem1770; } $xfer += $input->readListEnd(); } else { @@ -170,8 +170,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->part_vals)); - foreach ($this->part_vals as $iter1764) { - $xfer += $output->writeString($iter1764); + foreach ($this->part_vals as $iter1771) { + $xfer += $output->writeString($iter1771); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php index 6ce4ee0b957d..012c00c0b81e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_args.php @@ -87,13 +87,13 @@ public function read($input) case 2: if ($ftype == TType::LST) { $this->group_names = array(); - $_size1841 = 0; - $_etype1844 = 0; - $xfer += $input->readListBegin($_etype1844, $_size1841); - for ($_i1845 = 0; $_i1845 < $_size1841; ++$_i1845) { - $elem1846 = null; - $xfer += $input->readString($elem1846); - $this->group_names []= $elem1846; + $_size1848 = 0; + $_etype1851 = 0; + $xfer += $input->readListBegin($_etype1851, $_size1848); + for ($_i1852 = 0; $_i1852 < $_size1848; ++$_i1852) { + $elem1853 = null; + $xfer += $input->readString($elem1853); + $this->group_names []= $elem1853; } $xfer += $input->readListEnd(); } else { @@ -125,8 +125,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('group_names', TType::LST, 2); $output->writeListBegin(TType::STRING, count($this->group_names)); - foreach ($this->group_names as $iter1847) { - $xfer += $output->writeString($iter1847); + foreach ($this->group_names as $iter1854) { + $xfer += $output->writeString($iter1854); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php index 7436e4e22bb7..ec466c2cfba7 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_set_ugi_result.php @@ -81,13 +81,13 @@ public function read($input) case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size1848 = 0; - $_etype1851 = 0; - $xfer += $input->readListBegin($_etype1851, $_size1848); - for ($_i1852 = 0; $_i1852 < $_size1848; ++$_i1852) { - $elem1853 = null; - $xfer += $input->readString($elem1853); - $this->success []= $elem1853; + $_size1855 = 0; + $_etype1858 = 0; + $xfer += $input->readListBegin($_etype1858, $_size1855); + for ($_i1859 = 0; $_i1859 < $_size1855; ++$_i1859) { + $elem1860 = null; + $xfer += $input->readString($elem1860); + $this->success []= $elem1860; } $xfer += $input->readListEnd(); } else { @@ -122,8 +122,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('success', TType::LST, 0); $output->writeListBegin(TType::STRING, count($this->success)); - foreach ($this->success as $iter1854) { - $xfer += $output->writeString($iter1854); + foreach ($this->success as $iter1861) { + $xfer += $output->writeString($iter1861); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php index 9b895253caeb..61d43f864f30 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_truncate_table_args.php @@ -106,13 +106,13 @@ public function read($input) case 3: if ($ftype == TType::LST) { $this->partNames = array(); - $_size1446 = 0; - $_etype1449 = 0; - $xfer += $input->readListBegin($_etype1449, $_size1446); - for ($_i1450 = 0; $_i1450 < $_size1446; ++$_i1450) { - $elem1451 = null; - $xfer += $input->readString($elem1451); - $this->partNames []= $elem1451; + $_size1453 = 0; + $_etype1456 = 0; + $xfer += $input->readListBegin($_etype1456, $_size1453); + for ($_i1457 = 0; $_i1457 < $_size1453; ++$_i1457) { + $elem1458 = null; + $xfer += $input->readString($elem1458); + $this->partNames []= $elem1458; } $xfer += $input->readListEnd(); } else { @@ -149,8 +149,8 @@ public function write($output) } $xfer += $output->writeFieldBegin('partNames', TType::LST, 3); $output->writeListBegin(TType::STRING, count($this->partNames)); - foreach ($this->partNames as $iter1452) { - $xfer += $output->writeString($iter1452); + foreach ($this->partNames as $iter1459) { + $xfer += $output->writeString($iter1459); } $output->writeListEnd(); $xfer += $output->writeFieldEnd(); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py index fd7009c45ca5..80ea3fda4e53 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py @@ -22459,10 +22459,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1346, _size1343) = iprot.readListBegin() - for _i1347 in range(_size1343): - _elem1348 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1348) + (_etype1353, _size1350) = iprot.readListBegin() + for _i1354 in range(_size1350): + _elem1355 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1355) iprot.readListEnd() else: iprot.skip(ftype) @@ -22484,8 +22484,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1349 in self.success: - oprot.writeString(iter1349.encode('utf-8') if sys.version_info[0] == 2 else iter1349) + for iter1356 in self.success: + oprot.writeString(iter1356.encode('utf-8') if sys.version_info[0] == 2 else iter1356) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -22583,10 +22583,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1353, _size1350) = iprot.readListBegin() - for _i1354 in range(_size1350): - _elem1355 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1355) + (_etype1360, _size1357) = iprot.readListBegin() + for _i1361 in range(_size1357): + _elem1362 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1362) iprot.readListEnd() else: iprot.skip(ftype) @@ -22608,8 +22608,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1356 in self.success: - oprot.writeString(iter1356.encode('utf-8') if sys.version_info[0] == 2 else iter1356) + for iter1363 in self.success: + oprot.writeString(iter1363.encode('utf-8') if sys.version_info[0] == 2 else iter1363) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -23326,10 +23326,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1360, _size1357) = iprot.readListBegin() - for _i1361 in range(_size1357): - _elem1362 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1362) + (_etype1367, _size1364) = iprot.readListBegin() + for _i1368 in range(_size1364): + _elem1369 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1369) iprot.readListEnd() else: iprot.skip(ftype) @@ -23351,8 +23351,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1363 in self.success: - oprot.writeString(iter1363.encode('utf-8') if sys.version_info[0] == 2 else iter1363) + for iter1370 in self.success: + oprot.writeString(iter1370.encode('utf-8') if sys.version_info[0] == 2 else iter1370) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24073,12 +24073,12 @@ def read(self, iprot): if fid == 0: if ftype == TType.MAP: self.success = {} - (_ktype1365, _vtype1366, _size1364) = iprot.readMapBegin() - for _i1368 in range(_size1364): - _key1369 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1370 = Type() - _val1370.read(iprot) - self.success[_key1369] = _val1370 + (_ktype1372, _vtype1373, _size1371) = iprot.readMapBegin() + for _i1375 in range(_size1371): + _key1376 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1377 = Type() + _val1377.read(iprot) + self.success[_key1376] = _val1377 iprot.readMapEnd() else: iprot.skip(ftype) @@ -24100,9 +24100,9 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.MAP, 0) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success)) - for kiter1371, viter1372 in self.success.items(): - oprot.writeString(kiter1371.encode('utf-8') if sys.version_info[0] == 2 else kiter1371) - viter1372.write(oprot) + for kiter1378, viter1379 in self.success.items(): + oprot.writeString(kiter1378.encode('utf-8') if sys.version_info[0] == 2 else kiter1378) + viter1379.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.o2 is not None: @@ -24235,11 +24235,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1376, _size1373) = iprot.readListBegin() - for _i1377 in range(_size1373): - _elem1378 = FieldSchema() - _elem1378.read(iprot) - self.success.append(_elem1378) + (_etype1383, _size1380) = iprot.readListBegin() + for _i1384 in range(_size1380): + _elem1385 = FieldSchema() + _elem1385.read(iprot) + self.success.append(_elem1385) iprot.readListEnd() else: iprot.skip(ftype) @@ -24271,8 +24271,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1379 in self.success: - iter1379.write(oprot) + for iter1386 in self.success: + iter1386.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24428,11 +24428,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1383, _size1380) = iprot.readListBegin() - for _i1384 in range(_size1380): - _elem1385 = FieldSchema() - _elem1385.read(iprot) - self.success.append(_elem1385) + (_etype1390, _size1387) = iprot.readListBegin() + for _i1391 in range(_size1387): + _elem1392 = FieldSchema() + _elem1392.read(iprot) + self.success.append(_elem1392) iprot.readListEnd() else: iprot.skip(ftype) @@ -24464,8 +24464,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1386 in self.success: - iter1386.write(oprot) + for iter1393 in self.success: + iter1393.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24769,11 +24769,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1390, _size1387) = iprot.readListBegin() - for _i1391 in range(_size1387): - _elem1392 = FieldSchema() - _elem1392.read(iprot) - self.success.append(_elem1392) + (_etype1397, _size1394) = iprot.readListBegin() + for _i1398 in range(_size1394): + _elem1399 = FieldSchema() + _elem1399.read(iprot) + self.success.append(_elem1399) iprot.readListEnd() else: iprot.skip(ftype) @@ -24805,8 +24805,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1393 in self.success: - iter1393.write(oprot) + for iter1400 in self.success: + iter1400.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -24962,11 +24962,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1397, _size1394) = iprot.readListBegin() - for _i1398 in range(_size1394): - _elem1399 = FieldSchema() - _elem1399.read(iprot) - self.success.append(_elem1399) + (_etype1404, _size1401) = iprot.readListBegin() + for _i1405 in range(_size1401): + _elem1406 = FieldSchema() + _elem1406.read(iprot) + self.success.append(_elem1406) iprot.readListEnd() else: iprot.skip(ftype) @@ -24998,8 +24998,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1400 in self.success: - iter1400.write(oprot) + for iter1407 in self.success: + iter1407.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -25576,66 +25576,66 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.primaryKeys = [] - (_etype1404, _size1401) = iprot.readListBegin() - for _i1405 in range(_size1401): - _elem1406 = SQLPrimaryKey() - _elem1406.read(iprot) - self.primaryKeys.append(_elem1406) + (_etype1411, _size1408) = iprot.readListBegin() + for _i1412 in range(_size1408): + _elem1413 = SQLPrimaryKey() + _elem1413.read(iprot) + self.primaryKeys.append(_elem1413) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: self.foreignKeys = [] - (_etype1410, _size1407) = iprot.readListBegin() - for _i1411 in range(_size1407): - _elem1412 = SQLForeignKey() - _elem1412.read(iprot) - self.foreignKeys.append(_elem1412) + (_etype1417, _size1414) = iprot.readListBegin() + for _i1418 in range(_size1414): + _elem1419 = SQLForeignKey() + _elem1419.read(iprot) + self.foreignKeys.append(_elem1419) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.LIST: self.uniqueConstraints = [] - (_etype1416, _size1413) = iprot.readListBegin() - for _i1417 in range(_size1413): - _elem1418 = SQLUniqueConstraint() - _elem1418.read(iprot) - self.uniqueConstraints.append(_elem1418) + (_etype1423, _size1420) = iprot.readListBegin() + for _i1424 in range(_size1420): + _elem1425 = SQLUniqueConstraint() + _elem1425.read(iprot) + self.uniqueConstraints.append(_elem1425) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 5: if ftype == TType.LIST: self.notNullConstraints = [] - (_etype1422, _size1419) = iprot.readListBegin() - for _i1423 in range(_size1419): - _elem1424 = SQLNotNullConstraint() - _elem1424.read(iprot) - self.notNullConstraints.append(_elem1424) + (_etype1429, _size1426) = iprot.readListBegin() + for _i1430 in range(_size1426): + _elem1431 = SQLNotNullConstraint() + _elem1431.read(iprot) + self.notNullConstraints.append(_elem1431) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 6: if ftype == TType.LIST: self.defaultConstraints = [] - (_etype1428, _size1425) = iprot.readListBegin() - for _i1429 in range(_size1425): - _elem1430 = SQLDefaultConstraint() - _elem1430.read(iprot) - self.defaultConstraints.append(_elem1430) + (_etype1435, _size1432) = iprot.readListBegin() + for _i1436 in range(_size1432): + _elem1437 = SQLDefaultConstraint() + _elem1437.read(iprot) + self.defaultConstraints.append(_elem1437) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 7: if ftype == TType.LIST: self.checkConstraints = [] - (_etype1434, _size1431) = iprot.readListBegin() - for _i1435 in range(_size1431): - _elem1436 = SQLCheckConstraint() - _elem1436.read(iprot) - self.checkConstraints.append(_elem1436) + (_etype1441, _size1438) = iprot.readListBegin() + for _i1442 in range(_size1438): + _elem1443 = SQLCheckConstraint() + _elem1443.read(iprot) + self.checkConstraints.append(_elem1443) iprot.readListEnd() else: iprot.skip(ftype) @@ -25656,43 +25656,43 @@ def write(self, oprot): if self.primaryKeys is not None: oprot.writeFieldBegin('primaryKeys', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys)) - for iter1437 in self.primaryKeys: - iter1437.write(oprot) + for iter1444 in self.primaryKeys: + iter1444.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.foreignKeys is not None: oprot.writeFieldBegin('foreignKeys', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys)) - for iter1438 in self.foreignKeys: - iter1438.write(oprot) + for iter1445 in self.foreignKeys: + iter1445.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.uniqueConstraints is not None: oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4) oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints)) - for iter1439 in self.uniqueConstraints: - iter1439.write(oprot) + for iter1446 in self.uniqueConstraints: + iter1446.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.notNullConstraints is not None: oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5) oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints)) - for iter1440 in self.notNullConstraints: - iter1440.write(oprot) + for iter1447 in self.notNullConstraints: + iter1447.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.defaultConstraints is not None: oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6) oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints)) - for iter1441 in self.defaultConstraints: - iter1441.write(oprot) + for iter1448 in self.defaultConstraints: + iter1448.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.checkConstraints is not None: oprot.writeFieldBegin('checkConstraints', TType.LIST, 7) oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints)) - for iter1442 in self.checkConstraints: - iter1442.write(oprot) + for iter1449 in self.checkConstraints: + iter1449.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27485,10 +27485,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.partNames = [] - (_etype1446, _size1443) = iprot.readListBegin() - for _i1447 in range(_size1443): - _elem1448 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partNames.append(_elem1448) + (_etype1453, _size1450) = iprot.readListBegin() + for _i1454 in range(_size1450): + _elem1455 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partNames.append(_elem1455) iprot.readListEnd() else: iprot.skip(ftype) @@ -27513,8 +27513,8 @@ def write(self, oprot): if self.partNames is not None: oprot.writeFieldBegin('partNames', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.partNames)) - for iter1449 in self.partNames: - oprot.writeString(iter1449.encode('utf-8') if sys.version_info[0] == 2 else iter1449) + for iter1456 in self.partNames: + oprot.writeString(iter1456.encode('utf-8') if sys.version_info[0] == 2 else iter1456) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -27840,10 +27840,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1453, _size1450) = iprot.readListBegin() - for _i1454 in range(_size1450): - _elem1455 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1455) + (_etype1460, _size1457) = iprot.readListBegin() + for _i1461 in range(_size1457): + _elem1462 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1462) iprot.readListEnd() else: iprot.skip(ftype) @@ -27865,8 +27865,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1456 in self.success: - oprot.writeString(iter1456.encode('utf-8') if sys.version_info[0] == 2 else iter1456) + for iter1463 in self.success: + oprot.writeString(iter1463.encode('utf-8') if sys.version_info[0] == 2 else iter1463) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28007,10 +28007,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1460, _size1457) = iprot.readListBegin() - for _i1461 in range(_size1457): - _elem1462 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1462) + (_etype1467, _size1464) = iprot.readListBegin() + for _i1468 in range(_size1464): + _elem1469 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1469) iprot.readListEnd() else: iprot.skip(ftype) @@ -28032,8 +28032,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1463 in self.success: - oprot.writeString(iter1463.encode('utf-8') if sys.version_info[0] == 2 else iter1463) + for iter1470 in self.success: + oprot.writeString(iter1470.encode('utf-8') if sys.version_info[0] == 2 else iter1470) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28131,11 +28131,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1467, _size1464) = iprot.readListBegin() - for _i1468 in range(_size1464): - _elem1469 = Table() - _elem1469.read(iprot) - self.success.append(_elem1469) + (_etype1474, _size1471) = iprot.readListBegin() + for _i1475 in range(_size1471): + _elem1476 = Table() + _elem1476.read(iprot) + self.success.append(_elem1476) iprot.readListEnd() else: iprot.skip(ftype) @@ -28157,8 +28157,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1470 in self.success: - iter1470.write(oprot) + for iter1477 in self.success: + iter1477.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28275,10 +28275,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1474, _size1471) = iprot.readListBegin() - for _i1475 in range(_size1471): - _elem1476 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1476) + (_etype1481, _size1478) = iprot.readListBegin() + for _i1482 in range(_size1478): + _elem1483 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1483) iprot.readListEnd() else: iprot.skip(ftype) @@ -28300,8 +28300,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1477 in self.success: - oprot.writeString(iter1477.encode('utf-8') if sys.version_info[0] == 2 else iter1477) + for iter1484 in self.success: + oprot.writeString(iter1484.encode('utf-8') if sys.version_info[0] == 2 else iter1484) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28368,10 +28368,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.tbl_types = [] - (_etype1481, _size1478) = iprot.readListBegin() - for _i1482 in range(_size1478): - _elem1483 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.tbl_types.append(_elem1483) + (_etype1488, _size1485) = iprot.readListBegin() + for _i1489 in range(_size1485): + _elem1490 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tbl_types.append(_elem1490) iprot.readListEnd() else: iprot.skip(ftype) @@ -28396,8 +28396,8 @@ def write(self, oprot): if self.tbl_types is not None: oprot.writeFieldBegin('tbl_types', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.tbl_types)) - for iter1484 in self.tbl_types: - oprot.writeString(iter1484.encode('utf-8') if sys.version_info[0] == 2 else iter1484) + for iter1491 in self.tbl_types: + oprot.writeString(iter1491.encode('utf-8') if sys.version_info[0] == 2 else iter1491) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28450,11 +28450,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1488, _size1485) = iprot.readListBegin() - for _i1489 in range(_size1485): - _elem1490 = TableMeta() - _elem1490.read(iprot) - self.success.append(_elem1490) + (_etype1495, _size1492) = iprot.readListBegin() + for _i1496 in range(_size1492): + _elem1497 = TableMeta() + _elem1497.read(iprot) + self.success.append(_elem1497) iprot.readListEnd() else: iprot.skip(ftype) @@ -28476,8 +28476,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1491 in self.success: - iter1491.write(oprot) + for iter1498 in self.success: + iter1498.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28594,10 +28594,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1495, _size1492) = iprot.readListBegin() - for _i1496 in range(_size1492): - _elem1497 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1497) + (_etype1502, _size1499) = iprot.readListBegin() + for _i1503 in range(_size1499): + _elem1504 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1504) iprot.readListEnd() else: iprot.skip(ftype) @@ -28619,8 +28619,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1498 in self.success: - oprot.writeString(iter1498.encode('utf-8') if sys.version_info[0] == 2 else iter1498) + for iter1505 in self.success: + oprot.writeString(iter1505.encode('utf-8') if sys.version_info[0] == 2 else iter1505) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -28840,10 +28840,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.tbl_names = [] - (_etype1502, _size1499) = iprot.readListBegin() - for _i1503 in range(_size1499): - _elem1504 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.tbl_names.append(_elem1504) + (_etype1509, _size1506) = iprot.readListBegin() + for _i1510 in range(_size1506): + _elem1511 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.tbl_names.append(_elem1511) iprot.readListEnd() else: iprot.skip(ftype) @@ -28864,8 +28864,8 @@ def write(self, oprot): if self.tbl_names is not None: oprot.writeFieldBegin('tbl_names', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.tbl_names)) - for iter1505 in self.tbl_names: - oprot.writeString(iter1505.encode('utf-8') if sys.version_info[0] == 2 else iter1505) + for iter1512 in self.tbl_names: + oprot.writeString(iter1512.encode('utf-8') if sys.version_info[0] == 2 else iter1512) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28915,11 +28915,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1509, _size1506) = iprot.readListBegin() - for _i1510 in range(_size1506): - _elem1511 = Table() - _elem1511.read(iprot) - self.success.append(_elem1511) + (_etype1516, _size1513) = iprot.readListBegin() + for _i1517 in range(_size1513): + _elem1518 = Table() + _elem1518.read(iprot) + self.success.append(_elem1518) iprot.readListEnd() else: iprot.skip(ftype) @@ -28936,8 +28936,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1512 in self.success: - iter1512.write(oprot) + for iter1519 in self.success: + iter1519.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -29050,11 +29050,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1516, _size1513) = iprot.readListBegin() - for _i1517 in range(_size1513): - _elem1518 = ExtendedTableInfo() - _elem1518.read(iprot) - self.success.append(_elem1518) + (_etype1523, _size1520) = iprot.readListBegin() + for _i1524 in range(_size1520): + _elem1525 = ExtendedTableInfo() + _elem1525.read(iprot) + self.success.append(_elem1525) iprot.readListEnd() else: iprot.skip(ftype) @@ -29076,8 +29076,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1519 in self.success: - iter1519.write(oprot) + for iter1526 in self.success: + iter1526.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -29890,10 +29890,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1523, _size1520) = iprot.readListBegin() - for _i1524 in range(_size1520): - _elem1525 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1525) + (_etype1530, _size1527) = iprot.readListBegin() + for _i1531 in range(_size1527): + _elem1532 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1532) iprot.readListEnd() else: iprot.skip(ftype) @@ -29925,8 +29925,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1526 in self.success: - oprot.writeString(iter1526.encode('utf-8') if sys.version_info[0] == 2 else iter1526) + for iter1533 in self.success: + oprot.writeString(iter1533.encode('utf-8') if sys.version_info[0] == 2 else iter1533) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -30981,11 +30981,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.new_parts = [] - (_etype1530, _size1527) = iprot.readListBegin() - for _i1531 in range(_size1527): - _elem1532 = Partition() - _elem1532.read(iprot) - self.new_parts.append(_elem1532) + (_etype1537, _size1534) = iprot.readListBegin() + for _i1538 in range(_size1534): + _elem1539 = Partition() + _elem1539.read(iprot) + self.new_parts.append(_elem1539) iprot.readListEnd() else: iprot.skip(ftype) @@ -31002,8 +31002,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1533 in self.new_parts: - iter1533.write(oprot) + for iter1540 in self.new_parts: + iter1540.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -31149,11 +31149,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.new_parts = [] - (_etype1537, _size1534) = iprot.readListBegin() - for _i1538 in range(_size1534): - _elem1539 = PartitionSpec() - _elem1539.read(iprot) - self.new_parts.append(_elem1539) + (_etype1544, _size1541) = iprot.readListBegin() + for _i1545 in range(_size1541): + _elem1546 = PartitionSpec() + _elem1546.read(iprot) + self.new_parts.append(_elem1546) iprot.readListEnd() else: iprot.skip(ftype) @@ -31170,8 +31170,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1540 in self.new_parts: - iter1540.write(oprot) + for iter1547 in self.new_parts: + iter1547.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -31331,10 +31331,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1544, _size1541) = iprot.readListBegin() - for _i1545 in range(_size1541): - _elem1546 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1546) + (_etype1551, _size1548) = iprot.readListBegin() + for _i1552 in range(_size1548): + _elem1553 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1553) iprot.readListEnd() else: iprot.skip(ftype) @@ -31359,8 +31359,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1547 in self.part_vals: - oprot.writeString(iter1547.encode('utf-8') if sys.version_info[0] == 2 else iter1547) + for iter1554 in self.part_vals: + oprot.writeString(iter1554.encode('utf-8') if sys.version_info[0] == 2 else iter1554) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -31686,10 +31686,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1551, _size1548) = iprot.readListBegin() - for _i1552 in range(_size1548): - _elem1553 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1553) + (_etype1558, _size1555) = iprot.readListBegin() + for _i1559 in range(_size1555): + _elem1560 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1560) iprot.readListEnd() else: iprot.skip(ftype) @@ -31720,8 +31720,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1554 in self.part_vals: - oprot.writeString(iter1554.encode('utf-8') if sys.version_info[0] == 2 else iter1554) + for iter1561 in self.part_vals: + oprot.writeString(iter1561.encode('utf-8') if sys.version_info[0] == 2 else iter1561) oprot.writeListEnd() oprot.writeFieldEnd() if self.environment_context is not None: @@ -32272,10 +32272,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1558, _size1555) = iprot.readListBegin() - for _i1559 in range(_size1555): - _elem1560 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1560) + (_etype1565, _size1562) = iprot.readListBegin() + for _i1566 in range(_size1562): + _elem1567 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1567) iprot.readListEnd() else: iprot.skip(ftype) @@ -32305,8 +32305,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1561 in self.part_vals: - oprot.writeString(iter1561.encode('utf-8') if sys.version_info[0] == 2 else iter1561) + for iter1568 in self.part_vals: + oprot.writeString(iter1568.encode('utf-8') if sys.version_info[0] == 2 else iter1568) oprot.writeListEnd() oprot.writeFieldEnd() if self.deleteData is not None: @@ -32465,10 +32465,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1565, _size1562) = iprot.readListBegin() - for _i1566 in range(_size1562): - _elem1567 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1567) + (_etype1572, _size1569) = iprot.readListBegin() + for _i1573 in range(_size1569): + _elem1574 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1574) iprot.readListEnd() else: iprot.skip(ftype) @@ -32504,8 +32504,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1568 in self.part_vals: - oprot.writeString(iter1568.encode('utf-8') if sys.version_info[0] == 2 else iter1568) + for iter1575 in self.part_vals: + oprot.writeString(iter1575.encode('utf-8') if sys.version_info[0] == 2 else iter1575) oprot.writeListEnd() oprot.writeFieldEnd() if self.deleteData is not None: @@ -33193,10 +33193,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1572, _size1569) = iprot.readListBegin() - for _i1573 in range(_size1569): - _elem1574 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1574) + (_etype1579, _size1576) = iprot.readListBegin() + for _i1580 in range(_size1576): + _elem1581 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1581) iprot.readListEnd() else: iprot.skip(ftype) @@ -33221,8 +33221,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1575 in self.part_vals: - oprot.writeString(iter1575.encode('utf-8') if sys.version_info[0] == 2 else iter1575) + for iter1582 in self.part_vals: + oprot.writeString(iter1582.encode('utf-8') if sys.version_info[0] == 2 else iter1582) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -33516,11 +33516,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.partitionSpecs = {} - (_ktype1577, _vtype1578, _size1576) = iprot.readMapBegin() - for _i1580 in range(_size1576): - _key1581 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1582 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionSpecs[_key1581] = _val1582 + (_ktype1584, _vtype1585, _size1583) = iprot.readMapBegin() + for _i1587 in range(_size1583): + _key1588 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1589 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionSpecs[_key1588] = _val1589 iprot.readMapEnd() else: iprot.skip(ftype) @@ -33557,9 +33557,9 @@ def write(self, oprot): if self.partitionSpecs is not None: oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) - for kiter1583, viter1584 in self.partitionSpecs.items(): - oprot.writeString(kiter1583.encode('utf-8') if sys.version_info[0] == 2 else kiter1583) - oprot.writeString(viter1584.encode('utf-8') if sys.version_info[0] == 2 else viter1584) + for kiter1590, viter1591 in self.partitionSpecs.items(): + oprot.writeString(kiter1590.encode('utf-8') if sys.version_info[0] == 2 else kiter1590) + oprot.writeString(viter1591.encode('utf-8') if sys.version_info[0] == 2 else viter1591) oprot.writeMapEnd() oprot.writeFieldEnd() if self.source_db is not None: @@ -33746,11 +33746,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.partitionSpecs = {} - (_ktype1586, _vtype1587, _size1585) = iprot.readMapBegin() - for _i1589 in range(_size1585): - _key1590 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1591 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.partitionSpecs[_key1590] = _val1591 + (_ktype1593, _vtype1594, _size1592) = iprot.readMapBegin() + for _i1596 in range(_size1592): + _key1597 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1598 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partitionSpecs[_key1597] = _val1598 iprot.readMapEnd() else: iprot.skip(ftype) @@ -33787,9 +33787,9 @@ def write(self, oprot): if self.partitionSpecs is not None: oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) - for kiter1592, viter1593 in self.partitionSpecs.items(): - oprot.writeString(kiter1592.encode('utf-8') if sys.version_info[0] == 2 else kiter1592) - oprot.writeString(viter1593.encode('utf-8') if sys.version_info[0] == 2 else viter1593) + for kiter1599, viter1600 in self.partitionSpecs.items(): + oprot.writeString(kiter1599.encode('utf-8') if sys.version_info[0] == 2 else kiter1599) + oprot.writeString(viter1600.encode('utf-8') if sys.version_info[0] == 2 else viter1600) oprot.writeMapEnd() oprot.writeFieldEnd() if self.source_db is not None: @@ -33866,11 +33866,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1597, _size1594) = iprot.readListBegin() - for _i1598 in range(_size1594): - _elem1599 = Partition() - _elem1599.read(iprot) - self.success.append(_elem1599) + (_etype1604, _size1601) = iprot.readListBegin() + for _i1605 in range(_size1601): + _elem1606 = Partition() + _elem1606.read(iprot) + self.success.append(_elem1606) iprot.readListEnd() else: iprot.skip(ftype) @@ -33907,8 +33907,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1600 in self.success: - iter1600.write(oprot) + for iter1607 in self.success: + iter1607.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -33994,10 +33994,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1604, _size1601) = iprot.readListBegin() - for _i1605 in range(_size1601): - _elem1606 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1606) + (_etype1611, _size1608) = iprot.readListBegin() + for _i1612 in range(_size1608): + _elem1613 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1613) iprot.readListEnd() else: iprot.skip(ftype) @@ -34009,10 +34009,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.group_names = [] - (_etype1610, _size1607) = iprot.readListBegin() - for _i1611 in range(_size1607): - _elem1612 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1612) + (_etype1617, _size1614) = iprot.readListBegin() + for _i1618 in range(_size1614): + _elem1619 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1619) iprot.readListEnd() else: iprot.skip(ftype) @@ -34037,8 +34037,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1613 in self.part_vals: - oprot.writeString(iter1613.encode('utf-8') if sys.version_info[0] == 2 else iter1613) + for iter1620 in self.part_vals: + oprot.writeString(iter1620.encode('utf-8') if sys.version_info[0] == 2 else iter1620) oprot.writeListEnd() oprot.writeFieldEnd() if self.user_name is not None: @@ -34048,8 +34048,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1614 in self.group_names: - oprot.writeString(iter1614.encode('utf-8') if sys.version_info[0] == 2 else iter1614) + for iter1621 in self.group_names: + oprot.writeString(iter1621.encode('utf-8') if sys.version_info[0] == 2 else iter1621) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -34450,11 +34450,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1618, _size1615) = iprot.readListBegin() - for _i1619 in range(_size1615): - _elem1620 = Partition() - _elem1620.read(iprot) - self.success.append(_elem1620) + (_etype1625, _size1622) = iprot.readListBegin() + for _i1626 in range(_size1622): + _elem1627 = Partition() + _elem1627.read(iprot) + self.success.append(_elem1627) iprot.readListEnd() else: iprot.skip(ftype) @@ -34481,8 +34481,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1621 in self.success: - iter1621.write(oprot) + for iter1628 in self.success: + iter1628.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -34717,10 +34717,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.group_names = [] - (_etype1625, _size1622) = iprot.readListBegin() - for _i1626 in range(_size1622): - _elem1627 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1627) + (_etype1632, _size1629) = iprot.readListBegin() + for _i1633 in range(_size1629): + _elem1634 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1634) iprot.readListEnd() else: iprot.skip(ftype) @@ -34753,8 +34753,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1628 in self.group_names: - oprot.writeString(iter1628.encode('utf-8') if sys.version_info[0] == 2 else iter1628) + for iter1635 in self.group_names: + oprot.writeString(iter1635.encode('utf-8') if sys.version_info[0] == 2 else iter1635) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -34811,11 +34811,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1632, _size1629) = iprot.readListBegin() - for _i1633 in range(_size1629): - _elem1634 = Partition() - _elem1634.read(iprot) - self.success.append(_elem1634) + (_etype1639, _size1636) = iprot.readListBegin() + for _i1640 in range(_size1636): + _elem1641 = Partition() + _elem1641.read(iprot) + self.success.append(_elem1641) iprot.readListEnd() else: iprot.skip(ftype) @@ -34842,8 +34842,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1635 in self.success: - iter1635.write(oprot) + for iter1642 in self.success: + iter1642.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -34991,11 +34991,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1639, _size1636) = iprot.readListBegin() - for _i1640 in range(_size1636): - _elem1641 = PartitionSpec() - _elem1641.read(iprot) - self.success.append(_elem1641) + (_etype1646, _size1643) = iprot.readListBegin() + for _i1647 in range(_size1643): + _elem1648 = PartitionSpec() + _elem1648.read(iprot) + self.success.append(_elem1648) iprot.readListEnd() else: iprot.skip(ftype) @@ -35022,8 +35022,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1642 in self.success: - iter1642.write(oprot) + for iter1649 in self.success: + iter1649.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -35171,10 +35171,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1646, _size1643) = iprot.readListBegin() - for _i1647 in range(_size1643): - _elem1648 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1648) + (_etype1653, _size1650) = iprot.readListBegin() + for _i1654 in range(_size1650): + _elem1655 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1655) iprot.readListEnd() else: iprot.skip(ftype) @@ -35201,8 +35201,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1649 in self.success: - oprot.writeString(iter1649.encode('utf-8') if sys.version_info[0] == 2 else iter1649) + for iter1656 in self.success: + oprot.writeString(iter1656.encode('utf-8') if sys.version_info[0] == 2 else iter1656) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -35425,10 +35425,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1653, _size1650) = iprot.readListBegin() - for _i1654 in range(_size1650): - _elem1655 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1655) + (_etype1660, _size1657) = iprot.readListBegin() + for _i1661 in range(_size1657): + _elem1662 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1662) iprot.readListEnd() else: iprot.skip(ftype) @@ -35458,8 +35458,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1656 in self.part_vals: - oprot.writeString(iter1656.encode('utf-8') if sys.version_info[0] == 2 else iter1656) + for iter1663 in self.part_vals: + oprot.writeString(iter1663.encode('utf-8') if sys.version_info[0] == 2 else iter1663) oprot.writeListEnd() oprot.writeFieldEnd() if self.max_parts is not None: @@ -35519,11 +35519,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1660, _size1657) = iprot.readListBegin() - for _i1661 in range(_size1657): - _elem1662 = Partition() - _elem1662.read(iprot) - self.success.append(_elem1662) + (_etype1667, _size1664) = iprot.readListBegin() + for _i1668 in range(_size1664): + _elem1669 = Partition() + _elem1669.read(iprot) + self.success.append(_elem1669) iprot.readListEnd() else: iprot.skip(ftype) @@ -35550,8 +35550,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1663 in self.success: - iter1663.write(oprot) + for iter1670 in self.success: + iter1670.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -35629,10 +35629,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1667, _size1664) = iprot.readListBegin() - for _i1668 in range(_size1664): - _elem1669 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1669) + (_etype1674, _size1671) = iprot.readListBegin() + for _i1675 in range(_size1671): + _elem1676 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1676) iprot.readListEnd() else: iprot.skip(ftype) @@ -35649,10 +35649,10 @@ def read(self, iprot): elif fid == 6: if ftype == TType.LIST: self.group_names = [] - (_etype1673, _size1670) = iprot.readListBegin() - for _i1674 in range(_size1670): - _elem1675 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1675) + (_etype1680, _size1677) = iprot.readListBegin() + for _i1681 in range(_size1677): + _elem1682 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1682) iprot.readListEnd() else: iprot.skip(ftype) @@ -35677,8 +35677,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1676 in self.part_vals: - oprot.writeString(iter1676.encode('utf-8') if sys.version_info[0] == 2 else iter1676) + for iter1683 in self.part_vals: + oprot.writeString(iter1683.encode('utf-8') if sys.version_info[0] == 2 else iter1683) oprot.writeListEnd() oprot.writeFieldEnd() if self.max_parts is not None: @@ -35692,8 +35692,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 6) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1677 in self.group_names: - oprot.writeString(iter1677.encode('utf-8') if sys.version_info[0] == 2 else iter1677) + for iter1684 in self.group_names: + oprot.writeString(iter1684.encode('utf-8') if sys.version_info[0] == 2 else iter1684) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -35751,11 +35751,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1681, _size1678) = iprot.readListBegin() - for _i1682 in range(_size1678): - _elem1683 = Partition() - _elem1683.read(iprot) - self.success.append(_elem1683) + (_etype1688, _size1685) = iprot.readListBegin() + for _i1689 in range(_size1685): + _elem1690 = Partition() + _elem1690.read(iprot) + self.success.append(_elem1690) iprot.readListEnd() else: iprot.skip(ftype) @@ -35782,8 +35782,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1684 in self.success: - iter1684.write(oprot) + for iter1691 in self.success: + iter1691.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36006,10 +36006,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1688, _size1685) = iprot.readListBegin() - for _i1689 in range(_size1685): - _elem1690 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1690) + (_etype1695, _size1692) = iprot.readListBegin() + for _i1696 in range(_size1692): + _elem1697 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1697) iprot.readListEnd() else: iprot.skip(ftype) @@ -36039,8 +36039,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1691 in self.part_vals: - oprot.writeString(iter1691.encode('utf-8') if sys.version_info[0] == 2 else iter1691) + for iter1698 in self.part_vals: + oprot.writeString(iter1698.encode('utf-8') if sys.version_info[0] == 2 else iter1698) oprot.writeListEnd() oprot.writeFieldEnd() if self.max_parts is not None: @@ -36100,10 +36100,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1695, _size1692) = iprot.readListBegin() - for _i1696 in range(_size1692): - _elem1697 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1697) + (_etype1702, _size1699) = iprot.readListBegin() + for _i1703 in range(_size1699): + _elem1704 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1704) iprot.readListEnd() else: iprot.skip(ftype) @@ -36130,8 +36130,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1698 in self.success: - oprot.writeString(iter1698.encode('utf-8') if sys.version_info[0] == 2 else iter1698) + for iter1705 in self.success: + oprot.writeString(iter1705.encode('utf-8') if sys.version_info[0] == 2 else iter1705) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36405,10 +36405,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1702, _size1699) = iprot.readListBegin() - for _i1703 in range(_size1699): - _elem1704 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1704) + (_etype1709, _size1706) = iprot.readListBegin() + for _i1710 in range(_size1706): + _elem1711 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1711) iprot.readListEnd() else: iprot.skip(ftype) @@ -36435,8 +36435,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1705 in self.success: - oprot.writeString(iter1705.encode('utf-8') if sys.version_info[0] == 2 else iter1705) + for iter1712 in self.success: + oprot.writeString(iter1712.encode('utf-8') if sys.version_info[0] == 2 else iter1712) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36596,11 +36596,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1709, _size1706) = iprot.readListBegin() - for _i1710 in range(_size1706): - _elem1711 = Partition() - _elem1711.read(iprot) - self.success.append(_elem1711) + (_etype1716, _size1713) = iprot.readListBegin() + for _i1717 in range(_size1713): + _elem1718 = Partition() + _elem1718.read(iprot) + self.success.append(_elem1718) iprot.readListEnd() else: iprot.skip(ftype) @@ -36627,8 +36627,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1712 in self.success: - iter1712.write(oprot) + for iter1719 in self.success: + iter1719.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36753,11 +36753,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1716, _size1713) = iprot.readListBegin() - for _i1717 in range(_size1713): - _elem1718 = Partition() - _elem1718.read(iprot) - self.success.append(_elem1718) + (_etype1723, _size1720) = iprot.readListBegin() + for _i1724 in range(_size1720): + _elem1725 = Partition() + _elem1725.read(iprot) + self.success.append(_elem1725) iprot.readListEnd() else: iprot.skip(ftype) @@ -36784,8 +36784,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1719 in self.success: - iter1719.write(oprot) + for iter1726 in self.success: + iter1726.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -36945,11 +36945,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1723, _size1720) = iprot.readListBegin() - for _i1724 in range(_size1720): - _elem1725 = PartitionSpec() - _elem1725.read(iprot) - self.success.append(_elem1725) + (_etype1730, _size1727) = iprot.readListBegin() + for _i1731 in range(_size1727): + _elem1732 = PartitionSpec() + _elem1732.read(iprot) + self.success.append(_elem1732) iprot.readListEnd() else: iprot.skip(ftype) @@ -36976,8 +36976,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1726 in self.success: - iter1726.write(oprot) + for iter1733 in self.success: + iter1733.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -37518,10 +37518,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.names = [] - (_etype1730, _size1727) = iprot.readListBegin() - for _i1731 in range(_size1727): - _elem1732 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.names.append(_elem1732) + (_etype1737, _size1734) = iprot.readListBegin() + for _i1738 in range(_size1734): + _elem1739 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.names.append(_elem1739) iprot.readListEnd() else: iprot.skip(ftype) @@ -37546,8 +37546,8 @@ def write(self, oprot): if self.names is not None: oprot.writeFieldBegin('names', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.names)) - for iter1733 in self.names: - oprot.writeString(iter1733.encode('utf-8') if sys.version_info[0] == 2 else iter1733) + for iter1740 in self.names: + oprot.writeString(iter1740.encode('utf-8') if sys.version_info[0] == 2 else iter1740) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -37604,11 +37604,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1737, _size1734) = iprot.readListBegin() - for _i1738 in range(_size1734): - _elem1739 = Partition() - _elem1739.read(iprot) - self.success.append(_elem1739) + (_etype1744, _size1741) = iprot.readListBegin() + for _i1745 in range(_size1741): + _elem1746 = Partition() + _elem1746.read(iprot) + self.success.append(_elem1746) iprot.readListEnd() else: iprot.skip(ftype) @@ -37640,8 +37640,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1740 in self.success: - iter1740.write(oprot) + for iter1747 in self.success: + iter1747.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -38337,11 +38337,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.new_parts = [] - (_etype1744, _size1741) = iprot.readListBegin() - for _i1745 in range(_size1741): - _elem1746 = Partition() - _elem1746.read(iprot) - self.new_parts.append(_elem1746) + (_etype1751, _size1748) = iprot.readListBegin() + for _i1752 in range(_size1748): + _elem1753 = Partition() + _elem1753.read(iprot) + self.new_parts.append(_elem1753) iprot.readListEnd() else: iprot.skip(ftype) @@ -38366,8 +38366,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1747 in self.new_parts: - iter1747.write(oprot) + for iter1754 in self.new_parts: + iter1754.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -38508,11 +38508,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.new_parts = [] - (_etype1751, _size1748) = iprot.readListBegin() - for _i1752 in range(_size1748): - _elem1753 = Partition() - _elem1753.read(iprot) - self.new_parts.append(_elem1753) + (_etype1758, _size1755) = iprot.readListBegin() + for _i1759 in range(_size1755): + _elem1760 = Partition() + _elem1760.read(iprot) + self.new_parts.append(_elem1760) iprot.readListEnd() else: iprot.skip(ftype) @@ -38543,8 +38543,8 @@ def write(self, oprot): if self.new_parts is not None: oprot.writeFieldBegin('new_parts', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter1754 in self.new_parts: - iter1754.write(oprot) + for iter1761 in self.new_parts: + iter1761.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.environment_context is not None: @@ -39013,10 +39013,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype1758, _size1755) = iprot.readListBegin() - for _i1759 in range(_size1755): - _elem1760 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1760) + (_etype1765, _size1762) = iprot.readListBegin() + for _i1766 in range(_size1762): + _elem1767 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1767) iprot.readListEnd() else: iprot.skip(ftype) @@ -39047,8 +39047,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1761 in self.part_vals: - oprot.writeString(iter1761.encode('utf-8') if sys.version_info[0] == 2 else iter1761) + for iter1768 in self.part_vals: + oprot.writeString(iter1768.encode('utf-8') if sys.version_info[0] == 2 else iter1768) oprot.writeListEnd() oprot.writeFieldEnd() if self.new_part is not None: @@ -39329,10 +39329,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.part_vals = [] - (_etype1765, _size1762) = iprot.readListBegin() - for _i1766 in range(_size1762): - _elem1767 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals.append(_elem1767) + (_etype1772, _size1769) = iprot.readListBegin() + for _i1773 in range(_size1769): + _elem1774 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals.append(_elem1774) iprot.readListEnd() else: iprot.skip(ftype) @@ -39354,8 +39354,8 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 1) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter1768 in self.part_vals: - oprot.writeString(iter1768.encode('utf-8') if sys.version_info[0] == 2 else iter1768) + for iter1775 in self.part_vals: + oprot.writeString(iter1775.encode('utf-8') if sys.version_info[0] == 2 else iter1775) oprot.writeListEnd() oprot.writeFieldEnd() if self.throw_exception is not None: @@ -39693,10 +39693,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1772, _size1769) = iprot.readListBegin() - for _i1773 in range(_size1769): - _elem1774 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1774) + (_etype1779, _size1776) = iprot.readListBegin() + for _i1780 in range(_size1776): + _elem1781 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1781) iprot.readListEnd() else: iprot.skip(ftype) @@ -39718,8 +39718,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1775 in self.success: - oprot.writeString(iter1775.encode('utf-8') if sys.version_info[0] == 2 else iter1775) + for iter1782 in self.success: + oprot.writeString(iter1782.encode('utf-8') if sys.version_info[0] == 2 else iter1782) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -39836,11 +39836,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.MAP: self.success = {} - (_ktype1777, _vtype1778, _size1776) = iprot.readMapBegin() - for _i1780 in range(_size1776): - _key1781 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1782 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success[_key1781] = _val1782 + (_ktype1784, _vtype1785, _size1783) = iprot.readMapBegin() + for _i1787 in range(_size1783): + _key1788 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1789 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success[_key1788] = _val1789 iprot.readMapEnd() else: iprot.skip(ftype) @@ -39862,9 +39862,9 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.MAP, 0) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) - for kiter1783, viter1784 in self.success.items(): - oprot.writeString(kiter1783.encode('utf-8') if sys.version_info[0] == 2 else kiter1783) - oprot.writeString(viter1784.encode('utf-8') if sys.version_info[0] == 2 else viter1784) + for kiter1790, viter1791 in self.success.items(): + oprot.writeString(kiter1790.encode('utf-8') if sys.version_info[0] == 2 else kiter1790) + oprot.writeString(viter1791.encode('utf-8') if sys.version_info[0] == 2 else viter1791) oprot.writeMapEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -39933,11 +39933,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.MAP: self.part_vals = {} - (_ktype1786, _vtype1787, _size1785) = iprot.readMapBegin() - for _i1789 in range(_size1785): - _key1790 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1791 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals[_key1790] = _val1791 + (_ktype1793, _vtype1794, _size1792) = iprot.readMapBegin() + for _i1796 in range(_size1792): + _key1797 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1798 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals[_key1797] = _val1798 iprot.readMapEnd() else: iprot.skip(ftype) @@ -39967,9 +39967,9 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter1792, viter1793 in self.part_vals.items(): - oprot.writeString(kiter1792.encode('utf-8') if sys.version_info[0] == 2 else kiter1792) - oprot.writeString(viter1793.encode('utf-8') if sys.version_info[0] == 2 else viter1793) + for kiter1799, viter1800 in self.part_vals.items(): + oprot.writeString(kiter1799.encode('utf-8') if sys.version_info[0] == 2 else kiter1799) + oprot.writeString(viter1800.encode('utf-8') if sys.version_info[0] == 2 else viter1800) oprot.writeMapEnd() oprot.writeFieldEnd() if self.eventType is not None: @@ -40163,11 +40163,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.MAP: self.part_vals = {} - (_ktype1795, _vtype1796, _size1794) = iprot.readMapBegin() - for _i1798 in range(_size1794): - _key1799 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1800 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.part_vals[_key1799] = _val1800 + (_ktype1802, _vtype1803, _size1801) = iprot.readMapBegin() + for _i1805 in range(_size1801): + _key1806 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1807 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.part_vals[_key1806] = _val1807 iprot.readMapEnd() else: iprot.skip(ftype) @@ -40197,9 +40197,9 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter1801, viter1802 in self.part_vals.items(): - oprot.writeString(kiter1801.encode('utf-8') if sys.version_info[0] == 2 else kiter1801) - oprot.writeString(viter1802.encode('utf-8') if sys.version_info[0] == 2 else viter1802) + for kiter1808, viter1809 in self.part_vals.items(): + oprot.writeString(kiter1808.encode('utf-8') if sys.version_info[0] == 2 else kiter1808) + oprot.writeString(viter1809.encode('utf-8') if sys.version_info[0] == 2 else viter1809) oprot.writeMapEnd() oprot.writeFieldEnd() if self.eventType is not None: @@ -44241,10 +44241,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1806, _size1803) = iprot.readListBegin() - for _i1807 in range(_size1803): - _elem1808 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1808) + (_etype1813, _size1810) = iprot.readListBegin() + for _i1814 in range(_size1810): + _elem1815 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1815) iprot.readListEnd() else: iprot.skip(ftype) @@ -44266,8 +44266,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1809 in self.success: - oprot.writeString(iter1809.encode('utf-8') if sys.version_info[0] == 2 else iter1809) + for iter1816 in self.success: + oprot.writeString(iter1816.encode('utf-8') if sys.version_info[0] == 2 else iter1816) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -44913,10 +44913,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1813, _size1810) = iprot.readListBegin() - for _i1814 in range(_size1810): - _elem1815 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1815) + (_etype1820, _size1817) = iprot.readListBegin() + for _i1821 in range(_size1817): + _elem1822 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1822) iprot.readListEnd() else: iprot.skip(ftype) @@ -44938,8 +44938,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1816 in self.success: - oprot.writeString(iter1816.encode('utf-8') if sys.version_info[0] == 2 else iter1816) + for iter1823 in self.success: + oprot.writeString(iter1823.encode('utf-8') if sys.version_info[0] == 2 else iter1823) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -45422,11 +45422,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1820, _size1817) = iprot.readListBegin() - for _i1821 in range(_size1817): - _elem1822 = Role() - _elem1822.read(iprot) - self.success.append(_elem1822) + (_etype1827, _size1824) = iprot.readListBegin() + for _i1828 in range(_size1824): + _elem1829 = Role() + _elem1829.read(iprot) + self.success.append(_elem1829) iprot.readListEnd() else: iprot.skip(ftype) @@ -45448,8 +45448,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1823 in self.success: - iter1823.write(oprot) + for iter1830 in self.success: + iter1830.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -45928,10 +45928,10 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.group_names = [] - (_etype1827, _size1824) = iprot.readListBegin() - for _i1828 in range(_size1824): - _elem1829 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1829) + (_etype1834, _size1831) = iprot.readListBegin() + for _i1835 in range(_size1831): + _elem1836 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1836) iprot.readListEnd() else: iprot.skip(ftype) @@ -45956,8 +45956,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1830 in self.group_names: - oprot.writeString(iter1830.encode('utf-8') if sys.version_info[0] == 2 else iter1830) + for iter1837 in self.group_names: + oprot.writeString(iter1837.encode('utf-8') if sys.version_info[0] == 2 else iter1837) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -46171,11 +46171,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1834, _size1831) = iprot.readListBegin() - for _i1835 in range(_size1831): - _elem1836 = HiveObjectPrivilege() - _elem1836.read(iprot) - self.success.append(_elem1836) + (_etype1841, _size1838) = iprot.readListBegin() + for _i1842 in range(_size1838): + _elem1843 = HiveObjectPrivilege() + _elem1843.read(iprot) + self.success.append(_elem1843) iprot.readListEnd() else: iprot.skip(ftype) @@ -46197,8 +46197,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1837 in self.success: - iter1837.write(oprot) + for iter1844 in self.success: + iter1844.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -46829,10 +46829,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.group_names = [] - (_etype1841, _size1838) = iprot.readListBegin() - for _i1842 in range(_size1838): - _elem1843 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.group_names.append(_elem1843) + (_etype1848, _size1845) = iprot.readListBegin() + for _i1849 in range(_size1845): + _elem1850 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.group_names.append(_elem1850) iprot.readListEnd() else: iprot.skip(ftype) @@ -46853,8 +46853,8 @@ def write(self, oprot): if self.group_names is not None: oprot.writeFieldBegin('group_names', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter1844 in self.group_names: - oprot.writeString(iter1844.encode('utf-8') if sys.version_info[0] == 2 else iter1844) + for iter1851 in self.group_names: + oprot.writeString(iter1851.encode('utf-8') if sys.version_info[0] == 2 else iter1851) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -46906,10 +46906,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1848, _size1845) = iprot.readListBegin() - for _i1849 in range(_size1845): - _elem1850 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1850) + (_etype1855, _size1852) = iprot.readListBegin() + for _i1856 in range(_size1852): + _elem1857 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1857) iprot.readListEnd() else: iprot.skip(ftype) @@ -46931,8 +46931,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1851 in self.success: - oprot.writeString(iter1851.encode('utf-8') if sys.version_info[0] == 2 else iter1851) + for iter1858 in self.success: + oprot.writeString(iter1858.encode('utf-8') if sys.version_info[0] == 2 else iter1858) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -47815,10 +47815,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1855, _size1852) = iprot.readListBegin() - for _i1856 in range(_size1852): - _elem1857 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1857) + (_etype1862, _size1859) = iprot.readListBegin() + for _i1863 in range(_size1859): + _elem1864 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1864) iprot.readListEnd() else: iprot.skip(ftype) @@ -47835,8 +47835,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1858 in self.success: - oprot.writeString(iter1858.encode('utf-8') if sys.version_info[0] == 2 else iter1858) + for iter1865 in self.success: + oprot.writeString(iter1865.encode('utf-8') if sys.version_info[0] == 2 else iter1865) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -48333,10 +48333,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1862, _size1859) = iprot.readListBegin() - for _i1863 in range(_size1859): - _elem1864 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1864) + (_etype1869, _size1866) = iprot.readListBegin() + for _i1870 in range(_size1866): + _elem1871 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1871) iprot.readListEnd() else: iprot.skip(ftype) @@ -48353,8 +48353,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1865 in self.success: - oprot.writeString(iter1865.encode('utf-8') if sys.version_info[0] == 2 else iter1865) + for iter1872 in self.success: + oprot.writeString(iter1872.encode('utf-8') if sys.version_info[0] == 2 else iter1872) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -49521,11 +49521,11 @@ def read(self, iprot): elif fid == 2: if ftype == TType.MAP: self.writeIds = {} - (_ktype1867, _vtype1868, _size1866) = iprot.readMapBegin() - for _i1870 in range(_size1866): - _key1871 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - _val1872 = iprot.readI64() - self.writeIds[_key1871] = _val1872 + (_ktype1874, _vtype1875, _size1873) = iprot.readMapBegin() + for _i1877 in range(_size1873): + _key1878 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + _val1879 = iprot.readI64() + self.writeIds[_key1878] = _val1879 iprot.readMapEnd() else: iprot.skip(ftype) @@ -49546,9 +49546,9 @@ def write(self, oprot): if self.writeIds is not None: oprot.writeFieldBegin('writeIds', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.I64, len(self.writeIds)) - for kiter1873, viter1874 in self.writeIds.items(): - oprot.writeString(kiter1873.encode('utf-8') if sys.version_info[0] == 2 else kiter1873) - oprot.writeI64(viter1874) + for kiter1880, viter1881 in self.writeIds.items(): + oprot.writeString(kiter1880.encode('utf-8') if sys.version_info[0] == 2 else kiter1880) + oprot.writeI64(viter1881) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -52161,10 +52161,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1878, _size1875) = iprot.readListBegin() - for _i1879 in range(_size1875): - _elem1880 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1880) + (_etype1885, _size1882) = iprot.readListBegin() + for _i1886 in range(_size1882): + _elem1887 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1887) iprot.readListEnd() else: iprot.skip(ftype) @@ -52181,8 +52181,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1881 in self.success: - oprot.writeString(iter1881.encode('utf-8') if sys.version_info[0] == 2 else iter1881) + for iter1888 in self.success: + oprot.writeString(iter1888.encode('utf-8') if sys.version_info[0] == 2 else iter1888) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -58738,11 +58738,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1885, _size1882) = iprot.readListBegin() - for _i1886 in range(_size1882): - _elem1887 = SchemaVersion() - _elem1887.read(iprot) - self.success.append(_elem1887) + (_etype1892, _size1889) = iprot.readListBegin() + for _i1893 in range(_size1889): + _elem1894 = SchemaVersion() + _elem1894.read(iprot) + self.success.append(_elem1894) iprot.readListEnd() else: iprot.skip(ftype) @@ -58769,8 +58769,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1888 in self.success: - iter1888.write(oprot) + for iter1895 in self.success: + iter1895.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -60159,11 +60159,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1892, _size1889) = iprot.readListBegin() - for _i1893 in range(_size1889): - _elem1894 = RuntimeStat() - _elem1894.read(iprot) - self.success.append(_elem1894) + (_etype1899, _size1896) = iprot.readListBegin() + for _i1900 in range(_size1896): + _elem1901 = RuntimeStat() + _elem1901.read(iprot) + self.success.append(_elem1901) iprot.readListEnd() else: iprot.skip(ftype) @@ -60185,8 +60185,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1895 in self.success: - iter1895.write(oprot) + for iter1902 in self.success: + iter1902.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -61823,10 +61823,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1899, _size1896) = iprot.readListBegin() - for _i1900 in range(_size1896): - _elem1901 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1901) + (_etype1906, _size1903) = iprot.readListBegin() + for _i1907 in range(_size1903): + _elem1908 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1908) iprot.readListEnd() else: iprot.skip(ftype) @@ -61848,8 +61848,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1902 in self.success: - oprot.writeString(iter1902.encode('utf-8') if sys.version_info[0] == 2 else iter1902) + for iter1909 in self.success: + oprot.writeString(iter1909.encode('utf-8') if sys.version_info[0] == 2 else iter1909) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -62241,10 +62241,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1906, _size1903) = iprot.readListBegin() - for _i1907 in range(_size1903): - _elem1908 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() - self.success.append(_elem1908) + (_etype1913, _size1910) = iprot.readListBegin() + for _i1914 in range(_size1910): + _elem1915 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.success.append(_elem1915) iprot.readListEnd() else: iprot.skip(ftype) @@ -62266,8 +62266,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter1909 in self.success: - oprot.writeString(iter1909.encode('utf-8') if sys.version_info[0] == 2 else iter1909) + for iter1916 in self.success: + oprot.writeString(iter1916.encode('utf-8') if sys.version_info[0] == 2 else iter1916) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -62510,11 +62510,11 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype1913, _size1910) = iprot.readListBegin() - for _i1914 in range(_size1910): - _elem1915 = WriteEventInfo() - _elem1915.read(iprot) - self.success.append(_elem1915) + (_etype1920, _size1917) = iprot.readListBegin() + for _i1921 in range(_size1917): + _elem1922 = WriteEventInfo() + _elem1922.read(iprot) + self.success.append(_elem1922) iprot.readListEnd() else: iprot.skip(ftype) @@ -62536,8 +62536,8 @@ def write(self, oprot): if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter1916 in self.success: - iter1916.write(oprot) + for iter1923 in self.success: + iter1923.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py index 7b8052771aad..c7fe5de1aa0d 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -28595,11 +28595,12 @@ class GetPartitionsPsWithAuthRequest(object): - skipColumnSchemaForPartition - includeParamKeyPattern - excludeParamKeyPattern + - partNames """ - def __init__(self, catName=None, dbName=None, tblName=None, partVals=None, maxParts=-1, userName=None, groupNames=None, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): + def __init__(self, catName=None, dbName=None, tblName=None, partVals=None, maxParts=-1, userName=None, groupNames=None, validWriteIdList=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None, partNames=None,): self.catName = catName self.dbName = dbName self.tblName = tblName @@ -28612,6 +28613,7 @@ def __init__(self, catName=None, dbName=None, tblName=None, partVals=None, maxPa self.skipColumnSchemaForPartition = skipColumnSchemaForPartition self.includeParamKeyPattern = includeParamKeyPattern self.excludeParamKeyPattern = excludeParamKeyPattern + self.partNames = partNames def read(self, iprot): if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None: @@ -28692,6 +28694,16 @@ def read(self, iprot): self.excludeParamKeyPattern = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() else: iprot.skip(ftype) + elif fid == 13: + if ftype == TType.LIST: + self.partNames = [] + (_etype1323, _size1320) = iprot.readListBegin() + for _i1324 in range(_size1320): + _elem1325 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString() + self.partNames.append(_elem1325) + iprot.readListEnd() + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -28717,8 +28729,8 @@ def write(self, oprot): if self.partVals is not None: oprot.writeFieldBegin('partVals', TType.LIST, 4) oprot.writeListBegin(TType.STRING, len(self.partVals)) - for iter1320 in self.partVals: - oprot.writeString(iter1320.encode('utf-8') if sys.version_info[0] == 2 else iter1320) + for iter1326 in self.partVals: + oprot.writeString(iter1326.encode('utf-8') if sys.version_info[0] == 2 else iter1326) oprot.writeListEnd() oprot.writeFieldEnd() if self.maxParts is not None: @@ -28732,8 +28744,8 @@ def write(self, oprot): if self.groupNames is not None: oprot.writeFieldBegin('groupNames', TType.LIST, 7) oprot.writeListBegin(TType.STRING, len(self.groupNames)) - for iter1321 in self.groupNames: - oprot.writeString(iter1321.encode('utf-8') if sys.version_info[0] == 2 else iter1321) + for iter1327 in self.groupNames: + oprot.writeString(iter1327.encode('utf-8') if sys.version_info[0] == 2 else iter1327) oprot.writeListEnd() oprot.writeFieldEnd() if self.validWriteIdList is not None: @@ -28756,6 +28768,13 @@ def write(self, oprot): oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 12) oprot.writeString(self.excludeParamKeyPattern.encode('utf-8') if sys.version_info[0] == 2 else self.excludeParamKeyPattern) oprot.writeFieldEnd() + if self.partNames is not None: + oprot.writeFieldBegin('partNames', TType.LIST, 13) + oprot.writeListBegin(TType.STRING, len(self.partNames)) + for iter1328 in self.partNames: + oprot.writeString(iter1328.encode('utf-8') if sys.version_info[0] == 2 else iter1328) + oprot.writeListEnd() + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -28801,11 +28820,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.partitions = [] - (_etype1325, _size1322) = iprot.readListBegin() - for _i1326 in range(_size1322): - _elem1327 = Partition() - _elem1327.read(iprot) - self.partitions.append(_elem1327) + (_etype1332, _size1329) = iprot.readListBegin() + for _i1333 in range(_size1329): + _elem1334 = Partition() + _elem1334.read(iprot) + self.partitions.append(_elem1334) iprot.readListEnd() else: iprot.skip(ftype) @@ -28822,8 +28841,8 @@ def write(self, oprot): if self.partitions is not None: oprot.writeFieldBegin('partitions', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.partitions)) - for iter1328 in self.partitions: - iter1328.write(oprot) + for iter1335 in self.partitions: + iter1335.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -28987,11 +29006,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.replicationMetricList = [] - (_etype1332, _size1329) = iprot.readListBegin() - for _i1333 in range(_size1329): - _elem1334 = ReplicationMetrics() - _elem1334.read(iprot) - self.replicationMetricList.append(_elem1334) + (_etype1339, _size1336) = iprot.readListBegin() + for _i1340 in range(_size1336): + _elem1341 = ReplicationMetrics() + _elem1341.read(iprot) + self.replicationMetricList.append(_elem1341) iprot.readListEnd() else: iprot.skip(ftype) @@ -29008,8 +29027,8 @@ def write(self, oprot): if self.replicationMetricList is not None: oprot.writeFieldBegin('replicationMetricList', TType.LIST, 1) oprot.writeListBegin(TType.STRUCT, len(self.replicationMetricList)) - for iter1335 in self.replicationMetricList: - iter1335.write(oprot) + for iter1342 in self.replicationMetricList: + iter1342.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -29134,10 +29153,10 @@ def read(self, iprot): if fid == 1: if ftype == TType.LIST: self.excludeTxnTypes = [] - (_etype1339, _size1336) = iprot.readListBegin() - for _i1340 in range(_size1336): - _elem1341 = iprot.readI32() - self.excludeTxnTypes.append(_elem1341) + (_etype1346, _size1343) = iprot.readListBegin() + for _i1347 in range(_size1343): + _elem1348 = iprot.readI32() + self.excludeTxnTypes.append(_elem1348) iprot.readListEnd() else: iprot.skip(ftype) @@ -29154,8 +29173,8 @@ def write(self, oprot): if self.excludeTxnTypes is not None: oprot.writeFieldBegin('excludeTxnTypes', TType.LIST, 1) oprot.writeListBegin(TType.I32, len(self.excludeTxnTypes)) - for iter1342 in self.excludeTxnTypes: - oprot.writeI32(iter1342) + for iter1349 in self.excludeTxnTypes: + oprot.writeI32(iter1349) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -33513,6 +33532,7 @@ def __ne__(self, other): (10, TType.BOOL, 'skipColumnSchemaForPartition', None, None, ), # 10 (11, TType.STRING, 'includeParamKeyPattern', 'UTF8', None, ), # 11 (12, TType.STRING, 'excludeParamKeyPattern', 'UTF8', None, ), # 12 + (13, TType.LIST, 'partNames', (TType.STRING, 'UTF8', False), None, ), # 13 ) all_structs.append(GetPartitionsPsWithAuthResponse) GetPartitionsPsWithAuthResponse.thrift_spec = ( diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb index ab4c608d8e8e..634f58351bb5 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -7801,6 +7801,7 @@ class GetPartitionsPsWithAuthRequest SKIPCOLUMNSCHEMAFORPARTITION = 10 INCLUDEPARAMKEYPATTERN = 11 EXCLUDEPARAMKEYPATTERN = 12 + PARTNAMES = 13 FIELDS = { CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, @@ -7814,7 +7815,8 @@ class GetPartitionsPsWithAuthRequest ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true}, SKIPCOLUMNSCHEMAFORPARTITION => {:type => ::Thrift::Types::BOOL, :name => 'skipColumnSchemaForPartition', :optional => true}, INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern', :optional => true}, - EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true} + EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern', :optional => true}, + PARTNAMES => {:type => ::Thrift::Types::LIST, :name => 'partNames', :element => {:type => ::Thrift::Types::STRING}, :optional => true} } def struct_fields; FIELDS; end diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift index e5869736403f..422f23fdfffc 100644 --- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift +++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift @@ -2360,7 +2360,8 @@ struct GetPartitionsPsWithAuthRequest { 9: optional i64 id=-1 // table id 10: optional bool skipColumnSchemaForPartition, 11: optional string includeParamKeyPattern, - 12: optional string excludeParamKeyPattern + 12: optional string excludeParamKeyPattern, + 13: optional list partNames; } struct GetPartitionsPsWithAuthResponse { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index a9dbb03d989e..3ed850efbabe 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -6695,6 +6695,7 @@ public GetPartitionsPsWithAuthResponse get_partitions_ps_with_auth_req(GetPartit .skipColumnSchemaForPartition(req.isSkipColumnSchemaForPartition()) .includeParamKeyPattern(req.getIncludeParamKeyPattern()) .excludeParamKeyPattern(req.getExcludeParamKeyPattern()) + .partNames(req.getPartNames()) .build()); GetPartitionsPsWithAuthResponse res = new GetPartitionsPsWithAuthResponse(); res.setPartitions(partitions); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 436ebd932acc..0ae023bdd7e3 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -3890,22 +3890,27 @@ public List listPartitionsPsWithAuth(String catName, String db_name, String userName = args.getUserName(); List groupNames = args.getGroupNames(); List part_vals = args.getPart_vals(); + List partNames = args.getPartNames(); + boolean isAcidTable = TxnUtils.isAcidTable(mtbl.getParameters()); boolean getauth = null != userName && null != groupNames && "TRUE".equalsIgnoreCase( mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE")); - - if (canTryDirectSQL(part_vals)) { + // When partNames is given, sending to JDO directly. + if (canTryDirectSQL(part_vals) && partNames == null) { LOG.info( "Redirecting to directSQL enabled API: db: {} tbl: {} partVals: {}", db_name, tbl_name, part_vals); partitions = getPartitions(catName, db_name, tbl_name, args); } else { - Collection parts = getPartitionPsQueryResults(catName, db_name, tbl_name, - part_vals, max_parts, null); - boolean isAcidTable = TxnUtils.isAcidTable(mtbl.getParameters()); - for (Object o : parts) { - Partition part = convertToPart(catName, db_name, tbl_name, (MPartition) o, isAcidTable, args); - partitions.add(part); + if (partNames != null) { + partitions.addAll(getPartitionsViaOrmFilter(catName, db_name, tbl_name, isAcidTable, args)); + } else { + Collection parts = getPartitionPsQueryResults(catName, db_name, tbl_name, + part_vals, max_parts, null); + for (Object o : parts) { + Partition part = convertToPart(catName, db_name, tbl_name, (MPartition) o, isAcidTable, args); + partitions.add(part); + } } } if (getauth) { From 7c25c2e490cd02af57c5142b94c4fdb120eb2f59 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Sat, 23 Dec 2023 12:52:48 +0200 Subject: [PATCH 112/179] HIVE-27967: Iceberg: Fix dynamic runtime filtering (Denys Kuzmenko, reviewed by Attila Turoczy, Butao Zhang) Closes #4962 --- .../mr/hive/HiveIcebergFilterFactory.java | 3 - .../mr/hive/TestHiveIcebergFilterFactory.java | 84 ------------------- 2 files changed, 87 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java index d443672b0e8d..4db6627cbc9c 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergFilterFactory.java @@ -110,9 +110,6 @@ private static Expression translateLeaf(PredicateLeaf leaf) { return in(column, leafToLiteralList(leaf)); case BETWEEN: List icebergLiterals = leafToLiteralList(leaf); - if (icebergLiterals.size() < 2) { - throw new UnsupportedOperationException("Missing leaf literals: " + leaf); - } if (icebergLiterals.size() == 2) { return and(greaterThanOrEqual(column, icebergLiterals.get(0)), lessThanOrEqual(column, icebergLiterals.get(1))); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java index 1614d937c37f..3044f0467af3 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java @@ -24,9 +24,6 @@ import java.sql.Timestamp; import java.time.LocalDate; import java.time.ZoneOffset; -import java.util.Collections; -import java.util.List; -import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree; import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; @@ -39,7 +36,6 @@ import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -138,21 +134,6 @@ public void testBetweenOperand() { assertEquals(actual.right().op(), expected.right().op()); } - @Test - public void testUnsupportedBetweenOperandEmptyLeaves() { - SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(); - final SearchArgument arg = - new MockSearchArgument( - builder - .startAnd() - .between("salary", PredicateLeaf.Type.LONG, 9000L, 15000L) - .end() - .build()); - Assertions.assertThatThrownBy(() -> HiveIcebergFilterFactory.generateFilterExpression(arg)) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("Missing leaf literals: Leaf[empty]"); - } - @Test public void testIsNullOperand() { SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(); @@ -278,69 +259,4 @@ private void assertPredicatesMatch(UnboundPredicate expected, UnboundPredicate a assertEquals(expected.literal(), actual.literal()); assertEquals(expected.ref().name(), actual.ref().name()); } - - private static class MockSearchArgument implements SearchArgument { - - private final SearchArgument delegate; - - MockSearchArgument(SearchArgument original) { - delegate = original; - } - - @Override - public ExpressionTree getExpression() { - return delegate.getExpression(); - } - - @Override - public ExpressionTree getCompactExpression() { - return null; - } - - @Override - public TruthValue evaluate(TruthValue[] leaves) { - return delegate.evaluate(leaves); - } - - @Override - public List getLeaves() { - return Collections.singletonList( - new PredicateLeaf() { - @Override - public Operator getOperator() { - return Operator.BETWEEN; - } - - @Override - public Type getType() { - return Type.LONG; - } - - @Override - public String getColumnName() { - return "salary"; - } - - @Override - public Object getLiteral() { - return null; - } - - @Override - public List getLiteralList() { - return Collections.emptyList(); - } - - @Override - public int getId() { - return 0; - } - - @Override - public String toString() { - return "Leaf[empty]"; - } - }); - } - } } From d203bf9e03e51b01b75a0bfaa27e8749271fcafa Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sun, 24 Dec 2023 21:20:58 +0800 Subject: [PATCH 113/179] HIVE-27961: Beeline will print duplicate stats info when hive.tez.exec.print.summary is true (#4960)(Butao Zhang, reviewed by Attila Turoczy, Sourabh Badhya) --- .../org/apache/hadoop/hive/ql/stats/BasicStatsTask.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java index ed968edce0cf..96f43c426c81 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java @@ -311,8 +311,9 @@ private int aggregateStats(Hive db, Table tbl) { if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) { console.printInfo("Table " + tableFullName + " stats: [" + toString(p.getPartParameters()) + ']'); + } else { + LOG.info("Table " + tableFullName + " stats: [" + toString(p.getPartParameters()) + ']'); } - LOG.info("Table " + tableFullName + " stats: [" + toString(p.getPartParameters()) + ']'); // The table object is assigned to the latest table object. // So that it can be used by ColStatsProcessor. @@ -377,8 +378,9 @@ public Void call() throws Exception { updates.add((Partition) res); if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) { console.printInfo("Partition " + basicStatsProcessor.partish.getPartition().getSpec() + " stats: [" + toString(basicStatsProcessor.partish.getPartParameters()) + ']'); + } else { + LOG.info("Partition " + basicStatsProcessor.partish.getPartition().getSpec() + " stats: [" + toString(basicStatsProcessor.partish.getPartParameters()) + ']'); } - LOG.info("Partition " + basicStatsProcessor.partish.getPartition().getSpec() + " stats: [" + toString(basicStatsProcessor.partish.getPartParameters()) + ']'); } if (!updates.isEmpty()) { From 81b394104b08efdc42d782d444462f4fdc12a8fd Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sun, 24 Dec 2023 21:23:03 +0800 Subject: [PATCH 114/179] HIVE-23558: Remove compute_stats UDAF (#4928)(Butao Zhang, reviewed by Ayush Saxena) --- data/files/datasets/temp_udf/load.hive.sql | 1 + .../hadoop/hive/ql/exec/FunctionRegistry.java | 1 - .../clientnegative/compute_stats_long.q | 7 - .../test/queries/clientpositive/char_udf1.q | 1 + .../clientpositive/compute_stats_binary.q | 9 - .../clientpositive/compute_stats_boolean.q | 9 - .../clientpositive/compute_stats_date.q | 28 --- .../clientpositive/compute_stats_decimal.q | 11 -- .../clientpositive/compute_stats_double.q | 9 - .../compute_stats_empty_table.q | 12 -- .../clientpositive/compute_stats_long.q | 9 - .../clientpositive/compute_stats_string.q | 9 - .../reduceSinkDeDuplication_pRS_key_empty.q | 1 + .../queries/clientpositive/varchar_udf1.q | 1 + .../test/queries/clientpositive/vector_udf1.q | 1 + .../clientnegative/compute_stats_long.q.out | 57 ------ .../llap/compute_stats_binary.q.out | 34 ---- .../llap/compute_stats_boolean.q.out | 34 ---- .../llap/compute_stats_date.q.out | 176 ------------------ .../llap/compute_stats_decimal.q.out | Bin 1512 -> 0 bytes .../llap/compute_stats_double.q.out | Bin 1443 -> 0 bytes .../llap/compute_stats_empty_table.q.out | 62 ------ .../llap/compute_stats_long.q.out | Bin 1376 -> 0 bytes .../llap/compute_stats_string.q.out | Bin 1448 -> 0 bytes .../clientpositive/llap/show_functions.q.out | 3 - 25 files changed, 5 insertions(+), 470 deletions(-) create mode 100644 data/files/datasets/temp_udf/load.hive.sql delete mode 100644 ql/src/test/queries/clientnegative/compute_stats_long.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_binary.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_boolean.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_date.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_decimal.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_double.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_empty_table.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_long.q delete mode 100644 ql/src/test/queries/clientpositive/compute_stats_string.q delete mode 100644 ql/src/test/results/clientnegative/compute_stats_long.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_binary.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_boolean.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_date.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_decimal.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_double.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_empty_table.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_long.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/compute_stats_string.q.out diff --git a/data/files/datasets/temp_udf/load.hive.sql b/data/files/datasets/temp_udf/load.hive.sql new file mode 100644 index 000000000000..b8a21d37a23a --- /dev/null +++ b/data/files/datasets/temp_udf/load.hive.sql @@ -0,0 +1 @@ +CREATE FUNCTION compute_stats AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFComputeStats'; \ No newline at end of file diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java index c0b9519944dd..6c1c5fb0113b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java @@ -563,7 +563,6 @@ public final class FunctionRegistry { system.registerGenericUDAF("ngrams", new GenericUDAFnGrams()); system.registerGenericUDAF("context_ngrams", new GenericUDAFContextNGrams()); - system.registerGenericUDAF("compute_stats", new GenericUDAFComputeStats()); system.registerGenericUDF("ndv_compute_bit_vector", GenericUDFNDVComputeBitVector.class); system.registerGenericUDAF("compute_bit_vector_hll", new GenericUDAFComputeBitVectorHLL()); system.registerGenericUDAF("compute_bit_vector_fm", new GenericUDAFComputeBitVectorFMSketch()); diff --git a/ql/src/test/queries/clientnegative/compute_stats_long.q b/ql/src/test/queries/clientnegative/compute_stats_long.q deleted file mode 100644 index bf87393abd0f..000000000000 --- a/ql/src/test/queries/clientnegative/compute_stats_long.q +++ /dev/null @@ -1,7 +0,0 @@ -create table tab_int(a int); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; - --- this should raise an error since the number of bit vectors has a hard limit at 1024 -select compute_stats(a, 'fm', 10000) from tab_int; diff --git a/ql/src/test/queries/clientpositive/char_udf1.q b/ql/src/test/queries/clientpositive/char_udf1.q index 204404361502..4a6ac4020c36 100644 --- a/ql/src/test/queries/clientpositive/char_udf1.q +++ b/ql/src/test/queries/clientpositive/char_udf1.q @@ -1,4 +1,5 @@ --! qt:dataset:src +--! qt:dataset:temp_udf drop table char_udf_1; create table char_udf_1 (c1 string, c2 string, c3 char(10), c4 char(20)); diff --git a/ql/src/test/queries/clientpositive/compute_stats_binary.q b/ql/src/test/queries/clientpositive/compute_stats_binary.q deleted file mode 100644 index fd15634f202f..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_binary.q +++ /dev/null @@ -1,9 +0,0 @@ -create table tab_binary(a binary); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/binary.txt" INTO TABLE tab_binary; - -select count(*) from tab_binary; - --- compute statistical summary of data -select compute_stats(a, 16) from tab_binary; diff --git a/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/ql/src/test/queries/clientpositive/compute_stats_boolean.q deleted file mode 100644 index cddb53f8f64f..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_boolean.q +++ /dev/null @@ -1,9 +0,0 @@ -create table tab_bool(a boolean); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/bool.txt" INTO TABLE tab_bool; - -select count(*) from tab_bool; - --- compute statistical summary of data -select compute_stats(a, 16) from tab_bool; diff --git a/ql/src/test/queries/clientpositive/compute_stats_date.q b/ql/src/test/queries/clientpositive/compute_stats_date.q deleted file mode 100644 index bf478526ba8a..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_date.q +++ /dev/null @@ -1,28 +0,0 @@ - -create table tab_date ( - origin_city_name string, - dest_city_name string, - fl_date date, - arr_delay float, - fl_num int -); - --- insert some data -load data local inpath '../../data/files/flights_join.txt' overwrite into table tab_date; - -select count(*) from tab_date; - --- compute statistical summary of data -select compute_stats(fl_date, 'hll') from tab_date; - -explain -analyze table tab_date compute statistics for columns fl_date; - -analyze table tab_date compute statistics for columns fl_date; - -describe formatted tab_date fl_date; - --- Update stats manually. Try both yyyy-mm-dd and integer value for high/low value -alter table tab_date update statistics for column fl_date set ('numDVs'='19', 'highValue'='2015-01-01', 'lowValue'='0'); - -describe formatted tab_date fl_date; diff --git a/ql/src/test/queries/clientpositive/compute_stats_decimal.q b/ql/src/test/queries/clientpositive/compute_stats_decimal.q deleted file mode 100644 index 2beafaf219a1..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_decimal.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.stats.autogather=true; - -create table tab_decimal(a decimal(35,3)); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/decimal.txt" INTO TABLE tab_decimal; - -select count(*) from tab_decimal; - --- compute statistical summary of data -select compute_stats(a, 'fm', 18) from tab_decimal; diff --git a/ql/src/test/queries/clientpositive/compute_stats_double.q b/ql/src/test/queries/clientpositive/compute_stats_double.q deleted file mode 100644 index 6bae0643a897..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_double.q +++ /dev/null @@ -1,9 +0,0 @@ -create table tab_double(a double); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/double.txt" INTO TABLE tab_double; - -select count(*) from tab_double; - --- compute statistical summary of data -select compute_stats(a, 'fm', 16) from tab_double; diff --git a/ql/src/test/queries/clientpositive/compute_stats_empty_table.q b/ql/src/test/queries/clientpositive/compute_stats_empty_table.q deleted file mode 100644 index d4ed93fa6398..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_empty_table.q +++ /dev/null @@ -1,12 +0,0 @@ -create table tab_empty(a boolean, b int, c double, d string, e binary); - -select count(*) from tab_empty; - --- compute statistical summary of data -select compute_stats(a, 16) from tab_empty; -select compute_stats(b, 16) from tab_empty; -select compute_stats(c, 16) from tab_empty; -select compute_stats(d, 16) from tab_empty; -select compute_stats(e, 16) from tab_empty; - - diff --git a/ql/src/test/queries/clientpositive/compute_stats_long.q b/ql/src/test/queries/clientpositive/compute_stats_long.q deleted file mode 100644 index 48f4ebb9791d..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_long.q +++ /dev/null @@ -1,9 +0,0 @@ -create table tab_int(a int); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; - -select count(*) from tab_int; - --- compute statistical summary of data -select compute_stats(a, 'fm', 16) from tab_int; diff --git a/ql/src/test/queries/clientpositive/compute_stats_string.q b/ql/src/test/queries/clientpositive/compute_stats_string.q deleted file mode 100644 index 79a531e8ec4d..000000000000 --- a/ql/src/test/queries/clientpositive/compute_stats_string.q +++ /dev/null @@ -1,9 +0,0 @@ -create table tab_string(a string); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/string.txt" INTO TABLE tab_string; - -select count(*) from tab_string; - --- compute statistical summary of data -select compute_stats(a, 'fm', 16) from tab_string; diff --git a/ql/src/test/queries/clientpositive/reduceSinkDeDuplication_pRS_key_empty.q b/ql/src/test/queries/clientpositive/reduceSinkDeDuplication_pRS_key_empty.q index f92229560ae2..8af3ff493857 100644 --- a/ql/src/test/queries/clientpositive/reduceSinkDeDuplication_pRS_key_empty.q +++ b/ql/src/test/queries/clientpositive/reduceSinkDeDuplication_pRS_key_empty.q @@ -1,4 +1,5 @@ --! qt:dataset:src +--! qt:dataset:temp_udf set hive.mapred.mode=nonstrict; set hive.cbo.enable=false; diff --git a/ql/src/test/queries/clientpositive/varchar_udf1.q b/ql/src/test/queries/clientpositive/varchar_udf1.q index 7aee1ce2251e..80e184a8c199 100644 --- a/ql/src/test/queries/clientpositive/varchar_udf1.q +++ b/ql/src/test/queries/clientpositive/varchar_udf1.q @@ -1,4 +1,5 @@ --! qt:dataset:src +--! qt:dataset:temp_udf drop table varchar_udf_1; create table varchar_udf_1 (c1 string, c2 string, c3 varchar(10), c4 varchar(20)); diff --git a/ql/src/test/queries/clientpositive/vector_udf1.q b/ql/src/test/queries/clientpositive/vector_udf1.q index a258025dc10a..5a642dafb281 100644 --- a/ql/src/test/queries/clientpositive/vector_udf1.q +++ b/ql/src/test/queries/clientpositive/vector_udf1.q @@ -1,4 +1,5 @@ --! qt:dataset:src +--! qt:dataset:temp_udf SET hive.vectorized.execution.enabled=true; set hive.fetch.task.conversion=none; diff --git a/ql/src/test/results/clientnegative/compute_stats_long.q.out b/ql/src/test/results/clientnegative/compute_stats_long.q.out deleted file mode 100644 index 990a94179076..000000000000 --- a/ql/src/test/results/clientnegative/compute_stats_long.q.out +++ /dev/null @@ -1,57 +0,0 @@ -PREHOOK: query: create table tab_int(a int) -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@tab_int -POSTHOOK: query: create table tab_int(a int) -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@tab_int -PREHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@tab_int -POSTHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@tab_int -PREHOOK: query: select compute_stats(a, 'fm', 10000) from tab_int -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_int -#### A masked pattern was here #### -Status: Failed -Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: The maximum allowed value for number of bit vectors is 1024, but was passed 10000 bit vectors -#### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: The maximum allowed value for number of bit vectors is 1024, but was passed 10000 bit vectors -#### A masked pattern was here #### -]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE] -[Masked Vertex killed due to OTHER_VERTEX_FAILURE] -DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: The maximum allowed value for number of bit vectors is 1024, but was passed 10000 bit vectors -#### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row -#### A masked pattern was here #### -Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: The maximum allowed value for number of bit vectors is 1024, but was passed 10000 bit vectors -#### A masked pattern was here #### -]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE][Masked Vertex killed due to OTHER_VERTEX_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_binary.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_binary.q.out deleted file mode 100644 index fc90c894cd95..000000000000 --- a/ql/src/test/results/clientpositive/llap/compute_stats_binary.q.out +++ /dev/null @@ -1,34 +0,0 @@ -PREHOOK: query: create table tab_binary(a binary) -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@tab_binary -POSTHOOK: query: create table tab_binary(a binary) -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@tab_binary -PREHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/binary.txt" INTO TABLE tab_binary -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@tab_binary -POSTHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/binary.txt" INTO TABLE tab_binary -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@tab_binary -PREHOOK: query: select count(*) from tab_binary -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_binary -#### A masked pattern was here #### -POSTHOOK: query: select count(*) from tab_binary -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tab_binary -#### A masked pattern was here #### -10 -PREHOOK: query: select compute_stats(a, 16) from tab_binary -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_binary -#### A masked pattern was here #### -POSTHOOK: query: select compute_stats(a, 16) from tab_binary -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tab_binary -#### A masked pattern was here #### -{"columntype":"Binary","maxlength":58,"avglength":32.5,"countnulls":0} diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_boolean.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_boolean.q.out deleted file mode 100644 index be90ee222739..000000000000 --- a/ql/src/test/results/clientpositive/llap/compute_stats_boolean.q.out +++ /dev/null @@ -1,34 +0,0 @@ -PREHOOK: query: create table tab_bool(a boolean) -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@tab_bool -POSTHOOK: query: create table tab_bool(a boolean) -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@tab_bool -PREHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/bool.txt" INTO TABLE tab_bool -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@tab_bool -POSTHOOK: query: LOAD DATA LOCAL INPATH "../../data/files/bool.txt" INTO TABLE tab_bool -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@tab_bool -PREHOOK: query: select count(*) from tab_bool -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_bool -#### A masked pattern was here #### -POSTHOOK: query: select count(*) from tab_bool -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tab_bool -#### A masked pattern was here #### -33 -PREHOOK: query: select compute_stats(a, 16) from tab_bool -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_bool -#### A masked pattern was here #### -POSTHOOK: query: select compute_stats(a, 16) from tab_bool -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tab_bool -#### A masked pattern was here #### -{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1} diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out deleted file mode 100644 index b33be22d8581..000000000000 --- a/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out +++ /dev/null @@ -1,176 +0,0 @@ -PREHOOK: query: create table tab_date ( - origin_city_name string, - dest_city_name string, - fl_date date, - arr_delay float, - fl_num int -) -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@tab_date -POSTHOOK: query: create table tab_date ( - origin_city_name string, - dest_city_name string, - fl_date date, - arr_delay float, - fl_num int -) -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@tab_date -PREHOOK: query: load data local inpath '../../data/files/flights_join.txt' overwrite into table tab_date -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@tab_date -POSTHOOK: query: load data local inpath '../../data/files/flights_join.txt' overwrite into table tab_date -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@tab_date -PREHOOK: query: select count(*) from tab_date -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_date -#### A masked pattern was here #### -POSTHOOK: query: select count(*) from tab_date -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tab_date -#### A masked pattern was here #### -20 -PREHOOK: query: select compute_stats(fl_date, 'hll') from tab_date -PREHOOK: type: QUERY -PREHOOK: Input: default@tab_date -#### A masked pattern was here #### -POSTHOOK: query: select compute_stats(fl_date, 'hll') from tab_date -POSTHOOK: type: QUERY -POSTHOOK: Input: default@tab_date -#### A masked pattern was here #### -{"columntype":"Date","min":"2000-11-20","max":"2010-10-29","countnulls":0,"numdistinctvalues":19,"ndvbitvector":HLL������������t������������������������R��ո��������������Y������������������������������������������������犞} -PREHOOK: query: explain -analyze table tab_date compute statistics for columns fl_date -PREHOOK: type: ANALYZE_TABLE -PREHOOK: Input: default@tab_date -PREHOOK: Output: default@tab_date -#### A masked pattern was here #### -POSTHOOK: query: explain -analyze table tab_date compute statistics for columns fl_date -POSTHOOK: type: ANALYZE_TABLE -POSTHOOK: Input: default@tab_date -POSTHOOK: Output: default@tab_date -#### A masked pattern was here #### -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-2 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: tab_date - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: fl_date (type: date) - outputColumnNames: fl_date - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE - Group By Operator - aggregations: min(fl_date), max(fl_date), count(1), count(fl_date), compute_bit_vector_hll(fl_date) - minReductionHashAggr: 0.99 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary) - Execution mode: llap - LLAP IO: all inputs - Reducer 2 - Execution mode: vectorized, llap - Reduce Operator Tree: - Group By Operator - aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4) - mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 'DATE' (type: string), _col0 (type: date), _col1 (type: date), (_col2 - _col3) (type: bigint), COALESCE(ndv_compute_bit_vector(_col4),0) (type: bigint), _col4 (type: binary) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - Stage: Stage-2 - Stats Work - Basic Stats Work: - Column Stats Desc: - Columns: fl_date - Column Types: date - Table: default.tab_date - -PREHOOK: query: analyze table tab_date compute statistics for columns fl_date -PREHOOK: type: ANALYZE_TABLE -PREHOOK: Input: default@tab_date -PREHOOK: Output: default@tab_date -#### A masked pattern was here #### -POSTHOOK: query: analyze table tab_date compute statistics for columns fl_date -POSTHOOK: type: ANALYZE_TABLE -POSTHOOK: Input: default@tab_date -POSTHOOK: Output: default@tab_date -#### A masked pattern was here #### -PREHOOK: query: describe formatted tab_date fl_date -PREHOOK: type: DESCTABLE -PREHOOK: Input: default@tab_date -POSTHOOK: query: describe formatted tab_date fl_date -POSTHOOK: type: DESCTABLE -POSTHOOK: Input: default@tab_date -col_name fl_date -data_type date -min 2000-11-20 -max 2010-10-29 -num_nulls 0 -distinct_count 19 -avg_col_len -max_col_len -num_trues -num_falses -bit_vector HL -comment from deserializer -COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"fl_date\":\"true\"}} -PREHOOK: query: alter table tab_date update statistics for column fl_date set ('numDVs'='19', 'highValue'='2015-01-01', 'lowValue'='0') -PREHOOK: type: ALTERTABLE_UPDATETABLESTATS -PREHOOK: Input: default@tab_date -PREHOOK: Output: default@tab_date -POSTHOOK: query: alter table tab_date update statistics for column fl_date set ('numDVs'='19', 'highValue'='2015-01-01', 'lowValue'='0') -POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS -POSTHOOK: Input: default@tab_date -POSTHOOK: Output: default@tab_date -PREHOOK: query: describe formatted tab_date fl_date -PREHOOK: type: DESCTABLE -PREHOOK: Input: default@tab_date -POSTHOOK: query: describe formatted tab_date fl_date -POSTHOOK: type: DESCTABLE -POSTHOOK: Input: default@tab_date -col_name fl_date -data_type date -min 1970-01-01 -max 2015-01-01 -num_nulls 0 -distinct_count 19 -avg_col_len -max_col_len -num_trues -num_falses -bit_vector HL -comment from deserializer -COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"fl_date\":\"true\"}} diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_decimal.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_decimal.q.out deleted file mode 100644 index 810a8b232af21691d93beeb6c41cb11cd2613698..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1512 zcmd6n&u`N(6vvA=Ae95sE*$q{4r`~7t}A1mPlPqFiEin-aNsm~i7O%;x5ReYg!tci zSrTX3GWG+I7Ue~b-w%J^Cr_D7{k{MJlPhuV=DH9zA)oQTi z47%O+q{`Ii3r=Bo>U*Iddhf@6?Fx)u7)>E&Sdb}BV?JX>=!S^G@ebpvBiobUIDDu$ zo7$kx`fiqYJ)}urMWAFi4!jYJywHPju&osfP zFn|kgIrX~vs(8^rUc3DYFYq8`>Kl(?!L;U?gda@7SDtam-47)DZ>3v$_tkC_?U%U& zcEtscG(>YFb!WW`vuvLJ3FXmefBL27v@eShZ8+&pyshSP3T~HUGYUMV9=rROg8Dmk zZ&xLy*eX9!nrYQx4y?`6H3#~`$HVL%0xhrqLh&bw=E9^hFHR^K6@NkEkTj8mz8-G7 zo496Yg!VTFZ?@iU4|g`2RxWNjSPN%U8X**+m?updCrT$W(n}@`$K5_g$#{91=p_#G qJR|hO!Afhj)oS(7UZ6cIEq{NG_7d%~?`i4JV?REra|ZbStNj}*(aS&p diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_double.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_double.q.out deleted file mode 100644 index c6a902022495ea074af0e8131958e0b296821341..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1443 zcmd5+O;6iE5KYAaDIBQ8v4@Vf2SSxPN&^(_2}6uPq;&}9zzHq(Mp*bGcxO{k#eZiu zb~aK;`9`Yc@vdgx%)WU$a)03M`~D$^D=mu~4q+h$Nf? zj3kHMf#(KZ;O_LiS;*Ho(#RnaR7fbvBRLd0!8wRBuuZt?sCwT&4xY=&rpzH_WB+D# zA2Ns5wC_r(p6_;{>jp0L{9U&P2fzAmun){}TJ)OETf;b!s#OZ-;59M=1wKr~G7{U4 zr=?5wrgHrYJ>Wtr)TNAIBoJkh!Ed49LKYHC_+@IoC2ywTGqo9K-N|2$HCY;GjDOqu;e4mPxMqC4 zv*C0)O@hoRiuyq^KqI|aH_xbsIpI?9AAB5o%hf5>N8xClEd!6_aZ;?J_Y_P6dD{Eq>Vmado+`$-X%^Rvzu%_S zq^p$4GR=6e&6LTLRAZ`Q9E;3IyKA?T@p+14YQv&reE;}$W2Mn(yd|s=9v|*$TqW!f XT7*{w%XmY$sMpK&ys}hr@0vdVj4h<> diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_string.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_string.q.out deleted file mode 100644 index 135ac45e7541ae7ca9040dc15b95f805101bc593..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1448 zcmd5+&uiN-6b_6r$PO!v((6MzG#w^s7_3yMd96#!(mGAO!%joBmDEgDQtK&e8T;S& z6x#~4>DsOv%=ohi(U_>=e7D53jSc}RM|qi+>Eoc#VyAIR){Ml^#m^%@Am^H z=96anuDcsh5H+8+N*Ttt1Ey_}zX?2HIv-x?Bb86OU$@szXTI*G-)p|yF bNqA0hb2o1O&gQT|cuiO*INOUFKij_ml!C!p diff --git a/ql/src/test/results/clientpositive/llap/show_functions.q.out b/ql/src/test/results/clientpositive/llap/show_functions.q.out index e5a249647112..9642bfbd8b0e 100644 --- a/ql/src/test/results/clientpositive/llap/show_functions.q.out +++ b/ql/src/test/results/clientpositive/llap/show_functions.q.out @@ -83,7 +83,6 @@ collect_list collect_set compute_bit_vector_fm compute_bit_vector_hll -compute_stats concat concat_ws context_ngrams @@ -533,7 +532,6 @@ collect_list collect_set compute_bit_vector_fm compute_bit_vector_hll -compute_stats concat concat_ws context_ngrams @@ -713,7 +711,6 @@ collect_list collect_set compute_bit_vector_fm compute_bit_vector_hll -compute_stats concat concat_ws context_ngrams From 202906022b450ab2e90928f514f6cad39830ad1e Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Mon, 25 Dec 2023 19:24:04 +0800 Subject: [PATCH 115/179] HIVE-27530: Implement direct SQL for alter partitions to improve performance (Wechar Yu, reviewed by Denys Kuzmenko, Sai Hemanth Gantasala) Closes #4517 --- .../hadoop/hive/metastore/Batchable.java | 30 +- .../hive/metastore/DatabaseProduct.java | 16 + .../hive/metastore/DirectSqlInsertPart.java | 8 +- .../hive/metastore/DirectSqlUpdatePart.java | 1536 +++++++++++++++++ .../hive/metastore/DirectSqlUpdateStat.java | 727 -------- .../hadoop/hive/metastore/HMSHandler.java | 18 +- .../hive/metastore/MetaStoreDirectSql.java | 127 +- .../hadoop/hive/metastore/ObjectStore.java | 145 +- .../hadoop/hive/metastore/txn/TxnUtils.java | 16 + .../hive/metastore/TestObjectStore.java | 2 +- .../hive/metastore/tools/BenchmarkTool.java | 3 + .../hive/metastore/tools/HMSBenchmarks.java | 30 + 12 files changed, 1817 insertions(+), 841 deletions(-) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java delete mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java index e3fd5a4bf14a..571d6bdbd1dc 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Batchable.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.metastore; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import javax.jdo.Query; @@ -38,7 +39,7 @@ public abstract class Batchable { public static final int NO_BATCHING = -1; private List queries = null; - public abstract List run(List input) throws MetaException; + public abstract List run(List input) throws Exception; public void addQueryAfterUse(Query query) { if (queries == null) { @@ -70,18 +71,25 @@ public static List runBatched( final int batchSize, List input, Batchable runnable) throws MetaException { - if (batchSize == NO_BATCHING || batchSize >= input.size()) { - return runnable.run(input); + if (input == null || input.isEmpty()) { + return Collections.emptyList(); } - List result = new ArrayList(input.size()); - for (int fromIndex = 0, toIndex = 0; toIndex < input.size(); fromIndex = toIndex) { - toIndex = Math.min(fromIndex + batchSize, input.size()); - List batchedInput = input.subList(fromIndex, toIndex); - List batchedOutput = runnable.run(batchedInput); - if (batchedOutput != null) { - result.addAll(batchedOutput); + try { + if (batchSize == NO_BATCHING || batchSize >= input.size()) { + return runnable.run(input); } + List result = new ArrayList<>(input.size()); + for (int fromIndex = 0, toIndex = 0; toIndex < input.size(); fromIndex = toIndex) { + toIndex = Math.min(fromIndex + batchSize, input.size()); + List batchedInput = input.subList(fromIndex, toIndex); + List batchedOutput = runnable.run(batchedInput); + if (batchedOutput != null) { + result.addAll(batchedOutput); + } + } + return result; + } catch (Exception e) { + throw ExceptionHandler.newMetaException(e); } - return result; } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java index 6e04bf0d6f78..ea3faf09113d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Stream; +import java.util.stream.Collectors; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.conf.Configurable; @@ -748,6 +749,21 @@ public Object getBoolean(boolean val) { return val; } + /** + * Get the max rows in a query with paramSize. + * @param batch the configured batch size + * @param paramSize the parameter size in a query statement + * @return the max allowed rows in a query + */ + public int getMaxRows(int batch, int paramSize) { + if (isSQLSERVER()) { + // SQL Server supports a maximum of 2100 parameters in a request. Adjust the maxRowsInBatch accordingly + int maxAllowedRows = (2100 - paramSize) / paramSize; + return Math.min(batch, maxAllowedRows); + } + return batch; + } + // This class implements the Configurable interface for the benefit // of "plugin" instances created via reflection (see invocation of // ReflectionUtils.newInstance in method determineDatabaseProduct) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlInsertPart.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlInsertPart.java index be17470edd66..ba205ebe705b 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlInsertPart.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlInsertPart.java @@ -83,13 +83,7 @@ private void insertInBatch(String tableName, String columns, int columnCount, in return; } int maxRowsInBatch = batchSize > 0 ? batchSize : rowCount; - if (dbType.isSQLSERVER()) { - // SQL Server supports a maximum of 2100 parameters in a request. Adjust the maxRowsInBatch accordingly - int maxAllowedRows = (2100 - columnCount) / columnCount; - if (maxRowsInBatch > maxAllowedRows) { - maxRowsInBatch = maxAllowedRows; - } - } + maxRowsInBatch = dbType.getMaxRows(maxRowsInBatch, columnCount); int maxBatches = rowCount / maxRowsInBatch; int last = rowCount % maxRowsInBatch; String rowFormat = "(" + repeat(",?", columnCount).substring(1) + ")"; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java new file mode 100644 index 000000000000..f6e41f090943 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java @@ -0,0 +1,1536 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.metastore; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.metastore.api.ColumnStatistics; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.SkewedInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; +import org.apache.hadoop.hive.metastore.events.UpdatePartitionColumnStatEvent; +import org.apache.hadoop.hive.metastore.events.UpdatePartitionColumnStatEventBatch; +import org.apache.hadoop.hive.metastore.messaging.EventMessage; +import org.apache.hadoop.hive.metastore.model.MColumnDescriptor; +import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics; +import org.apache.hadoop.hive.metastore.model.MStringList; +import org.apache.hadoop.hive.metastore.tools.SQLGenerator; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; +import org.datanucleus.ExecutionContext; +import org.datanucleus.api.jdo.JDOPersistenceManager; +import org.datanucleus.metadata.AbstractClassMetaData; +import org.datanucleus.metadata.IdentityType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; + +import javax.jdo.PersistenceManager; +import javax.jdo.datastore.JDOConnection; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE; +import static org.apache.hadoop.hive.metastore.HMSHandler.getPartValsFromName; +import static org.apache.hadoop.hive.metastore.MetastoreDirectSqlUtils.executeWithArray; +import static org.apache.hadoop.hive.metastore.MetastoreDirectSqlUtils.extractSqlInt; +import static org.apache.hadoop.hive.metastore.MetastoreDirectSqlUtils.extractSqlLong; + +/** + * This class contains the optimizations for MetaStore that rely on direct SQL access to + * the underlying database. It should use ANSI SQL and be compatible with common databases + * such as MySQL (note that MySQL doesn't use full ANSI mode by default), Postgres, etc. + * + * This class separates out the update part from MetaStoreDirectSql class. + */ +class DirectSqlUpdatePart { + private static final Logger LOG = LoggerFactory.getLogger(DirectSqlUpdatePart.class.getName()); + + private final PersistenceManager pm; + private final Configuration conf; + private final DatabaseProduct dbType; + private final int maxBatchSize; + private final SQLGenerator sqlGenerator; + + private static final ReentrantLock derbyLock = new ReentrantLock(true); + + public DirectSqlUpdatePart(PersistenceManager pm, Configuration conf, + DatabaseProduct dbType, int batchSize) { + this.pm = pm; + this.conf = conf; + this.dbType = dbType; + this.maxBatchSize = batchSize; + sqlGenerator = new SQLGenerator(dbType, conf); + } + + /** + * {@link #lockInternal()} and {@link #unlockInternal()} are used to serialize those operations that require + * Select ... For Update to sequence operations properly. In practice that means when running + * with Derby database. See more notes at class level. + */ + private void lockInternal() { + if(dbType.isDERBY()) { + derbyLock.lock(); + } + } + + private void unlockInternal() { + if(dbType.isDERBY()) { + derbyLock.unlock(); + } + } + + void rollbackDBConn(Connection dbConn) { + try { + if (dbConn != null && !dbConn.isClosed()) dbConn.rollback(); + } catch (SQLException e) { + LOG.warn("Failed to rollback db connection ", e); + } + } + + void closeDbConn(JDOConnection jdoConn) { + try { + if (jdoConn != null) { + jdoConn.close(); + } + } catch (Exception e) { + LOG.warn("Failed to close db connection", e); + } + } + + void closeStmt(Statement stmt) { + try { + if (stmt != null && !stmt.isClosed()) stmt.close(); + } catch (SQLException e) { + LOG.warn("Failed to close statement ", e); + } + } + + void close(ResultSet rs) { + try { + if (rs != null && !rs.isClosed()) { + rs.close(); + } + } + catch(SQLException ex) { + LOG.warn("Failed to close statement ", ex); + } + } + + static String quoteString(String input) { + return "'" + input + "'"; + } + + void close(ResultSet rs, Statement stmt, JDOConnection dbConn) { + close(rs); + closeStmt(stmt); + closeDbConn(dbConn); + } + + private void populateInsertUpdateMap(Map statsPartInfoMap, + Map updateMap, + MapinsertMap, + Connection dbConn, Table tbl) throws SQLException, MetaException, NoSuchObjectException { + StringBuilder prefix = new StringBuilder(); + StringBuilder suffix = new StringBuilder(); + Statement statement = null; + ResultSet rs = null; + List queries = new ArrayList<>(); + Set selectedParts = new HashSet<>(); + + List partIdList = statsPartInfoMap.keySet().stream().map( + e -> e.partitionId).collect(Collectors.toList() + ); + + prefix.append("select \"PART_ID\", \"COLUMN_NAME\" from \"PART_COL_STATS\" WHERE "); + TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, + partIdList, "\"PART_ID\"", true, false); + + for (String query : queries) { + try { + statement = dbConn.createStatement(); + LOG.debug("Going to execute query " + query); + rs = statement.executeQuery(query); + while (rs.next()) { + selectedParts.add(new PartColNameInfo(rs.getLong(1), rs.getString(2))); + } + } finally { + close(rs, statement, null); + } + } + + for (Map.Entry entry : statsPartInfoMap.entrySet()) { + PartitionInfo partitionInfo = (PartitionInfo) entry.getKey(); + ColumnStatistics colStats = (ColumnStatistics) entry.getValue(); + long partId = partitionInfo.partitionId; + ColumnStatisticsDesc statsDesc = colStats.getStatsDesc(); + if (!statsDesc.isSetCatName()) { + statsDesc.setCatName(tbl.getCatName()); + } + for (ColumnStatisticsObj statisticsObj : colStats.getStatsObj()) { + PartColNameInfo temp = new PartColNameInfo(partId, statisticsObj.getColName()); + if (selectedParts.contains(temp)) { + updateMap.put(temp, StatObjectConverter. + convertToMPartitionColumnStatistics(null, statsDesc, statisticsObj, colStats.getEngine())); + } else { + insertMap.put(temp, StatObjectConverter. + convertToMPartitionColumnStatistics(null, statsDesc, statisticsObj, colStats.getEngine())); + } + } + } + } + + private void updatePartColStatTable(Map updateMap, + Connection dbConn) throws SQLException, MetaException, NoSuchObjectException { + PreparedStatement pst = null; + for (Map.Entry entry : updateMap.entrySet()) { + PartColNameInfo partColNameInfo = (PartColNameInfo) entry.getKey(); + Long partId = partColNameInfo.partitionId; + MPartitionColumnStatistics mPartitionColumnStatistics = (MPartitionColumnStatistics) entry.getValue(); + String update = "UPDATE \"PART_COL_STATS\" SET "; + update += StatObjectConverter.getUpdatedColumnSql(mPartitionColumnStatistics); + update += " WHERE \"PART_ID\" = " + partId + " AND " + + " \"COLUMN_NAME\" = " + quoteString(mPartitionColumnStatistics.getColName()); + try { + pst = dbConn.prepareStatement(update); + StatObjectConverter.initUpdatedColumnStatement(mPartitionColumnStatistics, pst); + LOG.debug("Going to execute update " + update); + int numUpdate = pst.executeUpdate(); + if (numUpdate != 1) { + throw new MetaException("Invalid state of PART_COL_STATS for PART_ID " + partId); + } + } finally { + closeStmt(pst); + } + } + } + + private void insertIntoPartColStatTable(Map insertMap, + long maxCsId, + Connection dbConn) throws SQLException, MetaException, NoSuchObjectException { + PreparedStatement preparedStatement = null; + int numRows = 0; + String insert = "INSERT INTO \"PART_COL_STATS\" (\"CS_ID\", \"CAT_NAME\", \"DB_NAME\"," + + "\"TABLE_NAME\", \"PARTITION_NAME\", \"COLUMN_NAME\", \"COLUMN_TYPE\", \"PART_ID\"," + + " \"LONG_LOW_VALUE\", \"LONG_HIGH_VALUE\", \"DOUBLE_HIGH_VALUE\", \"DOUBLE_LOW_VALUE\"," + + " \"BIG_DECIMAL_LOW_VALUE\", \"BIG_DECIMAL_HIGH_VALUE\", \"NUM_NULLS\", \"NUM_DISTINCTS\", \"BIT_VECTOR\" ," + + " \"HISTOGRAM\", \"AVG_COL_LEN\", \"MAX_COL_LEN\", \"NUM_TRUES\", \"NUM_FALSES\", \"LAST_ANALYZED\", \"ENGINE\") values " + + "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + + try { + preparedStatement = dbConn.prepareStatement(insert); + for (Map.Entry entry : insertMap.entrySet()) { + PartColNameInfo partColNameInfo = (PartColNameInfo) entry.getKey(); + Long partId = partColNameInfo.partitionId; + MPartitionColumnStatistics mPartitionColumnStatistics = (MPartitionColumnStatistics) entry.getValue(); + + preparedStatement.setLong(1, maxCsId); + preparedStatement.setString(2, mPartitionColumnStatistics.getCatName()); + preparedStatement.setString(3, mPartitionColumnStatistics.getDbName()); + preparedStatement.setString(4, mPartitionColumnStatistics.getTableName()); + preparedStatement.setString(5, mPartitionColumnStatistics.getPartitionName()); + preparedStatement.setString(6, mPartitionColumnStatistics.getColName()); + preparedStatement.setString(7, mPartitionColumnStatistics.getColType()); + preparedStatement.setLong(8, partId); + preparedStatement.setObject(9, mPartitionColumnStatistics.getLongLowValue()); + preparedStatement.setObject(10, mPartitionColumnStatistics.getLongHighValue()); + preparedStatement.setObject(11, mPartitionColumnStatistics.getDoubleHighValue()); + preparedStatement.setObject(12, mPartitionColumnStatistics.getDoubleLowValue()); + preparedStatement.setString(13, mPartitionColumnStatistics.getDecimalLowValue()); + preparedStatement.setString(14, mPartitionColumnStatistics.getDecimalHighValue()); + preparedStatement.setObject(15, mPartitionColumnStatistics.getNumNulls()); + preparedStatement.setObject(16, mPartitionColumnStatistics.getNumDVs()); + preparedStatement.setObject(17, mPartitionColumnStatistics.getBitVector()); + preparedStatement.setBytes(18, mPartitionColumnStatistics.getHistogram()); + preparedStatement.setObject(19, mPartitionColumnStatistics.getAvgColLen()); + preparedStatement.setObject(20, mPartitionColumnStatistics.getMaxColLen()); + preparedStatement.setObject(21, mPartitionColumnStatistics.getNumTrues()); + preparedStatement.setObject(22, mPartitionColumnStatistics.getNumFalses()); + preparedStatement.setLong(23, mPartitionColumnStatistics.getLastAnalyzed()); + preparedStatement.setString(24, mPartitionColumnStatistics.getEngine()); + + maxCsId++; + numRows++; + preparedStatement.addBatch(); + if (numRows == maxBatchSize) { + preparedStatement.executeBatch(); + numRows = 0; + } + } + + if (numRows != 0) { + preparedStatement.executeBatch(); + } + } finally { + closeStmt(preparedStatement); + } + } + + private Map getParamValues(Connection dbConn, List partIdList) throws SQLException { + List queries = new ArrayList<>(); + StringBuilder prefix = new StringBuilder(); + StringBuilder suffix = new StringBuilder(); + Statement statement = null; + ResultSet rs = null; + + prefix.append("select \"PART_ID\", \"PARAM_VALUE\" " + + " from \"PARTITION_PARAMS\" where " + + " \"PARAM_KEY\" = 'COLUMN_STATS_ACCURATE' " + + " and "); + TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, + partIdList, "\"PART_ID\"", true, false); + + Map partIdToParaMap = new HashMap<>(); + for (String query : queries) { + try { + statement = dbConn.createStatement(); + LOG.debug("Going to execute query " + query); + rs = statement.executeQuery(query); + while (rs.next()) { + partIdToParaMap.put(rs.getLong(1), rs.getString(2)); + } + } finally { + close(rs, statement, null); + } + } + return partIdToParaMap; + } + + private void updateWriteIdForPartitions(Connection dbConn, long writeId, List partIdList) throws SQLException { + StringBuilder prefix = new StringBuilder(); + List queries = new ArrayList<>(); + StringBuilder suffix = new StringBuilder(); + + prefix.append("UPDATE \"PARTITIONS\" set \"WRITE_ID\" = " + writeId + " where "); + TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, + partIdList, "\"PART_ID\"", false, false); + + Statement statement = null; + for (String query : queries) { + try { + statement = dbConn.createStatement(); + LOG.debug("Going to execute update " + query); + statement.executeUpdate(query); + } finally { + closeStmt(statement); + } + } + } + + private Map> updatePartitionParamTable(Connection dbConn, + Map partitionInfoMap, + String validWriteIds, + long writeId, + boolean isAcidTable) + throws SQLException, MetaException { + Map> result = new HashMap<>(); + boolean areTxnStatsSupported = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_TXN_STATS_ENABLED); + String insert = "INSERT INTO \"PARTITION_PARAMS\" (\"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\") " + + "VALUES( ? , 'COLUMN_STATS_ACCURATE' , ? )"; + String delete = "DELETE from \"PARTITION_PARAMS\" " + + " where \"PART_ID\" = ? " + + " and \"PARAM_KEY\" = 'COLUMN_STATS_ACCURATE'"; + String update = "UPDATE \"PARTITION_PARAMS\" set \"PARAM_VALUE\" = ? " + + " where \"PART_ID\" = ? " + + " and \"PARAM_KEY\" = 'COLUMN_STATS_ACCURATE'"; + int numInsert = 0; + int numDelete = 0; + int numUpdate = 0; + + List partIdList = partitionInfoMap.keySet().stream().map( + e -> e.partitionId).collect(Collectors.toList() + ); + + // get the old parameters from PARTITION_PARAMS table. + Map partIdToParaMap = getParamValues(dbConn, partIdList); + + try (PreparedStatement statementInsert = dbConn.prepareStatement(insert); + PreparedStatement statementDelete = dbConn.prepareStatement(delete); + PreparedStatement statementUpdate = dbConn.prepareStatement(update)) { + for (Map.Entry entry : partitionInfoMap.entrySet()) { + PartitionInfo partitionInfo = (PartitionInfo) entry.getKey(); + ColumnStatistics colStats = (ColumnStatistics) entry.getValue(); + List colNames = colStats.getStatsObj().stream().map(e -> e.getColName()).collect(Collectors.toList()); + long partWriteId = partitionInfo.writeId; + long partId = partitionInfo.partitionId; + Map newParameter; + + if (!partIdToParaMap.containsKey(partId)) { + newParameter = new HashMap<>(); + newParameter.put(COLUMN_STATS_ACCURATE, "TRUE"); + StatsSetupConst.setColumnStatsState(newParameter, colNames); + statementInsert.setLong(1, partId); + statementInsert.setString(2, newParameter.get(COLUMN_STATS_ACCURATE)); + numInsert++; + statementInsert.addBatch(); + if (numInsert == maxBatchSize) { + LOG.debug(" Executing insert " + insert); + statementInsert.executeBatch(); + numInsert = 0; + } + } else { + String oldStats = partIdToParaMap.get(partId); + + Map oldParameter = new HashMap<>(); + oldParameter.put(COLUMN_STATS_ACCURATE, oldStats); + + newParameter = new HashMap<>(); + newParameter.put(COLUMN_STATS_ACCURATE, oldStats); + StatsSetupConst.setColumnStatsState(newParameter, colNames); + + if (isAcidTable) { + String errorMsg = ObjectStore.verifyStatsChangeCtx( + colStats.getStatsDesc().getDbName() + "." + colStats.getStatsDesc().getTableName(), + oldParameter, newParameter, writeId, validWriteIds, true); + if (errorMsg != null) { + throw new MetaException(errorMsg); + } + } + + if (isAcidTable && + (!areTxnStatsSupported || !ObjectStore.isCurrentStatsValidForTheQuery(oldParameter, partWriteId, + validWriteIds, true))) { + statementDelete.setLong(1, partId); + statementDelete.addBatch(); + numDelete++; + if (numDelete == maxBatchSize) { + statementDelete.executeBatch(); + numDelete = 0; + LOG.debug("Removed COLUMN_STATS_ACCURATE from the parameters of the partition " + + colStats.getStatsDesc().getDbName() + "." + colStats.getStatsDesc().getTableName() + "." + + colStats.getStatsDesc().getPartName()); + } + } else { + statementUpdate.setString(1, newParameter.get(COLUMN_STATS_ACCURATE)); + statementUpdate.setLong(2, partId); + statementUpdate.addBatch(); + numUpdate++; + if (numUpdate == maxBatchSize) { + LOG.debug(" Executing update " + statementUpdate); + statementUpdate.executeBatch(); + numUpdate = 0; + } + } + } + result.put(partitionInfo.partitionName, newParameter); + } + + if (numInsert != 0) { + statementInsert.executeBatch(); + } + + if (numUpdate != 0) { + statementUpdate.executeBatch(); + } + + if (numDelete != 0) { + statementDelete.executeBatch(); + } + + if (isAcidTable) { + updateWriteIdForPartitions(dbConn, writeId, partIdList); + } + return result; + } + } + + + private Map getPartitionInfo(Connection dbConn, long tblId, + Map partColStatsMap) + throws SQLException, MetaException { + List queries = new ArrayList<>(); + StringBuilder prefix = new StringBuilder(); + StringBuilder suffix = new StringBuilder(); + Statement statement = null; + ResultSet rs = null; + Map partitionInfoMap = new HashMap<>(); + + List partKeys = partColStatsMap.keySet().stream().map( + e -> quoteString(e)).collect(Collectors.toList() + ); + + prefix.append("select \"PART_ID\", \"WRITE_ID\", \"PART_NAME\" from \"PARTITIONS\" where "); + suffix.append(" and \"TBL_ID\" = " + tblId); + TxnUtils.buildQueryWithINClauseStrings(conf, queries, prefix, suffix, + partKeys, "\"PART_NAME\"", true, false); + + for (String query : queries) { + // Select for update makes sure that the partitions are not modified while the stats are getting updated. + query = sqlGenerator.addForUpdateClause(query); + try { + statement = dbConn.createStatement(); + LOG.debug("Going to execute query <" + query + ">"); + rs = statement.executeQuery(query); + while (rs.next()) { + PartitionInfo partitionInfo = new PartitionInfo(rs.getLong(1), + rs.getLong(2), rs.getString(3)); + partitionInfoMap.put(partitionInfo, partColStatsMap.get(rs.getString(3))); + } + } finally { + close(rs, statement, null); + } + } + return partitionInfoMap; + } + + private void setAnsiQuotes(Connection dbConn) throws SQLException { + if (sqlGenerator.getDbProduct().isMYSQL()) { + try (Statement stmt = dbConn.createStatement()) { + stmt.execute("SET @@session.sql_mode=ANSI_QUOTES"); + } + } + } + + /** + * Update the statistics for the given partitions. Add the notification logs also. + * @return map of partition key to column stats if successful, null otherwise. + */ + public Map> updatePartitionColumnStatistics(Map partColStatsMap, + Table tbl, long csId, + String validWriteIds, long writeId, + List transactionalListeners) + throws MetaException { + JDOConnection jdoConn = null; + Connection dbConn = null; + boolean committed = false; + try { + lockInternal(); + jdoConn = pm.getDataStoreConnection(); + dbConn = (Connection) (jdoConn.getNativeConnection()); + + setAnsiQuotes(dbConn); + + Map partitionInfoMap = getPartitionInfo(dbConn, tbl.getId(), partColStatsMap); + + Map> result = + updatePartitionParamTable(dbConn, partitionInfoMap, validWriteIds, writeId, TxnUtils.isAcidTable(tbl)); + + Map insertMap = new HashMap<>(); + Map updateMap = new HashMap<>(); + populateInsertUpdateMap(partitionInfoMap, updateMap, insertMap, dbConn, tbl); + + LOG.info("Number of stats to insert " + insertMap.size() + " update " + updateMap.size()); + + if (insertMap.size() != 0) { + insertIntoPartColStatTable(insertMap, csId, dbConn); + } + + if (updateMap.size() != 0) { + updatePartColStatTable(updateMap, dbConn); + } + + if (transactionalListeners != null) { + UpdatePartitionColumnStatEventBatch eventBatch = new UpdatePartitionColumnStatEventBatch(null); + for (Map.Entry entry : result.entrySet()) { + Map parameters = (Map) entry.getValue(); + ColumnStatistics colStats = partColStatsMap.get(entry.getKey()); + List partVals = getPartValsFromName(tbl, colStats.getStatsDesc().getPartName()); + UpdatePartitionColumnStatEvent event = new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, + tbl, writeId, null); + eventBatch.addPartColStatEvent(event); + } + MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, + EventMessage.EventType.UPDATE_PARTITION_COLUMN_STAT_BATCH, eventBatch, dbConn, sqlGenerator); + } + dbConn.commit(); + committed = true; + return result; + } catch (Exception e) { + LOG.error("Unable to update Column stats for " + tbl.getTableName(), e); + throw new MetaException("Unable to update Column stats for " + tbl.getTableName() + + " due to: " + e.getMessage()); + } finally { + if (!committed) { + rollbackDBConn(dbConn); + } + closeDbConn(jdoConn); + unlockInternal(); + } + } + + /** + * Gets the next CS id from sequence MPartitionColumnStatistics and increment the CS id by numStats. + * @return The CD id before update. + */ + public long getNextCSIdForMPartitionColumnStatistics(long numStats) throws MetaException { + Statement statement = null; + ResultSet rs = null; + long maxCsId = 0; + boolean committed = false; + Connection dbConn = null; + JDOConnection jdoConn = null; + + try { + lockInternal(); + jdoConn = pm.getDataStoreConnection(); + dbConn = (Connection) (jdoConn.getNativeConnection()); + + setAnsiQuotes(dbConn); + + // This loop will be iterated at max twice. If there is no records, it will first insert and then do a select. + // We are not using any upsert operations as select for update and then update is required to make sure that + // the caller gets a reserved range for CSId not used by any other thread. + boolean insertDone = false; + while (maxCsId == 0) { + String query = sqlGenerator.addForUpdateClause("SELECT \"NEXT_VAL\" FROM \"SEQUENCE_TABLE\" " + + "WHERE \"SEQUENCE_NAME\"= " + + quoteString("org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics")); + LOG.debug("Going to execute query " + query); + statement = dbConn.createStatement(); + rs = statement.executeQuery(query); + if (rs.next()) { + maxCsId = rs.getLong(1); + } else if (insertDone) { + throw new MetaException("Invalid state of SEQUENCE_TABLE for MPartitionColumnStatistics"); + } else { + insertDone = true; + closeStmt(statement); + statement = dbConn.createStatement(); + query = "INSERT INTO \"SEQUENCE_TABLE\" (\"SEQUENCE_NAME\", \"NEXT_VAL\") VALUES ( " + + quoteString("org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics") + "," + 1 + + ")"; + try { + statement.executeUpdate(query); + } catch (SQLException e) { + // If the record is already inserted by some other thread continue to select. + if (dbType.isDuplicateKeyError(e)) { + continue; + } + LOG.error("Unable to insert into SEQUENCE_TABLE for MPartitionColumnStatistics.", e); + throw e; + } finally { + closeStmt(statement); + } + } + } + + long nextMaxCsId = maxCsId + numStats + 1; + closeStmt(statement); + statement = dbConn.createStatement(); + String query = "UPDATE \"SEQUENCE_TABLE\" SET \"NEXT_VAL\" = " + + nextMaxCsId + + " WHERE \"SEQUENCE_NAME\" = " + + quoteString("org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics"); + statement.executeUpdate(query); + dbConn.commit(); + committed = true; + return maxCsId; + } catch (Exception e) { + LOG.error("Unable to getNextCSIdForMPartitionColumnStatistics", e); + throw new MetaException("Unable to getNextCSIdForMPartitionColumnStatistics " + + " due to: " + e.getMessage()); + } finally { + if (!committed) { + rollbackDBConn(dbConn); + } + close(rs, statement, jdoConn); + unlockInternal(); + } + } + + public void alterPartitions(Map, Long> partValuesToId, Map partIdToSdId, + List newParts) throws MetaException { + List partIds = new ArrayList<>(newParts.size()); + Map>> partParamsOpt = new HashMap<>(); + Map idToSd = new HashMap<>(); + for (Partition newPart : newParts) { + Long partId = partValuesToId.get(newPart.getValues()); + Long sdId = partIdToSdId.get(partId); + partIds.add(partId); + partParamsOpt.put(partId, Optional.ofNullable(newPart.getParameters())); + idToSd.put(sdId, newPart.getSd()); + } + + // alter partitions does not change partition values, + // so only PARTITIONS and PARTITION_PARAMS need to update. + updatePartitionsInBatch(partValuesToId, newParts); + updateParamTableInBatch("\"PARTITION_PARAMS\"", "\"PART_ID\"", partIds, partParamsOpt); + updateStorageDescriptorInBatch(idToSd); + } + + private interface ThrowableConsumer { + void accept(T t) throws SQLException, MetaException; + } + + private List filterIdsByNonNullValue(List ids, Map map) { + return ids.stream().filter(id -> map.get(id) != null).collect(Collectors.toList()); + } + + private void updateWithStatement(ThrowableConsumer consumer, String query) + throws MetaException { + JDOConnection jdoConn = pm.getDataStoreConnection(); + boolean doTrace = LOG.isDebugEnabled(); + long start = doTrace ? System.nanoTime() : 0; + try (PreparedStatement statement = + ((Connection) jdoConn.getNativeConnection()).prepareStatement(query)) { + consumer.accept(statement); + MetastoreDirectSqlUtils.timingTrace(doTrace, query, start, doTrace ? System.nanoTime() : 0); + } catch (SQLException e) { + LOG.error("Failed to execute update query: " + query, e); + throw new MetaException("Unable to execute update due to: " + e.getMessage()); + } finally { + closeDbConn(jdoConn); + } + } + + private void updatePartitionsInBatch(Map, Long> partValuesToId, + List newParts) throws MetaException { + List columns = Arrays.asList("\"CREATE_TIME\"", "\"LAST_ACCESS_TIME\"", "\"WRITE_ID\""); + List conditionKeys = Arrays.asList("\"PART_ID\""); + String stmt = TxnUtils.createUpdatePreparedStmt("\"PARTITIONS\"", columns, conditionKeys); + int maxRows = dbType.getMaxRows(maxBatchSize, 4); + updateWithStatement(statement -> Batchable.runBatched(maxRows, newParts, new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Partition p : input) { + statement.setLong(1, p.getCreateTime()); + statement.setLong(2, p.getLastAccessTime()); + statement.setLong(3, p.getWriteId()); + statement.setLong(4, partValuesToId.get(p.getValues())); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + }), stmt); + } + + /* Get stringListId from both SKEWED_VALUES and SKEWED_COL_VALUE_LOC_MAP tables. */ + private List getStringListId(List sdIds) throws MetaException { + return Batchable.runBatched(maxBatchSize, sdIds, new Batchable() { + @Override + public List run(List input) throws Exception { + List result = new ArrayList<>(); + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryFromSkewedValues = "select \"STRING_LIST_ID_EID\" " + + "from \"SKEWED_VALUES\" where \"SD_ID_OID\" in (" + idLists + ")"; + try (QueryWrapper query = + new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryFromSkewedValues))) { + List sqlResult = executeWithArray(query.getInnerQuery(), null, queryFromSkewedValues); + result.addAll(sqlResult); + } + String queryFromValueLoc = "select \"STRING_LIST_ID_KID\" " + + "from \"SKEWED_COL_VALUE_LOC_MAP\" where \"SD_ID\" in (" + idLists + ")"; + try (QueryWrapper query = + new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryFromValueLoc))) { + List sqlResult = executeWithArray(query.getInnerQuery(), null, queryFromValueLoc); + result.addAll(sqlResult); + } + return result; + } + }); + } + + private void updateParamTableInBatch(String paramTable, String idColumn, List ids, + Map>> newParamsOpt) throws MetaException { + Map> oldParams = getParams(paramTable, idColumn, ids); + + List> paramsToDelete = new ArrayList<>(); + List>> paramsToUpdate = new ArrayList<>(); + List>> paramsToAdd = new ArrayList<>(); + + for (Long id : ids) { + Map oldParam = oldParams.getOrDefault(id, new HashMap<>()); + Map newParam = newParamsOpt.get(id).orElseGet(HashMap::new); + for (Map.Entry entry : oldParam.entrySet()) { + String key = entry.getKey(); + String oldValue = entry.getValue(); + if (!newParam.containsKey(key)) { + paramsToDelete.add(Pair.of(id, key)); + } else if (!oldValue.equals(newParam.get(key))) { + paramsToUpdate.add(Pair.of(id, Pair.of(key, newParam.get(key)))); + } + } + List>> newlyParams = newParam.entrySet().stream() + .filter(entry -> !oldParam.containsKey(entry.getKey())) + .map(entry -> Pair.of(id, Pair.of(entry.getKey(), entry.getValue()))) + .collect(Collectors.toList()); + paramsToAdd.addAll(newlyParams); + } + + deleteParams(paramTable, idColumn, paramsToDelete); + updateParams(paramTable, idColumn, paramsToUpdate); + insertParams(paramTable, idColumn, paramsToAdd); + } + + private Map> getParams(String paramTable, String idName, + List ids) throws MetaException { + Map> idToParams = new HashMap<>(); + Batchable.runBatched(maxBatchSize, ids, new Batchable() { + @Override + public List run(List input) throws MetaException { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryText = "select " + idName + ", \"PARAM_KEY\", \"PARAM_VALUE\" from " + + paramTable + " where " + idName + " in (" + idLists + ")"; + try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { + List sqlResult = executeWithArray(query.getInnerQuery(), null, queryText); + for (Object[] row : sqlResult) { + Long id = extractSqlLong(row[0]); + String paramKey = (String) row[1]; + String paramVal = (String) row[2]; + idToParams.computeIfAbsent(id, key -> new HashMap<>()).put(paramKey, paramVal); + } + } + return null; + } + }); + return idToParams; + } + + private void deleteParams(String paramTable, String idColumn, + List> deleteIdKeys) throws MetaException { + String deleteStmt = "delete from " + paramTable + " where " + idColumn + "=? and PARAM_KEY=?"; + int maxRows = dbType.getMaxRows(maxBatchSize, 2); + updateWithStatement(statement -> Batchable.runBatched(maxRows, deleteIdKeys, + new Batchable, Void>() { + @Override + public List run(List> input) throws SQLException { + for (Pair pair : input) { + statement.setLong(1, pair.getLeft()); + statement.setString(2, pair.getRight()); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + } + ), deleteStmt); + } + + private void updateParams(String paramTable, String idColumn, + List>> updateIdAndParams) throws MetaException { + List columns = Arrays.asList("\"PARAM_VALUE\""); + List conditionKeys = Arrays.asList(idColumn, "\"PARAM_KEY\""); + String stmt = TxnUtils.createUpdatePreparedStmt(paramTable, columns, conditionKeys); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, updateIdAndParams, + new Batchable>, Object>() { + @Override + public List run(List>> input) throws SQLException { + for (Pair> pair : input) { + statement.setString(1, pair.getRight().getRight()); + statement.setLong(2, pair.getLeft()); + statement.setString(3, pair.getRight().getLeft()); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + } + ), stmt); + } + + private void insertParams(String paramTable, String idColumn, + List>> addIdAndParams) throws MetaException { + List columns = Arrays.asList(idColumn, "\"PARAM_KEY\"", "\"PARAM_VALUE\""); + String query = TxnUtils.createInsertPreparedStmt(paramTable, columns); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, addIdAndParams, + new Batchable>, Void>() { + @Override + public List run(List>> input) throws SQLException { + for (Pair> pair : input) { + statement.setLong(1, pair.getLeft()); + statement.setString(2, pair.getRight().getLeft()); + statement.setString(3, pair.getRight().getRight()); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + } + ), query); + } + + private void updateStorageDescriptorInBatch(Map idToSd) + throws MetaException { + Map sdIdToCdId = new HashMap<>(); + Map sdIdToSerdeId = new HashMap<>(); + List cdIds = new ArrayList<>(); + List validSdIds = filterIdsByNonNullValue(new ArrayList<>(idToSd.keySet()), idToSd); + Batchable.runBatched(maxBatchSize, validSdIds, new Batchable() { + @Override + public List run(List input) throws Exception { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryText = "select \"SD_ID\", \"CD_ID\", \"SERDE_ID\" from \"SDS\" " + + "where \"SD_ID\" in (" + idLists + ")"; + try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { + List sqlResult = executeWithArray(query.getInnerQuery(), null, queryText); + for (Object[] row : sqlResult) { + Long sdId = extractSqlLong(row[0]); + Long cdId = extractSqlLong(row[1]); + Long serdeId = extractSqlLong(row[2]); + sdIdToCdId.put(sdId, cdId); + sdIdToSerdeId.put(sdId, serdeId); + cdIds.add(cdId); + } + } + return null; + } + }); + + Map>> sdParamsOpt = new HashMap<>(); + Map> idToBucketCols = new HashMap<>(); + Map> idToSortCols = new HashMap<>(); + Map idToSkewedInfo = new HashMap<>(); + Map> sdIdToNewColumns = new HashMap<>(); + List serdeIds = new ArrayList<>(); + Map serdeIdToSerde = new HashMap<>(); + Map>> serdeParamsOpt = new HashMap<>(); + for (Long sdId : validSdIds) { + StorageDescriptor sd = idToSd.get(sdId); + sdParamsOpt.put(sdId, Optional.ofNullable(sd.getParameters())); + idToBucketCols.put(sdId, sd.getBucketCols()); + idToSortCols.put(sdId, sd.getSortCols()); + idToSkewedInfo.put(sdId, sd.getSkewedInfo()); + sdIdToNewColumns.put(sdId, sd.getCols()); + + Long serdeId = sdIdToSerdeId.get(sdId); + serdeIds.add(serdeId); + serdeIdToSerde.put(serdeId, sd.getSerdeInfo()); + serdeParamsOpt.put(serdeId, Optional.ofNullable(sd.getSerdeInfo().getParameters())); + } + + updateParamTableInBatch("\"SD_PARAMS\"", "\"SD_ID\"", validSdIds, sdParamsOpt); + updateBucketColsInBatch(idToBucketCols, validSdIds); + updateSortColsInBatch(idToSortCols, validSdIds); + updateSkewedInfoInBatch(idToSkewedInfo, validSdIds); + Map sdIdToNewCdId = updateCDInBatch(cdIds, validSdIds, sdIdToCdId, sdIdToNewColumns); + updateSerdeInBatch(serdeIds, serdeIdToSerde); + updateParamTableInBatch("\"SERDE_PARAMS\"", "\"SERDE_ID\"", serdeIds, serdeParamsOpt); + + List cdIdsMayDelete = sdIdToCdId.entrySet().stream() + .filter(entry -> sdIdToNewCdId.containsKey(entry.getKey())) + .map(entry -> entry.getValue()) + .collect(Collectors.toList()); + + // Update SDS table after CDS to get the freshest CD_ID values. + sdIdToCdId.replaceAll((sdId, cdId) -> + sdIdToNewCdId.containsKey(sdId) ? sdIdToNewCdId.get(sdId) : cdId); + updateSDInBatch(validSdIds, idToSd, sdIdToCdId); + + List usedIds = Batchable.runBatched(maxBatchSize, cdIdsMayDelete, + new Batchable() { + @Override + public List run(List input) throws Exception { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryText = "select \"CD_ID\" from \"SDS\" where \"CD_ID\" in ( " + idLists + ")"; + try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { + List sqlResult = executeWithArray(query.getInnerQuery(), null, queryText); + return new ArrayList<>(sqlResult); + } + } + }); + List unusedCdIds = cdIdsMayDelete.stream().filter(id -> !usedIds.contains(id)).collect(Collectors.toList()); + + deleteCDInBatch(unusedCdIds); + } + + private void updateSDInBatch(List ids, Map idToSd, + Map idToCdId) throws MetaException { + List columns = Arrays.asList("\"CD_ID\"", "\"INPUT_FORMAT\"", "\"IS_COMPRESSED\"", + "\"IS_STOREDASSUBDIRECTORIES\"", "\"LOCATION\"", "\"NUM_BUCKETS\"", "\"OUTPUT_FORMAT\""); + List conditionKeys = Arrays.asList("\"SD_ID\""); + String stmt = TxnUtils.createUpdatePreparedStmt("\"SDS\"", columns, conditionKeys); + int maxRows = dbType.getMaxRows(maxBatchSize, 8); + updateWithStatement(statement -> Batchable.runBatched(maxRows, ids, + new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long sdId : input) { + StorageDescriptor sd = idToSd.get(sdId); + statement.setLong(1, idToCdId.get(sdId)); + statement.setString(2, sd.getInputFormat()); + statement.setBoolean(3, sd.isCompressed()); + statement.setBoolean(4, sd.isStoredAsSubDirectories()); + statement.setString(5, sd.getLocation()); + statement.setInt(6, sd.getNumBuckets()); + statement.setString(7, sd.getOutputFormat()); + statement.setLong(8, sdId); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + } + ), stmt); + } + + private void updateBucketColsInBatch(Map> sdIdToBucketCols, + List sdIds) throws MetaException { + Batchable.runBatched(maxBatchSize, sdIds, new Batchable() { + @Override + public List run(List input) throws MetaException { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryText = "delete from \"BUCKETING_COLS\" where \"SD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, queryText); + return null; + } + }); + List columns = Arrays.asList("\"SD_ID\"", "\"INTEGER_IDX\"", "\"BUCKET_COL_NAME\""); + String stmt = TxnUtils.createInsertPreparedStmt("\"BUCKETING_COLS\"", columns); + List idWithBucketCols = filterIdsByNonNullValue(sdIds, sdIdToBucketCols); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithBucketCols, new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long id : input) { + List bucketCols = sdIdToBucketCols.get(id); + for (int i = 0; i < bucketCols.size(); i++) { + statement.setLong(1, id); + statement.setInt(2, i); + statement.setString(3, bucketCols.get(i)); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + }), stmt); + } + + private void updateSortColsInBatch(Map> sdIdToSortCols, + List sdIds) throws MetaException { + Batchable.runBatched(maxBatchSize, sdIds, new Batchable() { + @Override + public List run(List input) throws MetaException { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryText = "delete from \"SORT_COLS\" where \"SD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, queryText); + return null; + } + }); + + List columns = Arrays.asList("\"SD_ID\"", "\"INTEGER_IDX\"", "\"COLUMN_NAME\"", "\"ORDER\""); + String stmt = TxnUtils.createInsertPreparedStmt("\"SORT_COLS\"", columns); + List idWithSortCols = filterIdsByNonNullValue(sdIds, sdIdToSortCols); + int maxRows = dbType.getMaxRows(maxBatchSize, 4); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithSortCols, new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long id : input) { + List bucketCols = sdIdToSortCols.get(id); + for (int i = 0; i < bucketCols.size(); i++) { + statement.setLong(1, id); + statement.setInt(2, i); + statement.setString(3, bucketCols.get(i).getCol()); + statement.setInt(4, bucketCols.get(i).getOrder()); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + }), stmt); + } + + private void updateSkewedInfoInBatch(Map sdIdToSkewedInfo, + List sdIds) throws MetaException { + // Delete all mapping old stringLists and skewedValues, + // skewedValues first for the foreign key constraint. + List stringListId = getStringListId(sdIds); + if (!stringListId.isEmpty()) { + Batchable.runBatched(maxBatchSize, sdIds, new Batchable() { + @Override + public List run(List input) throws Exception { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String deleteSkewValuesQuery = + "delete from \"SKEWED_VALUES\" where \"SD_ID_OID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteSkewValuesQuery); + String deleteSkewColValueLocMapQuery = + "delete from \"SKEWED_COL_VALUE_LOC_MAP\" where \"SD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteSkewColValueLocMapQuery); + String deleteSkewColNamesQuery = + "delete from \"SKEWED_COL_NAMES\" where \"SD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteSkewColNamesQuery); + return null; + } + }); + Batchable.runBatched(maxBatchSize, stringListId, new Batchable() { + @Override + public List run(List input) throws MetaException { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String deleteStringListValuesQuery = + "delete from \"SKEWED_STRING_LIST_VALUES\" where \"STRING_LIST_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteStringListValuesQuery); + String deleteStringListQuery = + "delete from \"SKEWED_STRING_LIST\" where \"STRING_LIST_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteStringListQuery); + return null; + } + }); + } + + // Generate new stringListId for each SdId + Map> idToSkewedColNames = new HashMap<>(); // used for SKEWED_COL_NAMES + List newStringListId = new ArrayList<>(); // used for SKEWED_STRING_LIST + Map> stringListIdToValues = new HashMap<>(); // used for SKEWED_STRING_LIST_VALUES + Map> sdIdToNewStringListId = new HashMap<>(); // used for SKEWED_VALUES + Map>> sdIdToValueLoc = new HashMap<>(); // used for SKEWED_COL_VALUE_LOC_MAP + + List idWithSkewedInfo = filterIdsByNonNullValue(sdIds, sdIdToSkewedInfo); + for (Long sdId : idWithSkewedInfo) { + SkewedInfo skewedInfo = sdIdToSkewedInfo.get(sdId); + idToSkewedColNames.put(sdId, skewedInfo.getSkewedColNames()); + List> skewedColValues = skewedInfo.getSkewedColValues(); + if (skewedColValues != null) { + for (List colValues : skewedColValues) { + Long nextStringListId = getDataStoreId(MStringList.class); + newStringListId.add(nextStringListId); + sdIdToNewStringListId.computeIfAbsent(sdId, k -> new ArrayList<>()).add(nextStringListId); + stringListIdToValues.put(nextStringListId, colValues); + } + } + Map, String> skewedColValueLocationMaps = skewedInfo.getSkewedColValueLocationMaps(); + if (skewedColValueLocationMaps != null) { + for (Map.Entry, String> entry : skewedColValueLocationMaps.entrySet()) { + List colValues = entry.getKey(); + String location = entry.getValue(); + Long nextStringListId = getDataStoreId(MStringList.class); + newStringListId.add(nextStringListId); + stringListIdToValues.put(nextStringListId, colValues); + sdIdToValueLoc.computeIfAbsent(sdId, k -> new ArrayList<>()).add(Pair.of(nextStringListId, location)); + } + } + } + + insertSkewedColNamesInBatch(idToSkewedColNames, sdIds); + insertStringListInBatch(newStringListId); + insertStringListValuesInBatch(stringListIdToValues, newStringListId); + insertSkewedValuesInBatch(sdIdToNewStringListId, sdIds); + insertSkewColValueLocInBatch(sdIdToValueLoc, sdIds); + } + + private Long getDataStoreId(Class modelClass) throws MetaException { + ExecutionContext ec = ((JDOPersistenceManager) pm).getExecutionContext(); + AbstractClassMetaData cmd = ec.getMetaDataManager().getMetaDataForClass(modelClass, ec.getClassLoaderResolver()); + if (cmd.getIdentityType() == IdentityType.DATASTORE) { + return (Long) ec.getStoreManager().getValueGenerationStrategyValue(ec, cmd, -1); + } else { + throw new MetaException("Identity type is not datastore."); + } + } + + private void insertSkewedColNamesInBatch(Map> sdIdToSkewedColNames, + List sdIds) throws MetaException { + List columns = Arrays.asList("\"SD_ID\"", "\"INTEGER_IDX\"", "\"SKEWED_COL_NAME\""); + String stmt = TxnUtils.createInsertPreparedStmt("\"SKEWED_COL_NAMES\"", columns); + List idWithSkewedCols = filterIdsByNonNullValue(sdIds, sdIdToSkewedColNames); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithSkewedCols, new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long id : input) { + List skewedColNames = sdIdToSkewedColNames.get(id); + for (int i = 0; i < skewedColNames.size(); i++) { + statement.setLong(1, id); + statement.setInt(2, i); + statement.setString(3, skewedColNames.get(i)); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + }), stmt); + } + + private void insertStringListInBatch(List stringListIds) throws MetaException { + List columns = Arrays.asList("\"STRING_LIST_ID\""); + String insertQuery = TxnUtils.createInsertPreparedStmt("\"SKEWED_STRING_LIST\"", columns); + int maxRows = dbType.getMaxRows(maxBatchSize, 1); + updateWithStatement(statement -> Batchable.runBatched(maxRows, stringListIds, + new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long id : input) { + statement.setLong(1, id); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + } + ), insertQuery); + } + + private void insertStringListValuesInBatch(Map> stringListIdToValues, + List stringListIds) throws MetaException { + List columns = Arrays.asList("\"STRING_LIST_ID\"", "\"INTEGER_IDX\"", "\"STRING_LIST_VALUE\""); + String insertQuery = TxnUtils.createInsertPreparedStmt("\"SKEWED_STRING_LIST_VALUES\"", columns); + List idWithStringList = filterIdsByNonNullValue(stringListIds, stringListIdToValues); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithStringList, + new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long stringListId : input) { + List values = stringListIdToValues.get(stringListId); + for (int i = 0; i < values.size(); i++) { + statement.setLong(1, stringListId); + statement.setInt(2, i); + statement.setString(3, values.get(i)); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + } + ), insertQuery); + } + + private void insertSkewedValuesInBatch(Map> sdIdToStringListId, + List sdIds) throws MetaException { + List columns = Arrays.asList("\"SD_ID_OID\"", "\"INTEGER_IDX\"", "\"STRING_LIST_ID_EID\""); + String insertQuery = TxnUtils.createInsertPreparedStmt("\"SKEWED_VALUES\"", columns); + List idWithSkewedValues = filterIdsByNonNullValue(sdIds, sdIdToStringListId); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithSkewedValues, + new Batchable() { + @Override + public List run(List input) throws Exception { + for (Long sdId : input) { + List stringListIds = sdIdToStringListId.get(sdId); + for (int i = 0; i < stringListIds.size(); i++) { + statement.setLong(1, sdId); + statement.setInt(2, i); + statement.setLong(3, stringListIds.get(i)); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + } + ), insertQuery); + } + + private void insertSkewColValueLocInBatch(Map>> sdIdToColValueLoc, + List sdIds) throws MetaException { + List columns = Arrays.asList("\"SD_ID\"", "\"STRING_LIST_ID_KID\"", "\"LOCATION\""); + String insertQuery = TxnUtils.createInsertPreparedStmt("\"SKEWED_COL_VALUE_LOC_MAP\"", columns); + List idWithColValueLoc = filterIdsByNonNullValue(sdIds, sdIdToColValueLoc); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithColValueLoc, + new Batchable() { + @Override + public List run(List input) throws Exception { + for (Long sdId : input) { + List> stringListIdAndLoc = sdIdToColValueLoc.get(sdId); + for (Pair pair : stringListIdAndLoc) { + statement.setLong(1, sdId); + statement.setLong(2, pair.getLeft()); + statement.setString(3, pair.getRight()); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + } + ), insertQuery); + } + + private Map updateCDInBatch(List cdIds, List sdIds, Map sdIdToCdId, + Map> sdIdToNewColumns) throws MetaException { + Map>> cdIdToColIdxPair = new HashMap<>(); + Batchable.runBatched(maxBatchSize, cdIds, new Batchable() { + @Override + public List run(List input) throws Exception { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + String queryText = "select \"CD_ID\", \"COMMENT\", \"COLUMN_NAME\", \"TYPE_NAME\", " + + "\"INTEGER_IDX\" from \"COLUMNS_V2\" where \"CD_ID\" in (" + idLists + ")"; + try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { + List sqlResult = executeWithArray(query.getInnerQuery(), null, queryText); + for (Object[] row : sqlResult) { + Long id = extractSqlLong(row[0]); + String comment = (String) row[1]; + String name = (String) row[2]; + String type = (String) row[3]; + int index = extractSqlInt(row[4]); + FieldSchema field = new FieldSchema(name, type, comment); + cdIdToColIdxPair.computeIfAbsent(id, k -> new ArrayList<>()).add(Pair.of(index, field)); + } + } + return null; + } + }); + List newCdIds = new ArrayList<>(); + Map> newCdIdToCols = new HashMap<>(); + Map oldCdIdToNewCdId = new HashMap<>(); + Map sdIdToNewCdId = new HashMap<>(); + // oldCdId -> [(oldIdx, newIdx)], used to update KEY_CONSTRAINTS + Map>> oldCdIdToColIdxPairs = new HashMap<>(); + for (Long sdId : sdIds) { + Long cdId = sdIdToCdId.get(sdId); + List> cols = cdIdToColIdxPair.get(cdId); + // Placeholder to avoid IndexOutOfBoundsException. + List oldCols = new ArrayList<>(Collections.nCopies(cols.size(), null)); + cols.forEach(pair -> oldCols.set(pair.getLeft(), pair.getRight())); + + List newCols = sdIdToNewColumns.get(sdId); + // Use the new column descriptor only if the old column descriptor differs from the new one. + if (oldCols == null || !oldCols.equals(newCols)) { + if (oldCols != null && newCols != null) { + Long newCdId = getDataStoreId(MColumnDescriptor.class); + newCdIds.add(newCdId); + newCdIdToCols.put(newCdId, newCols); + oldCdIdToNewCdId.put(cdId, newCdId); + sdIdToNewCdId.put(sdId, newCdId); + for (int i = 0; i < oldCols.size(); i++) { + FieldSchema oldCol = oldCols.get(i); + int newIdx = newCols.indexOf(oldCol); + if (newIdx != -1) { + oldCdIdToColIdxPairs.computeIfAbsent(cdId, k -> new ArrayList<>()).add(Pair.of(i, newIdx)); + } + } + } + } + } + + insertCDInBatch(newCdIds, newCdIdToCols); + // TODO: followed the jdo implement now, but it should be an error in such case: + // partitions use the default table cd, when changing partition cd with + // constraint key mapping, the constraints will be update unexpected. + updateKeyConstraintsInBatch(oldCdIdToNewCdId, oldCdIdToColIdxPairs); + + return sdIdToNewCdId; + } + + private void insertCDInBatch(List ids, Map> idToCols) + throws MetaException { + String insertCds = TxnUtils.createInsertPreparedStmt("\"CDS\"", Arrays.asList("\"CD_ID\"")); + int maxRows = dbType.getMaxRows(maxBatchSize, 1); + updateWithStatement(statement -> Batchable.runBatched(maxRows, ids, + new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long id : input) { + statement.setLong(1, id); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + }), insertCds); + + List columns = Arrays.asList("\"CD_ID\"", + "\"COMMENT\"", "\"COLUMN_NAME\"", "\"TYPE_NAME\"", "\"INTEGER_IDX\""); + String insertColumns = TxnUtils.createInsertPreparedStmt("\"COLUMNS_V2\"", columns); + int maxRowsForCDs = dbType.getMaxRows(maxBatchSize, 5); + updateWithStatement(statement -> Batchable.runBatched(maxRowsForCDs, ids, + new Batchable() { + @Override + public List run(List input) throws Exception { + for (Long id : input) { + List cols = idToCols.get(id); + for (int i = 0; i < cols.size(); i++) { + FieldSchema col = cols.get(i); + statement.setLong(1, id); + statement.setString(2, col.getComment()); + statement.setString(3, col.getName()); + statement.setString(4, col.getType()); + statement.setInt(5, i); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + }), insertColumns); + } + + private void updateKeyConstraintsInBatch(Map oldCdIdToNewCdId, + Map>> oldCdIdToColIdxPairs) throws MetaException { + List oldCdIds = new ArrayList<>(oldCdIdToNewCdId.keySet()); + String tableName = "\"KEY_CONSTRAINTS\""; + List parentColumns = Arrays.asList("\"PARENT_CD_ID\"", "\"PARENT_INTEGER_IDX\""); + List childColumns = Arrays.asList("\"CHILD_CD_ID\"", "\"CHILD_INTEGER_IDX\""); + + String updateParent = TxnUtils.createUpdatePreparedStmt(tableName, parentColumns, parentColumns); + String updateChild = TxnUtils.createUpdatePreparedStmt(tableName, childColumns, childColumns); + for (String updateStmt : new String[]{updateParent, updateChild}) { + int maxRows = dbType.getMaxRows(maxBatchSize, 4); + updateWithStatement(statement -> Batchable.runBatched(maxRows, oldCdIds, + new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long oldId : input) { + // Followed the jdo implement to update only mapping columns for KEY_CONSTRAINTS. + if (!oldCdIdToColIdxPairs.containsKey(oldId)) { + continue; + } + Long newId = oldCdIdToNewCdId.get(oldId); + for (Pair idx : oldCdIdToColIdxPairs.get(oldId)) { + statement.setLong(1, newId); + statement.setInt(2, idx.getRight()); + statement.setLong(3, oldId); + statement.setInt(4, idx.getLeft()); + statement.addBatch(); + } + } + statement.executeBatch(); + return null; + } + }), updateStmt); + } + } + + private void deleteCDInBatch(List cdIds) throws MetaException { + Batchable.runBatched(maxBatchSize, cdIds, new Batchable() { + @Override + public List run(List input) throws Exception { + String idLists = MetaStoreDirectSql.getIdListForIn(input); + // First remove any constraints that may be associated with these CDs + String deleteConstraintsByCd = "delete from \"KEY_CONSTRAINTS\" where \"CHILD_CD_ID\" in (" + + idLists + ") or \"PARENT_CD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteConstraintsByCd); + + // Then delete COLUMNS_V2 before CDS for foreign constraints. + String deleteColumns = "delete from \"COLUMNS_V2\" where \"CD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteColumns); + + // Finally delete CDS + String deleteCDs = "delete from \"CDS\" where \"CD_ID\" in (" + idLists + ")"; + updateWithStatement(PreparedStatement::executeUpdate, deleteCDs); + return null; + } + }); + } + + private void updateSerdeInBatch(List ids, Map idToSerde) + throws MetaException { + // Followed the jdo implement to update only NAME and SLIB of SERDES. + List columns = Arrays.asList("\"NAME\"", "\"SLIB\""); + List condKeys = Arrays.asList("\"SERDE_ID\""); + String updateStmt = TxnUtils.createUpdatePreparedStmt("\"SERDES\"", columns, condKeys); + List idWithSerde = filterIdsByNonNullValue(ids, idToSerde); + int maxRows = dbType.getMaxRows(maxBatchSize, 3); + updateWithStatement(statement -> Batchable.runBatched(maxRows, idWithSerde, + new Batchable() { + @Override + public List run(List input) throws SQLException { + for (Long id : input) { + SerDeInfo serde = idToSerde.get(id); + statement.setString(1, serde.getName()); + statement.setString(2, serde.getSerializationLib()); + statement.setLong(3, id); + statement.addBatch(); + } + statement.executeBatch(); + return null; + } + }), updateStmt); + } + + private static final class PartitionInfo { + long partitionId; + long writeId; + String partitionName; + public PartitionInfo(long partitionId, long writeId, String partitionName) { + this.partitionId = partitionId; + this.writeId = writeId; + this.partitionName = partitionName; + } + + @Override + public int hashCode() { + return (int)partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null) { + return false; + } + if (!(o instanceof PartitionInfo)) { + return false; + } + PartitionInfo other = (PartitionInfo)o; + if (this.partitionId != other.partitionId) { + return false; + } + return true; + } + } + + private static final class PartColNameInfo { + long partitionId; + String colName; + public PartColNameInfo(long partitionId, String colName) { + this.partitionId = partitionId; + this.colName = colName; + } + + @Override + public int hashCode() { + return (int)partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null) { + return false; + } + if (!(o instanceof PartColNameInfo)) { + return false; + } + PartColNameInfo other = (PartColNameInfo)o; + if (this.partitionId != other.partitionId) { + return false; + } + if (this.colName.equalsIgnoreCase(other.colName)) { + return true; + } + return false; + } + } +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java deleted file mode 100644 index df1f77f20649..000000000000 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java +++ /dev/null @@ -1,727 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.metastore; - -import org.apache.hadoop.hive.common.StatsSetupConst; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; -import org.apache.hadoop.hive.metastore.events.UpdatePartitionColumnStatEvent; -import org.apache.hadoop.hive.metastore.events.UpdatePartitionColumnStatEventBatch; -import org.apache.hadoop.hive.metastore.messaging.EventMessage; -import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics; -import org.apache.hadoop.hive.metastore.tools.SQLGenerator; -import org.apache.hadoop.hive.metastore.txn.TxnUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; - -import javax.jdo.PersistenceManager; -import javax.jdo.datastore.JDOConnection; -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; - -import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE; -import static org.apache.hadoop.hive.metastore.HMSHandler.getPartValsFromName; -import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog; - -/** - * This class contains the optimizations for MetaStore that rely on direct SQL access to - * the underlying database. It should use ANSI SQL and be compatible with common databases - * such as MySQL (note that MySQL doesn't use full ANSI mode by default), Postgres, etc. - * - * This class separates out the statistics update part from MetaStoreDirectSql class. - */ -class DirectSqlUpdateStat { - private static final Logger LOG = LoggerFactory.getLogger(DirectSqlUpdateStat.class.getName()); - PersistenceManager pm; - Configuration conf; - DatabaseProduct dbType; - int maxBatchSize; - SQLGenerator sqlGenerator; - private static final ReentrantLock derbyLock = new ReentrantLock(true); - - public DirectSqlUpdateStat(PersistenceManager pm, Configuration conf, - DatabaseProduct dbType, int batchSize) { - this.pm = pm; - this.conf = conf; - this.dbType = dbType; - this.maxBatchSize = batchSize; - sqlGenerator = new SQLGenerator(dbType, conf); - } - - /** - * {@link #lockInternal()} and {@link #unlockInternal()} are used to serialize those operations that require - * Select ... For Update to sequence operations properly. In practice that means when running - * with Derby database. See more notes at class level. - */ - private void lockInternal() { - if(dbType.isDERBY()) { - derbyLock.lock(); - } - } - - private void unlockInternal() { - if(dbType.isDERBY()) { - derbyLock.unlock(); - } - } - - void rollbackDBConn(Connection dbConn) { - try { - if (dbConn != null && !dbConn.isClosed()) dbConn.rollback(); - } catch (SQLException e) { - LOG.warn("Failed to rollback db connection ", e); - } - } - - void closeDbConn(JDOConnection jdoConn) { - try { - if (jdoConn != null) { - jdoConn.close(); - } - } catch (Exception e) { - LOG.warn("Failed to close db connection", e); - } - } - - void closeStmt(Statement stmt) { - try { - if (stmt != null && !stmt.isClosed()) stmt.close(); - } catch (SQLException e) { - LOG.warn("Failed to close statement ", e); - } - } - - void close(ResultSet rs) { - try { - if (rs != null && !rs.isClosed()) { - rs.close(); - } - } - catch(SQLException ex) { - LOG.warn("Failed to close statement ", ex); - } - } - - static String quoteString(String input) { - return "'" + input + "'"; - } - - void close(ResultSet rs, Statement stmt, JDOConnection dbConn) { - close(rs); - closeStmt(stmt); - closeDbConn(dbConn); - } - - private void populateInsertUpdateMap(Map statsPartInfoMap, - Map updateMap, - MapinsertMap, - Connection dbConn, Table tbl) throws SQLException, MetaException, NoSuchObjectException { - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - Statement statement = null; - ResultSet rs = null; - List queries = new ArrayList<>(); - Set selectedParts = new HashSet<>(); - - List partIdList = statsPartInfoMap.keySet().stream().map( - e -> e.partitionId).collect(Collectors.toList() - ); - - prefix.append("select \"PART_ID\", \"COLUMN_NAME\" from \"PART_COL_STATS\" WHERE "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, - partIdList, "\"PART_ID\"", true, false); - - for (String query : queries) { - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute query " + query); - rs = statement.executeQuery(query); - while (rs.next()) { - selectedParts.add(new PartColNameInfo(rs.getLong(1), rs.getString(2))); - } - } finally { - close(rs, statement, null); - } - } - - for (Map.Entry entry : statsPartInfoMap.entrySet()) { - PartitionInfo partitionInfo = (PartitionInfo) entry.getKey(); - ColumnStatistics colStats = (ColumnStatistics) entry.getValue(); - long partId = partitionInfo.partitionId; - ColumnStatisticsDesc statsDesc = colStats.getStatsDesc(); - if (!statsDesc.isSetCatName()) { - statsDesc.setCatName(tbl.getCatName()); - } - for (ColumnStatisticsObj statisticsObj : colStats.getStatsObj()) { - PartColNameInfo temp = new PartColNameInfo(partId, statisticsObj.getColName()); - if (selectedParts.contains(temp)) { - updateMap.put(temp, StatObjectConverter. - convertToMPartitionColumnStatistics(null, statsDesc, statisticsObj, colStats.getEngine())); - } else { - insertMap.put(temp, StatObjectConverter. - convertToMPartitionColumnStatistics(null, statsDesc, statisticsObj, colStats.getEngine())); - } - } - } - } - - private void updatePartColStatTable(Map updateMap, - Connection dbConn) throws SQLException, MetaException, NoSuchObjectException { - PreparedStatement pst = null; - for (Map.Entry entry : updateMap.entrySet()) { - PartColNameInfo partColNameInfo = (PartColNameInfo) entry.getKey(); - Long partId = partColNameInfo.partitionId; - MPartitionColumnStatistics mPartitionColumnStatistics = (MPartitionColumnStatistics) entry.getValue(); - String update = "UPDATE \"PART_COL_STATS\" SET "; - update += StatObjectConverter.getUpdatedColumnSql(mPartitionColumnStatistics); - update += " WHERE \"PART_ID\" = " + partId + " AND " - + " \"COLUMN_NAME\" = " + quoteString(mPartitionColumnStatistics.getColName()); - try { - pst = dbConn.prepareStatement(update); - StatObjectConverter.initUpdatedColumnStatement(mPartitionColumnStatistics, pst); - LOG.debug("Going to execute update " + update); - int numUpdate = pst.executeUpdate(); - if (numUpdate != 1) { - throw new MetaException("Invalid state of PART_COL_STATS for PART_ID " + partId); - } - } finally { - closeStmt(pst); - } - } - } - - private void insertIntoPartColStatTable(Map insertMap, - long maxCsId, - Connection dbConn) throws SQLException, MetaException, NoSuchObjectException { - PreparedStatement preparedStatement = null; - int numRows = 0; - String insert = "INSERT INTO \"PART_COL_STATS\" (\"CS_ID\", \"CAT_NAME\", \"DB_NAME\"," - + "\"TABLE_NAME\", \"PARTITION_NAME\", \"COLUMN_NAME\", \"COLUMN_TYPE\", \"PART_ID\"," - + " \"LONG_LOW_VALUE\", \"LONG_HIGH_VALUE\", \"DOUBLE_HIGH_VALUE\", \"DOUBLE_LOW_VALUE\"," - + " \"BIG_DECIMAL_LOW_VALUE\", \"BIG_DECIMAL_HIGH_VALUE\", \"NUM_NULLS\", \"NUM_DISTINCTS\", \"BIT_VECTOR\" ," - + " \"HISTOGRAM\", \"AVG_COL_LEN\", \"MAX_COL_LEN\", \"NUM_TRUES\", \"NUM_FALSES\", \"LAST_ANALYZED\", \"ENGINE\") values " - + "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; - - try { - preparedStatement = dbConn.prepareStatement(insert); - for (Map.Entry entry : insertMap.entrySet()) { - PartColNameInfo partColNameInfo = (PartColNameInfo) entry.getKey(); - Long partId = partColNameInfo.partitionId; - MPartitionColumnStatistics mPartitionColumnStatistics = (MPartitionColumnStatistics) entry.getValue(); - - preparedStatement.setLong(1, maxCsId); - preparedStatement.setString(2, mPartitionColumnStatistics.getCatName()); - preparedStatement.setString(3, mPartitionColumnStatistics.getDbName()); - preparedStatement.setString(4, mPartitionColumnStatistics.getTableName()); - preparedStatement.setString(5, mPartitionColumnStatistics.getPartitionName()); - preparedStatement.setString(6, mPartitionColumnStatistics.getColName()); - preparedStatement.setString(7, mPartitionColumnStatistics.getColType()); - preparedStatement.setLong(8, partId); - preparedStatement.setObject(9, mPartitionColumnStatistics.getLongLowValue()); - preparedStatement.setObject(10, mPartitionColumnStatistics.getLongHighValue()); - preparedStatement.setObject(11, mPartitionColumnStatistics.getDoubleHighValue()); - preparedStatement.setObject(12, mPartitionColumnStatistics.getDoubleLowValue()); - preparedStatement.setString(13, mPartitionColumnStatistics.getDecimalLowValue()); - preparedStatement.setString(14, mPartitionColumnStatistics.getDecimalHighValue()); - preparedStatement.setObject(15, mPartitionColumnStatistics.getNumNulls()); - preparedStatement.setObject(16, mPartitionColumnStatistics.getNumDVs()); - preparedStatement.setObject(17, mPartitionColumnStatistics.getBitVector()); - preparedStatement.setBytes(18, mPartitionColumnStatistics.getHistogram()); - preparedStatement.setObject(19, mPartitionColumnStatistics.getAvgColLen()); - preparedStatement.setObject(20, mPartitionColumnStatistics.getMaxColLen()); - preparedStatement.setObject(21, mPartitionColumnStatistics.getNumTrues()); - preparedStatement.setObject(22, mPartitionColumnStatistics.getNumFalses()); - preparedStatement.setLong(23, mPartitionColumnStatistics.getLastAnalyzed()); - preparedStatement.setString(24, mPartitionColumnStatistics.getEngine()); - - maxCsId++; - numRows++; - preparedStatement.addBatch(); - if (numRows == maxBatchSize) { - preparedStatement.executeBatch(); - numRows = 0; - } - } - - if (numRows != 0) { - preparedStatement.executeBatch(); - } - } finally { - closeStmt(preparedStatement); - } - } - - private Map getParamValues(Connection dbConn, List partIdList) throws SQLException { - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - Statement statement = null; - ResultSet rs = null; - - prefix.append("select \"PART_ID\", \"PARAM_VALUE\" " - + " from \"PARTITION_PARAMS\" where " - + " \"PARAM_KEY\" = 'COLUMN_STATS_ACCURATE' " - + " and "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, - partIdList, "\"PART_ID\"", true, false); - - Map partIdToParaMap = new HashMap<>(); - for (String query : queries) { - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute query " + query); - rs = statement.executeQuery(query); - while (rs.next()) { - partIdToParaMap.put(rs.getLong(1), rs.getString(2)); - } - } finally { - close(rs, statement, null); - } - } - return partIdToParaMap; - } - - private void updateWriteIdForPartitions(Connection dbConn, long writeId, List partIdList) throws SQLException { - StringBuilder prefix = new StringBuilder(); - List queries = new ArrayList<>(); - StringBuilder suffix = new StringBuilder(); - - prefix.append("UPDATE \"PARTITIONS\" set \"WRITE_ID\" = " + writeId + " where "); - TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, - partIdList, "\"PART_ID\"", false, false); - - Statement statement = null; - for (String query : queries) { - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute update " + query); - statement.executeUpdate(query); - } finally { - closeStmt(statement); - } - } - } - - private Map> updatePartitionParamTable(Connection dbConn, - Map partitionInfoMap, - String validWriteIds, - long writeId, - boolean isAcidTable) - throws SQLException, MetaException { - Map> result = new HashMap<>(); - boolean areTxnStatsSupported = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_TXN_STATS_ENABLED); - PreparedStatement statementInsert = null; - PreparedStatement statementDelete = null; - PreparedStatement statementUpdate = null; - String insert = "INSERT INTO \"PARTITION_PARAMS\" (\"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\") " - + "VALUES( ? , 'COLUMN_STATS_ACCURATE' , ? )"; - String delete = "DELETE from \"PARTITION_PARAMS\" " - + " where \"PART_ID\" = ? " - + " and \"PARAM_KEY\" = 'COLUMN_STATS_ACCURATE'"; - String update = "UPDATE \"PARTITION_PARAMS\" set \"PARAM_VALUE\" = ? " - + " where \"PART_ID\" = ? " - + " and \"PARAM_KEY\" = 'COLUMN_STATS_ACCURATE'"; - int numInsert = 0; - int numDelete = 0; - int numUpdate = 0; - - List partIdList = partitionInfoMap.keySet().stream().map( - e -> e.partitionId).collect(Collectors.toList() - ); - - // get the old parameters from PARTITION_PARAMS table. - Map partIdToParaMap = getParamValues(dbConn, partIdList); - - try { - statementInsert = dbConn.prepareStatement(insert); - statementDelete = dbConn.prepareStatement(delete); - statementUpdate = dbConn.prepareStatement(update); - for (Map.Entry entry : partitionInfoMap.entrySet()) { - PartitionInfo partitionInfo = (PartitionInfo) entry.getKey(); - ColumnStatistics colStats = (ColumnStatistics) entry.getValue(); - List colNames = colStats.getStatsObj().stream().map(e -> e.getColName()).collect(Collectors.toList()); - long partWriteId = partitionInfo.writeId; - long partId = partitionInfo.partitionId; - Map newParameter; - - if (!partIdToParaMap.containsKey(partId)) { - newParameter = new HashMap<>(); - newParameter.put(COLUMN_STATS_ACCURATE, "TRUE"); - StatsSetupConst.setColumnStatsState(newParameter, colNames); - statementInsert.setLong(1, partId); - statementInsert.setString(2, newParameter.get(COLUMN_STATS_ACCURATE)); - numInsert++; - statementInsert.addBatch(); - if (numInsert == maxBatchSize) { - LOG.debug(" Executing insert " + insert); - statementInsert.executeBatch(); - numInsert = 0; - } - } else { - String oldStats = partIdToParaMap.get(partId); - - Map oldParameter = new HashMap<>(); - oldParameter.put(COLUMN_STATS_ACCURATE, oldStats); - - newParameter = new HashMap<>(); - newParameter.put(COLUMN_STATS_ACCURATE, oldStats); - StatsSetupConst.setColumnStatsState(newParameter, colNames); - - if (isAcidTable) { - String errorMsg = ObjectStore.verifyStatsChangeCtx( - colStats.getStatsDesc().getDbName() + "." + colStats.getStatsDesc().getTableName(), - oldParameter, newParameter, writeId, validWriteIds, true); - if (errorMsg != null) { - throw new MetaException(errorMsg); - } - } - - if (isAcidTable && - (!areTxnStatsSupported || !ObjectStore.isCurrentStatsValidForTheQuery(oldParameter, partWriteId, - validWriteIds, true))) { - statementDelete.setLong(1, partId); - statementDelete.addBatch(); - numDelete++; - if (numDelete == maxBatchSize) { - statementDelete.executeBatch(); - numDelete = 0; - LOG.debug("Removed COLUMN_STATS_ACCURATE from the parameters of the partition " - + colStats.getStatsDesc().getDbName() + "." + colStats.getStatsDesc().getTableName() + "." - + colStats.getStatsDesc().getPartName()); - } - } else { - statementUpdate.setString(1, newParameter.get(COLUMN_STATS_ACCURATE)); - statementUpdate.setLong(2, partId); - statementUpdate.addBatch(); - numUpdate++; - if (numUpdate == maxBatchSize) { - LOG.debug(" Executing update " + statementUpdate); - statementUpdate.executeBatch(); - numUpdate = 0; - } - } - } - result.put(partitionInfo.partitionName, newParameter); - } - - if (numInsert != 0) { - statementInsert.executeBatch(); - } - - if (numUpdate != 0) { - statementUpdate.executeBatch(); - } - - if (numDelete != 0) { - statementDelete.executeBatch(); - } - - if (isAcidTable) { - updateWriteIdForPartitions(dbConn, writeId, partIdList); - } - return result; - } finally { - closeStmt(statementInsert); - closeStmt(statementUpdate); - closeStmt(statementDelete); - } - } - - private static class PartitionInfo { - long partitionId; - long writeId; - String partitionName; - public PartitionInfo(long partitionId, long writeId, String partitionName) { - this.partitionId = partitionId; - this.writeId = writeId; - this.partitionName = partitionName; - } - - @Override - public int hashCode() { - return (int)partitionId; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null) { - return false; - } - if (!(o instanceof PartitionInfo)) { - return false; - } - PartitionInfo other = (PartitionInfo)o; - if (this.partitionId != other.partitionId) { - return false; - } - return true; - } - } - - private static class PartColNameInfo { - long partitionId; - String colName; - public PartColNameInfo(long partitionId, String colName) { - this.partitionId = partitionId; - this.colName = colName; - } - - @Override - public int hashCode() { - return (int)partitionId; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null) { - return false; - } - if (!(o instanceof PartColNameInfo)) { - return false; - } - PartColNameInfo other = (PartColNameInfo)o; - if (this.partitionId != other.partitionId) { - return false; - } - if (this.colName.equalsIgnoreCase(other.colName)) { - return true; - } - return false; - } - } - - private Map getPartitionInfo(Connection dbConn, long tblId, - Map partColStatsMap) - throws SQLException, MetaException { - List queries = new ArrayList<>(); - StringBuilder prefix = new StringBuilder(); - StringBuilder suffix = new StringBuilder(); - Statement statement = null; - ResultSet rs = null; - Map partitionInfoMap = new HashMap<>(); - - List partKeys = partColStatsMap.keySet().stream().map( - e -> quoteString(e)).collect(Collectors.toList() - ); - - prefix.append("select \"PART_ID\", \"WRITE_ID\", \"PART_NAME\" from \"PARTITIONS\" where "); - suffix.append(" and \"TBL_ID\" = " + tblId); - TxnUtils.buildQueryWithINClauseStrings(conf, queries, prefix, suffix, - partKeys, "\"PART_NAME\"", true, false); - - for (String query : queries) { - // Select for update makes sure that the partitions are not modified while the stats are getting updated. - query = sqlGenerator.addForUpdateClause(query); - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute query <" + query + ">"); - rs = statement.executeQuery(query); - while (rs.next()) { - PartitionInfo partitionInfo = new PartitionInfo(rs.getLong(1), - rs.getLong(2), rs.getString(3)); - partitionInfoMap.put(partitionInfo, partColStatsMap.get(rs.getString(3))); - } - } finally { - close(rs, statement, null); - } - } - return partitionInfoMap; - } - - private void setAnsiQuotes(Connection dbConn) throws SQLException { - if (sqlGenerator.getDbProduct().isMYSQL()) { - try (Statement stmt = dbConn.createStatement()) { - stmt.execute("SET @@session.sql_mode=ANSI_QUOTES"); - } - } - } - - /** - * Update the statistics for the given partitions. Add the notification logs also. - * @return map of partition key to column stats if successful, null otherwise. - */ - public Map> updatePartitionColumnStatistics(Map partColStatsMap, - Table tbl, long csId, - String validWriteIds, long writeId, - List transactionalListeners) - throws MetaException { - JDOConnection jdoConn = null; - Connection dbConn = null; - boolean committed = false; - try { - lockInternal(); - jdoConn = pm.getDataStoreConnection(); - dbConn = (Connection) (jdoConn.getNativeConnection()); - - setAnsiQuotes(dbConn); - - Map partitionInfoMap = getPartitionInfo(dbConn, tbl.getId(), partColStatsMap); - - Map> result = - updatePartitionParamTable(dbConn, partitionInfoMap, validWriteIds, writeId, TxnUtils.isAcidTable(tbl)); - - Map insertMap = new HashMap<>(); - Map updateMap = new HashMap<>(); - populateInsertUpdateMap(partitionInfoMap, updateMap, insertMap, dbConn, tbl); - - LOG.info("Number of stats to insert " + insertMap.size() + " update " + updateMap.size()); - - if (insertMap.size() != 0) { - insertIntoPartColStatTable(insertMap, csId, dbConn); - } - - if (updateMap.size() != 0) { - updatePartColStatTable(updateMap, dbConn); - } - - if (transactionalListeners != null) { - UpdatePartitionColumnStatEventBatch eventBatch = new UpdatePartitionColumnStatEventBatch(null); - for (Map.Entry entry : result.entrySet()) { - Map parameters = (Map) entry.getValue(); - ColumnStatistics colStats = partColStatsMap.get(entry.getKey()); - List partVals = getPartValsFromName(tbl, colStats.getStatsDesc().getPartName()); - UpdatePartitionColumnStatEvent event = new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, - tbl, writeId, null); - eventBatch.addPartColStatEvent(event); - } - MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, - EventMessage.EventType.UPDATE_PARTITION_COLUMN_STAT_BATCH, eventBatch, dbConn, sqlGenerator); - } - dbConn.commit(); - committed = true; - return result; - } catch (Exception e) { - LOG.error("Unable to update Column stats for " + tbl.getTableName(), e); - throw new MetaException("Unable to update Column stats for " + tbl.getTableName() - + " due to: " + e.getMessage()); - } finally { - if (!committed) { - rollbackDBConn(dbConn); - } - closeDbConn(jdoConn); - unlockInternal(); - } - } - - /** - * Gets the next CS id from sequence MPartitionColumnStatistics and increment the CS id by numStats. - * @return The CD id before update. - */ - public long getNextCSIdForMPartitionColumnStatistics(long numStats) throws MetaException { - Statement statement = null; - ResultSet rs = null; - long maxCsId = 0; - boolean committed = false; - Connection dbConn = null; - JDOConnection jdoConn = null; - - try { - lockInternal(); - jdoConn = pm.getDataStoreConnection(); - dbConn = (Connection) (jdoConn.getNativeConnection()); - - setAnsiQuotes(dbConn); - - // This loop will be iterated at max twice. If there is no records, it will first insert and then do a select. - // We are not using any upsert operations as select for update and then update is required to make sure that - // the caller gets a reserved range for CSId not used by any other thread. - boolean insertDone = false; - while (maxCsId == 0) { - String query = sqlGenerator.addForUpdateClause("SELECT \"NEXT_VAL\" FROM \"SEQUENCE_TABLE\" " - + "WHERE \"SEQUENCE_NAME\"= " - + quoteString("org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics")); - LOG.debug("Going to execute query " + query); - statement = dbConn.createStatement(); - rs = statement.executeQuery(query); - if (rs.next()) { - maxCsId = rs.getLong(1); - } else if (insertDone) { - throw new MetaException("Invalid state of SEQUENCE_TABLE for MPartitionColumnStatistics"); - } else { - insertDone = true; - closeStmt(statement); - statement = dbConn.createStatement(); - query = "INSERT INTO \"SEQUENCE_TABLE\" (\"SEQUENCE_NAME\", \"NEXT_VAL\") VALUES ( " - + quoteString("org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics") + "," + 1 - + ")"; - try { - statement.executeUpdate(query); - } catch (SQLException e) { - // If the record is already inserted by some other thread continue to select. - if (dbType.isDuplicateKeyError(e)) { - continue; - } - LOG.error("Unable to insert into SEQUENCE_TABLE for MPartitionColumnStatistics.", e); - throw e; - } finally { - closeStmt(statement); - } - } - } - - long nextMaxCsId = maxCsId + numStats + 1; - closeStmt(statement); - statement = dbConn.createStatement(); - String query = "UPDATE \"SEQUENCE_TABLE\" SET \"NEXT_VAL\" = " - + nextMaxCsId - + " WHERE \"SEQUENCE_NAME\" = " - + quoteString("org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics"); - statement.executeUpdate(query); - dbConn.commit(); - committed = true; - return maxCsId; - } catch (Exception e) { - LOG.error("Unable to getNextCSIdForMPartitionColumnStatistics", e); - throw new MetaException("Unable to getNextCSIdForMPartitionColumnStatistics " - + " due to: " + e.getMessage()); - } finally { - if (!committed) { - rollbackDBConn(dbConn); - } - close(rs, statement, jdoConn); - unlockInternal(); - } - } -} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index 3ed850efbabe..7100bf93ae15 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -5791,9 +5791,8 @@ public void alter_partition_with_environment_context(final String dbName, final EnvironmentContext envContext) throws TException { String[] parsedDbName = parseDbName(dbName, conf); - // TODO: this method name is confusing, it actually does full alter (sortof) - rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, null, newPartition, - envContext, null); + alter_partition_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, null, + newPartition, envContext, null); } @Deprecated @@ -5801,9 +5800,9 @@ public void alter_partition_with_environment_context(final String dbName, public void rename_partition(final String db_name, final String tbl_name, final List part_vals, final Partition new_part) throws TException { - // Call rename_partition without an environment context. + // Call alter_partition_core without an environment context. String[] parsedDbName = parseDbName(db_name, conf); - rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals, new_part, + alter_partition_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals, new_part, null, null); } @@ -5813,12 +5812,12 @@ public RenamePartitionResponse rename_partition_req(RenamePartitionRequest req) context.putToProperties(RENAME_PARTITION_MAKE_COPY, String.valueOf(req.isClonePart())); context.putToProperties(hive_metastoreConstants.TXN_ID, String.valueOf(req.getTxnId())); - rename_partition(req.getCatName(), req.getDbName(), req.getTableName(), req.getPartVals(), + alter_partition_core(req.getCatName(), req.getDbName(), req.getTableName(), req.getPartVals(), req.getNewPart(), context, req.getValidWriteIdList()); return new RenamePartitionResponse(); }; - private void rename_partition(String catName, String db_name, String tbl_name, + private void alter_partition_core(String catName, String db_name, String tbl_name, List part_vals, Partition new_part, EnvironmentContext envContext, String validWriteIds) throws TException { startTableFunction("alter_partition", catName, db_name, tbl_name); @@ -5847,8 +5846,7 @@ private void rename_partition(String catName, String db_name, String tbl_name, Partition oldPart = null; Exception ex = null; try { - Table table = null; - table = getMS().getTable(catName, db_name, tbl_name, null); + Table table = getMS().getTable(catName, db_name, tbl_name, null); firePreEvent(new PreAlterPartitionEvent(db_name, tbl_name, table, part_vals, new_part, this)); if (part_vals != null && !part_vals.isEmpty()) { @@ -5859,8 +5857,6 @@ private void rename_partition(String catName, String db_name, String tbl_name, oldPart = alterHandler.alterPartition(getMS(), wh, catName, db_name, tbl_name, part_vals, new_part, envContext, this, validWriteIds); - // Only fetch the table if we actually have a listener - if (!listeners.isEmpty()) { MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_PARTITION, diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java index 97956660791a..d5e0db5366c1 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java @@ -95,13 +95,13 @@ import org.apache.hadoop.hive.metastore.model.MConstraint; import org.apache.hadoop.hive.metastore.model.MCreationMetadata; import org.apache.hadoop.hive.metastore.model.MDatabase; -import org.apache.hadoop.hive.metastore.model.MFunction; import org.apache.hadoop.hive.metastore.model.MNotificationLog; import org.apache.hadoop.hive.metastore.model.MNotificationNextId; import org.apache.hadoop.hive.metastore.model.MPartition; import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege; import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics; import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege; +import org.apache.hadoop.hive.metastore.model.MTable; import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics; import org.apache.hadoop.hive.metastore.model.MWMResourcePlan; import org.apache.hadoop.hive.metastore.parser.ExpressionTree; @@ -111,6 +111,7 @@ import org.apache.hadoop.hive.metastore.parser.ExpressionTree.Operator; import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeNode; import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeVisitor; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; @@ -152,6 +153,7 @@ class MetaStoreDirectSql { private final int batchSize; private final boolean convertMapNullsToEmptyStrings; private final String defaultPartName; + private final boolean isTxnStatsEnabled; /** * Whether direct SQL can be used with the current datastore backing {@link #pm}. @@ -160,7 +162,7 @@ class MetaStoreDirectSql { private final boolean isAggregateStatsCacheEnabled; private final ImmutableMap fieldnameToTableName; private AggregateStatsCache aggrStatsCache; - private DirectSqlUpdateStat updateStat; + private DirectSqlUpdatePart directSqlUpdatePart; private DirectSqlInsertPart directSqlInsertPart; /** @@ -203,7 +205,8 @@ public MetaStoreDirectSql(PersistenceManager pm, Configuration conf, String sche batchSize = dbType.needsInBatching() ? 1000 : NO_BATCHING; } this.batchSize = batchSize; - this.updateStat = new DirectSqlUpdateStat(pm, conf, dbType, batchSize); + this.isTxnStatsEnabled = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_TXN_STATS_ENABLED); + this.directSqlUpdatePart = new DirectSqlUpdatePart(pm, conf, dbType, batchSize); ImmutableMap.Builder fieldNameToTableNameBuilder = new ImmutableMap.Builder<>(); @@ -535,6 +538,69 @@ public void addPartitions(List parts, List directSqlInsertPart.addPartitions(parts, partPrivilegesList, partColPrivilegesList); } + /** + * Alter partitions in batch using direct SQL + * @param table the target table + * @param partNames list of partition names + * @param newParts list of new partitions + * @param queryWriteIdList valid write id list + * @return + * @throws MetaException + */ + public List alterPartitions(MTable table, List partNames, + List newParts, String queryWriteIdList) throws MetaException { + List rows = Batchable.runBatched(batchSize, partNames, new Batchable() { + @Override + public List run(List input) throws Exception { + String filter = "" + PARTITIONS + ".\"PART_NAME\" in (" + makeParams(input.size()) + ")"; + List columns = Arrays.asList("\"PART_ID\"", "\"PART_NAME\"", "\"SD_ID\"", "\"WRITE_ID\""); + return getPartitionFieldsViaSqlFilter(table.getDatabase().getCatalogName(), table.getDatabase().getName(), + table.getTableName(), columns, filter, input, Collections.emptyList(), null); + } + }); + Map, Long> partValuesToId = new HashMap<>(); + Map partIdToSdId = new HashMap<>(); + Map partIdToWriteId = new HashMap<>(); + for (Object[] row : rows) { + Long partId = MetastoreDirectSqlUtils.extractSqlLong(row[0]); + Long sdId = MetastoreDirectSqlUtils.extractSqlLong(row[2]); + Long writeId = MetastoreDirectSqlUtils.extractSqlLong(row[3]); + partIdToSdId.put(partId, sdId); + partIdToWriteId.put(partId, writeId); + List partValues = Warehouse.getPartValuesFromPartName((String) row[1]); + partValuesToId.put(partValues, partId); + } + + boolean isTxn = TxnUtils.isTransactionalTable(table.getParameters()); + for (Partition newPart : newParts) { + Long partId = partValuesToId.get(newPart.getValues()); + boolean useOldWriteId = true; + // If transactional, add/update the MUPdaterTransaction + // for the current updater query. + if (isTxn) { + if (!isTxnStatsEnabled) { + StatsSetupConst.setBasicStatsState(newPart.getParameters(), StatsSetupConst.FALSE); + } else if (queryWriteIdList != null && newPart.getWriteId() > 0) { + // Check concurrent INSERT case and set false to the flag. + if (!ObjectStore.isCurrentStatsValidForTheQuery(newPart.getParameters(), + partIdToWriteId.get(partId), queryWriteIdList, true)) { + StatsSetupConst.setBasicStatsState(newPart.getParameters(), StatsSetupConst.FALSE); + LOG.info("Removed COLUMN_STATS_ACCURATE from the parameters of the partition " + + Warehouse.getQualifiedName(newPart) + " will be made persistent."); + } + useOldWriteId = false; + } + } + + if (useOldWriteId) { + newPart.setWriteId(partIdToWriteId.get(partId)); + } + } + + directSqlUpdatePart.alterPartitions(partValuesToId, partIdToSdId, newParts); + return newParts; + } + /** * Get partition names by using direct SQL queries. * @param filter filter to use with direct sql @@ -901,6 +967,28 @@ private List getPartitionIdsViaSqlFilter( String catName, String dbName, String tblName, String sqlFilter, List paramsForFilter, List joinsForFilter, Integer max) throws MetaException { + return getPartitionFieldsViaSqlFilter(catName, dbName, tblName, + Arrays.asList("\"PART_ID\""), sqlFilter, paramsForFilter, joinsForFilter, max); + } + + /** + * Get partition fields for the query using direct SQL queries, to avoid bazillion + * queries created by DN retrieving stuff for each object individually. + * @param catName MetaStore catalog name + * @param dbName MetaStore db name + * @param tblName MetaStore table name + * @param partColumns part fields want to get + * @param sqlFilter SQL filter to use. Better be SQL92-compliant. + * @param paramsForFilter params for ?-s in SQL filter text. Params must be in order. + * @param joinsForFilter if the filter needs additional join statement, they must be in + * this list. Better be SQL92-compliant. + * @param max The maximum number of partitions to return. + * @return List of partition objects. + */ + public List getPartitionFieldsViaSqlFilter( + String catName, String dbName, String tblName, List partColumns, String sqlFilter, + List paramsForFilter, List joinsForFilter, Integer max) + throws MetaException { boolean doTrace = LOG.isDebugEnabled(); final String dbNameLcase = dbName.toLowerCase(); final String tblNameLcase = tblName.toLowerCase(); @@ -908,16 +996,17 @@ private List getPartitionIdsViaSqlFilter( // We have to be mindful of order during filtering if we are not returning all partitions. String orderForFilter = (max != null) ? " order by " + MetastoreConf.getVar(conf, ConfVars.PARTITION_ORDER_EXPR) : ""; + String columns = partColumns.stream().map(col -> PARTITIONS + "." + col).collect(Collectors.joining(",")); String queryText = - "select " + PARTITIONS + ".\"PART_ID\" from " + PARTITIONS + "" - + " inner join " + TBLS + " on " + PARTITIONS + ".\"TBL_ID\" = " + TBLS + ".\"TBL_ID\" " - + " and " + TBLS + ".\"TBL_NAME\" = ? " - + " inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" " - + " and " + DBS + ".\"NAME\" = ? " - + join(joinsForFilter, ' ') - + " where " + DBS + ".\"CTLG_NAME\" = ? " - + (StringUtils.isBlank(sqlFilter) ? "" : (" and " + sqlFilter)) + orderForFilter; + "select " + columns + " from " + PARTITIONS + "" + + " inner join " + TBLS + " on " + PARTITIONS + ".\"TBL_ID\" = " + TBLS + ".\"TBL_ID\" " + + " and " + TBLS + ".\"TBL_NAME\" = ? " + + " inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" " + + " and " + DBS + ".\"NAME\" = ? " + + join(joinsForFilter, ' ') + + " where " + DBS + ".\"CTLG_NAME\" = ? " + + (StringUtils.isBlank(sqlFilter) ? "" : (" and " + sqlFilter)) + orderForFilter; Object[] params = new Object[paramsForFilter.size() + 3]; params[0] = tblNameLcase; params[1] = dbNameLcase; @@ -928,19 +1017,11 @@ private List getPartitionIdsViaSqlFilter( long start = doTrace ? System.nanoTime() : 0; try (QueryWrapper query = new QueryWrapper(pm.newQuery("javax.jdo.query.SQL", queryText))) { - List sqlResult = executeWithArray(query.getInnerQuery(), params, queryText, + List sqlResult = executeWithArray(query.getInnerQuery(), params, queryText, ((max == null) ? -1 : max.intValue())); long queryTime = doTrace ? System.nanoTime() : 0; MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, queryTime); - final List result; - if (sqlResult.isEmpty()) { - result = Collections.emptyList(); // no partitions, bail early. - } else { - result = new ArrayList<>(sqlResult.size()); - for (Object fields : sqlResult) { - result.add(MetastoreDirectSqlUtils.extractSqlLong(fields)); - } - } + List result = new ArrayList<>(sqlResult); return result; } } @@ -3056,8 +3137,8 @@ public Map> updatePartitionColumnStatisticsBatch( ColumnStatistics colStats = (ColumnStatistics) entry.getValue(); numStats += colStats.getStatsObjSize(); } - long csId = updateStat.getNextCSIdForMPartitionColumnStatistics(numStats); - return updateStat.updatePartitionColumnStatistics(partColStatsMap, tbl, csId, validWriteIds, writeId, listeners); + long csId = directSqlUpdatePart.getNextCSIdForMPartitionColumnStatistics(numStats); + return directSqlUpdatePart.updatePartitionColumnStatistics(partColStatsMap, tbl, csId, validWriteIds, writeId, listeners); } public List getFunctions(String catName) throws MetaException { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java index 0ae023bdd7e3..de88e482b716 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java @@ -4399,7 +4399,7 @@ protected boolean canUseDirectSql(GetHelper ctx) throws MetaException { protected abstract String describeResult(); protected abstract T getSqlResult(GetHelper ctx) throws MetaException; protected abstract T getJdoResult( - GetHelper ctx) throws MetaException, NoSuchObjectException; + GetHelper ctx) throws MetaException, NoSuchObjectException, InvalidObjectException; public T run(boolean initTable) throws MetaException, NoSuchObjectException { try { @@ -5261,91 +5261,114 @@ public List alterPartitions(String catName, String dbName, String tbl List> part_vals, List newParts, long writeId, String queryWriteIdList) throws InvalidObjectException, MetaException { - boolean success = false; - Exception e = null; List results = new ArrayList<>(newParts.size()); if (newParts.isEmpty()) { return results; } + catName = normalizeIdentifier(catName); + dbName = normalizeIdentifier(dbName); + tblName = normalizeIdentifier(tblName); + + boolean success = false; try { openTransaction(); - MTable table = this.getMTable(catName, dbName, tblName); - if (table == null) { - throw new NoSuchObjectException( - TableName.getQualified(catName, dbName, tblName) + " table not found"); + MTable table = ensureGetMTable(catName, dbName, tblName); + // Validate new parts: StorageDescriptor and SerDeInfo must be set in Partition. + if (!TableType.VIRTUAL_VIEW.name().equals(table.getTableType())) { + for (Partition newPart : newParts) { + if (!newPart.isSetSd() || !newPart.getSd().isSetSerdeInfo()) { + throw new InvalidObjectException("Partition does not set storageDescriptor or serdeInfo."); + } + } } + if (writeId > 0) { + newParts.forEach(newPart -> newPart.setWriteId(writeId)); + } + + List partCols = convertToFieldSchemas(table.getPartitionKeys()); List partNames = new ArrayList<>(); for (List partVal : part_vals) { - partNames.add( - Warehouse.makePartName(convertToFieldSchemas(table.getPartitionKeys()), partVal) - ); + partNames.add(Warehouse.makePartName(partCols, partVal)); } - catName = normalizeIdentifier(catName); - dbName = normalizeIdentifier(dbName); - tblName = normalizeIdentifier(tblName); - List mPartitionList; - - try (Query query = pm.newQuery(MPartition.class, - "table.tableName == t1 && table.database.name == t2 && t3.contains(partitionName) " + - " && table.database.catalogName == t4")) { - query.declareParameters("java.lang.String t1, java.lang.String t2, java.util.Collection t3, " - + "java.lang.String t4"); - mPartitionList = (List) query.executeWithArray(tblName, dbName, partNames, catName); - pm.retrieveAll(mPartitionList); - - if (mPartitionList.size() > newParts.size()) { - throw new MetaException("Expecting only one partition but more than one partitions are found."); + results = new GetListHelper(catName, dbName, tblName, true, true) { + @Override + protected List getSqlResult(GetHelper> ctx) + throws MetaException { + return directSql.alterPartitions(table, partNames, newParts, queryWriteIdList); } - Map, MPartition> mPartsMap = new HashMap(); - for (MPartition mPartition : mPartitionList) { - mPartsMap.put(mPartition.getValues(), mPartition); + @Override + protected List getJdoResult(GetHelper> ctx) + throws MetaException, InvalidObjectException { + return alterPartitionsViaJdo(table, partNames, newParts, queryWriteIdList); } + }.run(false); - Set oldCds = new HashSet<>(); - Ref oldCdRef = new Ref<>(); - for (Partition tmpPart : newParts) { - if (!tmpPart.getDbName().equalsIgnoreCase(dbName)) { - throw new MetaException("Invalid DB name : " + tmpPart.getDbName()); - } - - if (!tmpPart.getTableName().equalsIgnoreCase(tblName)) { - throw new MetaException("Invalid table name : " + tmpPart.getDbName()); - } - - if (writeId > 0) { - tmpPart.setWriteId(writeId); - } - oldCdRef.t = null; - Partition result = alterPartitionNoTxn(catName, dbName, tblName, mPartsMap.get(tmpPart.getValues()), - tmpPart, queryWriteIdList, oldCdRef, table); - results.add(result); - if (oldCdRef.t != null) { - oldCds.add(oldCdRef.t); - } - } - for (MColumnDescriptor oldCd : oldCds) { - removeUnusedColumnDescriptor(oldCd); - } - } // commit the changes success = commitTransaction(); } catch (Exception exception) { - e = exception; - LOG.error("Alter failed", e); + LOG.error("Alter failed", exception); + throw new MetaException(exception.getMessage()); } finally { if (!success) { rollbackTransaction(); - MetaException metaException = new MetaException( - "The transaction for alter partition did not commit successfully."); - if (e != null) { - metaException.initCause(e); + } + } + return results; + } + + private List alterPartitionsViaJdo(MTable table, List partNames, + List newParts, String queryWriteIdList) + throws MetaException, InvalidObjectException { + String catName = table.getDatabase().getCatalogName(); + String dbName = table.getDatabase().getName(); + String tblName = table.getTableName(); + List results = new ArrayList<>(newParts.size()); + List mPartitionList; + + try (QueryWrapper query = new QueryWrapper(pm.newQuery(MPartition.class, + "table.tableName == t1 && table.database.name == t2 && t3.contains(partitionName) " + + " && table.database.catalogName == t4"))) { + query.declareParameters("java.lang.String t1, java.lang.String t2, java.util.Collection t3, " + + "java.lang.String t4"); + mPartitionList = (List) query.executeWithArray(tblName, dbName, partNames, catName); + pm.retrieveAll(mPartitionList); + + if (mPartitionList.size() > newParts.size()) { + throw new MetaException("Expecting only one partition but more than one partitions are found."); + } + + Map, MPartition> mPartsMap = new HashMap(); + for (MPartition mPartition : mPartitionList) { + mPartsMap.put(mPartition.getValues(), mPartition); + } + + Set oldCds = new HashSet<>(); + Ref oldCdRef = new Ref<>(); + for (Partition tmpPart : newParts) { + if (!tmpPart.getDbName().equalsIgnoreCase(dbName)) { + throw new MetaException("Invalid DB name : " + tmpPart.getDbName()); } - throw metaException; + + if (!tmpPart.getTableName().equalsIgnoreCase(tblName)) { + throw new MetaException("Invalid table name : " + tmpPart.getDbName()); + } + + oldCdRef.t = null; + Partition result = alterPartitionNoTxn(catName, dbName, tblName, + mPartsMap.get(tmpPart.getValues()), tmpPart, queryWriteIdList, oldCdRef, table); + results.add(result); + if (oldCdRef.t != null) { + oldCds.add(oldCdRef.t); + } + } + for (MColumnDescriptor oldCd : oldCds) { + removeUnusedColumnDescriptor(oldCd); } } + return results; } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java index e7e97b5f23de..f490798be56a 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java @@ -680,4 +680,20 @@ public static SQLException getSqlException(Throwable ex) throws IllegalArgumentE return (SQLException)ex; } + public static String createUpdatePreparedStmt(String tableName, List columnNames, List conditionKeys) { + StringBuilder sb = new StringBuilder(); + sb.append("update " + tableName + " set "); + sb.append(columnNames.stream().map(col -> col + "=?").collect(Collectors.joining(","))); + sb.append(" where " + conditionKeys.stream().map(cond -> cond + "=?").collect(Collectors.joining(" and "))); + return sb.toString(); + } + + public static String createInsertPreparedStmt(String tableName, List columnNames) { + StringBuilder sb = new StringBuilder(); + sb.append("insert into " + tableName + "("); + sb.append(columnNames.stream().collect(Collectors.joining(","))); + String placeholder = columnNames.stream().map(col -> "?").collect(Collectors.joining(",")); + sb.append(") values (" + placeholder + ")"); + return sb.toString(); + } } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java index 940b18d1db46..a6e510e5c4db 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java @@ -503,7 +503,7 @@ public void testPartitionOpsWhenTableDoesNotExist() throws InvalidObjectExceptio objectStore.alterPartitions(DEFAULT_CATALOG_NAME, DB1, "not_existed_table", part_vals, parts, 0, ""); } catch (MetaException e) { // expected - Assert.assertTrue(e.getCause() instanceof NoSuchObjectException); + Assert.assertEquals(e.getMessage(), "Specified catalog.database.table does not exist : hive.testobjectstoredb1.not_existed_table"); } } diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java index 90672bf483db..f19576d69405 100644 --- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java +++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java @@ -45,6 +45,7 @@ import java.util.regex.Pattern; import static org.apache.hadoop.hive.metastore.tools.Constants.HMS_DEFAULT_PORT; +import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkAlterPartitions; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkCreatePartition; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkCreatePartitions; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkDeleteCreate; @@ -311,6 +312,8 @@ private void runNonAcidBenchmarks() { () -> benchmarkCreatePartitions(bench, bData, howMany)) .add("dropPartitions" + '.' + howMany, () -> benchmarkDropPartitions(bench, bData, howMany)) + .add("alterPartitions" + '.' + howMany, + () -> benchmarkAlterPartitions(bench, bData, howMany)) .add("renameTable" + '.' + howMany, () -> benchmarkRenameTable(bench, bData, howMany)) .add("dropDatabase" + '.' + howMany, diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java index 214e9e1cd6bb..a2f97eb31704 100644 --- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java +++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.PartitionManagementTask; import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.thrift.TException; @@ -338,6 +339,35 @@ static DescriptiveStatistics benchmarkDropPartitions(@NotNull MicroBenchmark ben } } + static DescriptiveStatistics benchmarkAlterPartitions(@NotNull MicroBenchmark bench, + @NotNull BenchData data, + int count) { + final HMSClient client = data.getClient(); + String dbName = data.dbName; + String tableName = data.tableName; + + BenchmarkUtils.createPartitionedTable(client, dbName, tableName); + try { + return bench.measure( + () -> addManyPartitionsNoException(client, dbName, tableName, null, + Collections.singletonList("d"), count), + () -> throwingSupplierWrapper(() -> { + List newPartitions = client.getPartitions(dbName, tableName); + newPartitions.forEach(p -> { + p.getParameters().put("new_param", "param_val"); + p.getSd().setCols(Arrays.asList(new FieldSchema("new_col", "string", null))); + }); + client.alterPartitions(dbName, tableName, newPartitions); + return null; + }), + () -> throwingSupplierWrapper(() -> + client.dropPartitions(dbName, tableName, null)) + ); + } finally { + throwingSupplierWrapper(() -> client.dropTable(dbName, tableName)); + } + } + static DescriptiveStatistics benchmarkGetPartitionNames(@NotNull MicroBenchmark bench, @NotNull BenchData data, int count) { From 70ee442bdeffcfeba800069b58c4b2ad1062abde Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Tue, 26 Dec 2023 01:50:48 +0800 Subject: [PATCH 116/179] HIVE-27966: Disable flaky testFetchResultsOfLogWithOrientation (#4967). (Wechar, reviewed by Ayush Saxena, Akshat Mathur) --- .../service/cli/operation/TestOperationLoggingAPIWithMr.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithMr.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithMr.java index c7dade3874a9..66325b128c9c 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithMr.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithMr.java @@ -30,6 +30,7 @@ import org.apache.hive.service.cli.RowSet; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; /** @@ -127,6 +128,7 @@ public void testFetchResultsOfLogAsync() throws Exception { } @Test + @Ignore("HIVE-27966") public void testFetchResultsOfLogWithOrientation() throws Exception { // (FETCH_FIRST) execute a sql, and fetch its sql operation log as expected value OperationHandle operationHandle = client.executeStatement(sessionHandle, sql, null); From bd16e0098916aa5fc2dede99492c6a240b51e677 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Tue, 26 Dec 2023 16:07:14 +0800 Subject: [PATCH 117/179] HIVE-27780: Implement direct SQL for get_all_functions - ADDENDUM (#4971). (zhangbutao, reviewed by Ayush Saxena) --- .../org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java index d5e0db5366c1..18bffce36f7d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java @@ -3143,6 +3143,9 @@ public Map> updatePartitionColumnStatisticsBatch( public List getFunctions(String catName) throws MetaException { List funcIds = getFunctionIds(catName); + if (funcIds.isEmpty()) { + return Collections.emptyList(); // no functions, bail early. + } // Get full objects. For Oracle/etc. do it in batches. return Batchable.runBatched(batchSize, funcIds, new Batchable() { @Override From b33b3d3454cc9c65a1879c68679f33f207f21c0e Mon Sep 17 00:00:00 2001 From: kokila-19 <35219892+kokila-19@users.noreply.github.com> Date: Wed, 3 Jan 2024 16:43:24 +0530 Subject: [PATCH 118/179] HIVE-27925: HiveConf: unify ConfVars enum and use underscore for better readability (#4919) (Kokila N reviewed by Laszlo Bodor) --- .../java/org/apache/hive/beeline/BeeLine.java | 2 +- .../org/apache/hive/beeline/BeeLineOpts.java | 4 +- .../apache/hive/beeline/cli/TestHiveCli.java | 2 +- .../org/apache/hadoop/hive/cli/CliDriver.java | 6 +- .../apache/hadoop/hive/common/LogUtils.java | 12 +- .../hadoop/hive/common/ServerUtils.java | 2 +- .../org/apache/hadoop/hive/conf/HiveConf.java | 584 +++++++++--------- .../hive/conf/VariableSubstitution.java | 4 +- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 2 +- .../apache/hadoop/hive/conf/TestHiveConf.java | 24 +- .../hive/conf/TestHiveConfRestrictList.java | 26 +- .../metastore/hooks/SampleURLHook.java | 2 +- .../hive/druid/DruidStorageHandler.java | 2 +- .../druid/io/DruidQueryBasedInputFormat.java | 2 +- .../hive/druid/TestDruidStorageHandler.java | 2 +- .../hadoop/hive/hbase/TestHBaseQueries.java | 2 +- .../hive/hcatalog/common/HCatConstants.java | 2 +- .../hive/hcatalog/common/HiveClientCache.java | 3 +- ...micPartitionFileRecordWriterContainer.java | 4 +- .../FileOutputCommitterContainer.java | 2 +- .../hcatalog/mapreduce/HCatOutputFormat.java | 4 +- .../rcfile/RCFileMapReduceInputFormat.java | 2 +- .../hive/hcatalog/cli/TestPermsGrp.java | 24 +- .../hcatalog/cli/TestSemanticAnalysis.java | 2 +- .../hive/hcatalog/cli/TestUseDatabase.java | 4 +- .../hcatalog/common/TestHiveClientCache.java | 20 +- .../hive/hcatalog/data/HCatDataCheckUtil.java | 4 +- .../hive/hcatalog/mapreduce/HCatBaseTest.java | 10 +- .../mapreduce/TestHCatDynamicPartitioned.java | 3 +- .../mapreduce/TestHCatMultiOutputFormat.java | 22 +- .../mapreduce/TestHCatPartitionPublish.java | 19 +- .../mapreduce/TestPassProperties.java | 6 +- .../TestRCFileMapReduceInputFormat.java | 3 +- .../apache/hive/hcatalog/pig/PigHCatUtil.java | 4 +- .../hive/hcatalog/pig/TestE2EScenarios.java | 6 +- .../pig/TestHCatLoaderComplexSchema.java | 4 +- .../pig/TestHCatLoaderEncryption.java | 6 +- .../hcatalog/pig/TestHCatStorerMulti.java | 8 +- .../hcatalog/messaging/MessageFactory.java | 2 +- .../hive/hcatalog/api/TestHCatClient.java | 20 +- .../hive/hcatalog/templeton/AppConfig.java | 2 +- .../apache/iceberg/hive/CachedClientPool.java | 2 +- .../org/apache/iceberg/hive/HiveCatalog.java | 8 +- .../apache/iceberg/hive/TestHiveCatalog.java | 2 +- .../iceberg/hive/TestHiveClientPool.java | 6 +- .../iceberg/hive/TestHiveMetastore.java | 8 +- .../mr/hive/HiveIcebergOutputCommitter.java | 2 +- .../mr/hive/HiveIcebergOutputFormat.java | 2 +- .../iceberg/mr/hive/HiveIcebergSerDe.java | 4 +- .../mr/hive/HiveIcebergStorageHandler.java | 4 +- .../apache/iceberg/mr/hive/HiveTableUtil.java | 4 +- ...veIcebergStorageHandlerWithEngineBase.java | 4 +- .../iceberg/mr/hive/TestHiveIcebergCRUD.java | 8 +- .../hive/TestHiveIcebergOutputCommitter.java | 2 +- .../TestHiveIcebergStorageHandlerNoScan.java | 2 +- .../apache/iceberg/mr/hive/TestHiveShell.java | 16 +- .../iceberg/mr/hive/TestOptimisticRetry.java | 6 +- .../hive/hcatalog/hbase/ManyMiniCluster.java | 8 +- .../hbase/TestPigHBaseStorageHandler.java | 6 +- .../mapreduce/TestSequenceFileReadWrite.java | 6 +- .../mapjoin/load/AbstractHTLoadBench.java | 4 +- ...egacyVectorMapJoinFastHashTableLoader.java | 4 +- .../minikdc/JdbcWithMiniKdcSQLAuthTest.java | 2 +- .../hive/minikdc/TestHs2HooksWithMiniKdc.java | 6 +- .../minikdc/TestJdbcWithDBTokenStore.java | 4 +- .../TestJdbcWithDBTokenStoreNoDoAs.java | 6 +- .../minikdc/TestJdbcWithMiniKdcCookie.java | 2 +- .../hive/minikdc/TestSSLWithMiniKdc.java | 2 +- ...MetastoreAuthorizationProviderWithACL.java | 5 +- .../org/apache/hadoop/hive/UtilsForTest.java | 2 +- .../hadoop/hive/hooks/TestHs2Hooks.java | 5 +- ...bstractTestAuthorizationApiAuthorizer.java | 6 +- .../metastore/TestMetaStoreAuthorization.java | 2 +- .../hive/metastore/TestMetaStoreMetrics.java | 2 +- .../TestMetaStoreMultipleEncryptionZones.java | 62 +- .../hive/metastore/TestMetastoreVersion.java | 2 +- .../hive/metastore/TestReplChangeManager.java | 16 +- .../metastore/TestServerSpecificConfig.java | 2 +- .../tools/metatool/TestHiveMetaTool.java | 11 +- .../apache/hadoop/hive/ql/TestAcidOnTez.java | 24 +- .../hadoop/hive/ql/TestConstraintsMerge.java | 12 +- ...tDDLWithRemoteMetastoreSecondNamenode.java | 6 +- .../hive/ql/TestDatabaseTableDefault.java | 5 +- .../TestMetaStoreLimitPartitionRequest.java | 3 +- .../hive/ql/history/TestHiveHistory.java | 2 +- .../ql/parse/TestReplWithReadOnlyHook.java | 3 +- .../TestReplicationOnHDFSEncryptedZones.java | 5 +- .../TestReplicationOptimisedBootstrap.java | 47 +- .../ql/parse/TestReplicationScenarios.java | 31 +- ...plicationScenariosAcidTablesBootstrap.java | 3 +- ...stReplicationScenariosAcrossInstances.java | 30 +- ...tReplicationScenariosExclusiveReplica.java | 14 +- ...estReplicationScenariosExternalTables.java | 4 +- ...estReplicationScenariosUsingSnapshots.java | 2 +- .../TestScheduledReplicationScenarios.java | 4 +- .../parse/TestStatsReplicationScenarios.java | 8 +- .../hive/ql/parse/WarehouseInstance.java | 32 +- .../StorageBasedMetastoreTestBase.java | 7 +- .../TestAuthorizationPreEventListener.java | 9 +- .../TestClientSideAuthorizationProvider.java | 10 +- .../TestMetastoreAuthorizationProvider.java | 10 +- ...astoreClientSideAuthorizationProvider.java | 10 +- ...estMultiAuthorizationPreEventListener.java | 3 +- .../TestHiveAuthorizerCheckInvocation.java | 2 +- .../session/TestClearDanglingScratchDir.java | 9 +- .../ql/txn/compactor/CompactorOnTezTest.java | 11 +- .../compactor/TestCleanerWithReplication.java | 4 +- .../ql/txn/compactor/TestCompactorBase.java | 10 +- .../txn/compactor/TestCrudCompactorOnTez.java | 16 +- .../txn/compactor/TestMRCompactorOnTez.java | 12 +- .../txn/compactor/TestMmCompactorOnTez.java | 10 +- .../hive/beeline/TestBeeLineWithArgs.java | 2 +- .../hive/beeline/TestHplSqlViaBeeLine.java | 2 +- .../org/apache/hive/jdbc/TestJdbcDriver2.java | 10 +- .../apache/hive/jdbc/TestJdbcWithMiniHS2.java | 18 +- .../hive/jdbc/cbo_rp_TestJdbcDriver2.java | 3 +- .../hive/service/TestDFSErrorHandling.java | 4 +- .../TestEmbeddedThriftBinaryCLIService.java | 2 +- .../TestHS2ClearDanglingScratchDir.java | 5 +- .../service/server/TestHS2SessionHive.java | 2 +- .../hadoop/hive/ql/QTestMetaStoreHandler.java | 2 +- .../hadoop/hive/ql/QTestMiniClusters.java | 8 +- .../org/apache/hadoop/hive/ql/QTestUtil.java | 7 +- .../hive/ql/hooks/MapJoinCounterHook.java | 2 +- .../hive/ql/stats/DummyStatsAggregator.java | 2 +- .../hive/ql/stats/DummyStatsPublisher.java | 4 +- .../jdbc/miniHS2/AbstractHiveService.java | 6 +- .../org/apache/hive/jdbc/miniHS2/MiniHS2.java | 8 +- .../jdbc/conf/JdbcStorageConfigManager.java | 6 +- .../hadoop/hive/kafka/KafkaOutputFormat.java | 2 +- .../hive/kafka/KafkaStorageHandler.java | 3 +- .../kafka/TransactionalKafkaWriterTest.java | 2 +- .../llap/io/api/impl/LlapRecordReader.java | 4 +- .../llap/tezplugins/LlapTaskCommunicator.java | 4 +- .../tezplugins/LlapTaskSchedulerService.java | 2 +- .../tezplugins/TestLlapTaskCommunicator.java | 2 +- .../hive/metastore/HiveClientCache.java | 2 +- .../org/apache/hadoop/hive/ql/Context.java | 6 +- .../hadoop/hive/ql/DriverTxnHandler.java | 4 +- .../org/apache/hadoop/hive/ql/Executor.java | 6 +- .../org/apache/hadoop/hive/ql/QueryState.java | 12 +- .../database/lock/LockDatabaseAnalyzer.java | 2 +- .../table/AbstractBaseAlterTableAnalyzer.java | 2 +- .../ql/ddl/table/lock/LockTableAnalyzer.java | 2 +- .../ddl/table/partition/PartitionUtils.java | 2 +- .../AlterTableAlterPartitionOperation.java | 2 +- .../partition/show/ShowPartitionAnalyzer.java | 2 +- .../AbstractAlterTableArchiveAnalyzer.java | 2 +- .../archive/AlterTableArchiveOperation.java | 2 +- .../hive/ql/exec/BinaryRecordReader.java | 2 +- .../hive/ql/exec/CommonJoinOperator.java | 6 +- .../hive/ql/exec/CommonMergeJoinOperator.java | 4 +- .../hadoop/hive/ql/exec/ExplainTask.java | 2 +- .../hadoop/hive/ql/exec/FileSinkOperator.java | 2 +- .../hadoop/hive/ql/exec/FilterOperator.java | 4 +- .../hadoop/hive/ql/exec/GroupByOperator.java | 4 +- .../hive/ql/exec/HashTableSinkOperator.java | 4 +- .../hadoop/hive/ql/exec/LimitOperator.java | 6 +- .../hadoop/hive/ql/exec/MapJoinOperator.java | 2 +- .../apache/hadoop/hive/ql/exec/MoveTask.java | 4 +- .../hadoop/hive/ql/exec/PTFPartition.java | 2 +- .../hive/ql/exec/PartitionKeySampler.java | 4 +- .../hive/ql/exec/SMBMapJoinOperator.java | 6 +- .../hadoop/hive/ql/exec/ScriptOperator.java | 24 +- .../hadoop/hive/ql/exec/SelectOperator.java | 2 +- .../hive/ql/exec/TableScanOperator.java | 2 +- .../hadoop/hive/ql/exec/TextRecordReader.java | 2 +- .../hadoop/hive/ql/exec/TextRecordWriter.java | 2 +- .../hadoop/hive/ql/exec/UDTFOperator.java | 2 +- .../apache/hadoop/hive/ql/exec/Utilities.java | 10 +- .../exec/errors/MapAggrMemErrorHeuristic.java | 6 +- .../hadoop/hive/ql/exec/mr/ExecDriver.java | 24 +- .../hadoop/hive/ql/exec/mr/ExecMapper.java | 2 +- .../hive/ql/exec/mr/HadoopJobExecHelper.java | 21 +- .../hadoop/hive/ql/exec/mr/MapRedTask.java | 22 +- .../hive/ql/exec/mr/MapredLocalTask.java | 10 +- .../persistence/BytesBytesMultiHashMap.java | 4 +- .../ql/exec/persistence/HashMapWrapper.java | 13 +- .../persistence/HybridHashTableContainer.java | 19 +- .../MapJoinBytesTableContainer.java | 9 +- .../hive/ql/exec/repl/ReplDumpTask.java | 2 +- .../hive/ql/exec/repl/ReplLoadTask.java | 5 +- .../repl/bootstrap/load/util/PathInfo.java | 2 +- .../hive/ql/exec/repl/util/ReplUtils.java | 3 +- .../hadoop/hive/ql/exec/tez/DagUtils.java | 48 +- .../hive/ql/exec/tez/HashTableLoader.java | 12 +- .../hive/ql/exec/tez/HiveSplitGenerator.java | 9 +- .../hive/ql/exec/tez/MapRecordProcessor.java | 2 +- .../ql/exec/tez/MergeFileRecordProcessor.java | 2 +- .../ql/exec/tez/ReduceRecordProcessor.java | 2 +- .../hive/ql/exec/tez/TezSessionPool.java | 2 +- .../ql/exec/tez/TezSessionPoolManager.java | 4 +- .../hive/ql/exec/tez/TezSessionState.java | 8 +- .../hadoop/hive/ql/exec/tez/TezTask.java | 10 +- .../hive/ql/exec/tez/WorkloadManager.java | 3 +- .../ql/exec/tez/monitoring/DAGSummary.java | 3 +- .../ql/exec/vector/VectorFilterOperator.java | 3 +- .../ql/exec/vector/VectorGroupByOperator.java | 4 +- .../VectorInBloomFilterColDynamicValue.java | 2 +- .../VectorMapJoinFastHashTableLoader.java | 7 +- .../fast/VectorMapJoinFastTableContainer.java | 8 +- .../hive/ql/history/HiveHistoryImpl.java | 3 +- .../hive/ql/hooks/HiveProtoLoggingHook.java | 2 +- .../hadoop/hive/ql/hooks/HookContext.java | 8 +- .../ql/hooks/PostExecTezSummaryPrinter.java | 3 +- .../hadoop/hive/ql/io/HiveInputFormat.java | 2 +- .../org/apache/hadoop/hive/ql/io/RCFile.java | 2 +- .../hadoop/hive/ql/io/RCFileRecordReader.java | 2 +- .../hive/ql/io/merge/MergeFileTask.java | 6 +- .../hadoop/hive/ql/io/orc/ExternalCache.java | 3 +- .../hadoop/hive/ql/io/orc/OrcInputFormat.java | 7 +- .../hive/ql/io/orc/OrcRecordUpdater.java | 2 +- .../rcfile/truncate/ColumnTruncateTask.java | 14 +- .../hadoop/hive/ql/lockmgr/DbTxnManager.java | 8 +- .../apache/hadoop/hive/ql/metadata/Hive.java | 14 +- .../HiveMetaStoreClientWithLocalCache.java | 3 +- .../metadata/SessionHiveMetaStoreClient.java | 2 +- .../hive/ql/metadata/VirtualColumn.java | 2 +- .../hive/ql/optimizer/BucketMapjoinProc.java | 2 +- .../hive/ql/optimizer/ConvertJoinMapJoin.java | 20 +- .../optimizer/CountDistinctRewriteProc.java | 2 +- .../DynamicPartitionPruningOptimization.java | 16 +- .../hive/ql/optimizer/GenMapRedUtils.java | 32 +- .../hive/ql/optimizer/GroupByOptimizer.java | 4 +- .../ql/optimizer/LimitPushdownOptimizer.java | 4 +- .../hive/ql/optimizer/MapJoinProcessor.java | 20 +- .../hadoop/hive/ql/optimizer/Optimizer.java | 56 +- .../ql/optimizer/SemiJoinReductionMerge.java | 8 +- .../ql/optimizer/SetReducerParallelism.java | 6 +- .../ql/optimizer/SimpleFetchOptimizer.java | 10 +- .../SortedDynPartitionOptimizer.java | 2 +- .../SortedMergeBucketMapjoinProc.java | 2 +- .../HiveDefaultRelMetadataProvider.java | 2 +- .../opconventer/HiveGBOpConvUtil.java | 14 +- .../AbstractCorrelationProcCtx.java | 12 +- .../correlation/CorrelationOptimizer.java | 4 +- .../correlation/ReduceSinkDeDuplication.java | 12 +- .../ReduceSinkDeDuplicationUtils.java | 11 +- .../physical/CommonJoinTaskDispatcher.java | 12 +- .../physical/GenMRSkewJoinProcessor.java | 8 +- .../ql/optimizer/physical/LlapDecider.java | 4 +- .../physical/LocalMapJoinProcFactory.java | 8 +- .../ql/optimizer/physical/MemoryDecider.java | 6 +- .../optimizer/physical/PhysicalOptimizer.java | 13 +- .../physical/SortMergeJoinTaskDispatcher.java | 2 +- .../physical/StageIDsRearranger.java | 2 +- .../ql/optimizer/physical/Vectorizer.java | 4 +- .../ql/optimizer/ppr/PartitionPruner.java | 2 +- .../annotation/StatsRulesProcFactory.java | 10 +- .../hive/ql/parse/BaseSemanticAnalyzer.java | 8 +- .../hadoop/hive/ql/parse/CalcitePlanner.java | 22 +- .../ql/parse/ExecuteStatementAnalyzer.java | 4 +- .../apache/hadoop/hive/ql/parse/EximUtil.java | 8 +- .../ql/parse/ExplainSemanticAnalyzer.java | 2 +- .../hive/ql/parse/GenTezProcContext.java | 2 +- .../hadoop/hive/ql/parse/GenTezUtils.java | 5 +- .../hadoop/hive/ql/parse/GenTezWork.java | 2 +- .../hive/ql/parse/LoadSemanticAnalyzer.java | 4 +- .../hive/ql/parse/MapReduceCompiler.java | 8 +- .../hadoop/hive/ql/parse/ParseUtils.java | 4 +- .../ql/parse/ReplicationSemanticAnalyzer.java | 4 +- .../hive/ql/parse/SemanticAnalyzer.java | 128 ++-- .../hadoop/hive/ql/parse/StorageFormat.java | 10 +- .../hadoop/hive/ql/parse/TaskCompiler.java | 10 +- .../hadoop/hive/ql/parse/TezCompiler.java | 28 +- .../plan/ConditionalResolverCommonJoin.java | 2 +- .../plan/ConditionalResolverMergeFiles.java | 4 +- .../hadoop/hive/ql/plan/DynamicValue.java | 5 +- .../hadoop/hive/ql/plan/MapJoinDesc.java | 3 +- .../apache/hadoop/hive/ql/plan/PlanUtils.java | 2 +- .../hive/ql/ppd/ExprWalkerProcFactory.java | 2 +- .../hadoop/hive/ql/ppd/OpProcFactory.java | 28 +- .../hive/ql/processors/SetProcessor.java | 2 +- .../ScheduledQueryExecutionService.java | 2 +- .../fallback/FallbackHiveAuthorizer.java | 6 +- .../sqlstd/SQLStdHiveAccessController.java | 6 +- .../ql/session/ClearDanglingScratchDir.java | 6 +- .../hadoop/hive/ql/session/SessionState.java | 30 +- .../hive/ql/stats/BasicStatsNoJobTask.java | 2 +- .../hive/ql/stats/ColStatsProcessor.java | 2 +- .../ql/stats/OperatorStatsReaderHook.java | 2 +- .../hadoop/hive/ql/stats/StatsUtils.java | 2 +- .../hive/ql/txn/compactor/MRCompactor.java | 2 +- .../compactor/service/CompactionService.java | 4 +- .../hive/ql/udf/generic/GenericUDFRegExp.java | 6 +- .../ql/udf/generic/GenericUDTFGetSplits.java | 8 +- .../ql/udf/ptf/WindowingTableFunction.java | 4 +- .../ql/util/HiveStrictManagedMigration.java | 4 +- .../hadoop/hive/ql/util/UpgradeTool.java | 2 +- .../hive/metastore/txn/TestTxnHandler.java | 4 +- .../hadoop/hive/ql/TestTxnCommands.java | 21 +- .../hadoop/hive/ql/TestTxnCommands2.java | 110 ++-- .../hadoop/hive/ql/TestTxnCommands3.java | 4 +- .../hive/ql/TestTxnCommandsForMmTable.java | 7 +- .../apache/hadoop/hive/ql/TestTxnExIm.java | 4 +- .../hadoop/hive/ql/TestTxnNoBuckets.java | 22 +- .../hive/ql/TxnCommandsBaseForTests.java | 8 +- .../hadoop/hive/ql/exec/TestExecDriver.java | 4 +- .../hadoop/hive/ql/exec/TestExplainTask.java | 2 +- .../hive/ql/exec/TestLimitOperator.java | 2 +- .../hadoop/hive/ql/exec/TestOperators.java | 12 +- .../hadoop/hive/ql/exec/TestUtilities.java | 7 +- .../ql/exec/errors/TestTaskLogProcessor.java | 2 +- .../hive/ql/exec/mr/TestMapRedTask.java | 2 +- .../exec/vector/TestVectorLimitOperator.java | 2 +- .../hadoop/hive/ql/hooks/TestHiveHooks.java | 4 +- .../hadoop/hive/ql/hooks/TestHooks.java | 4 +- .../hadoop/hive/ql/io/StorageFormats.java | 13 +- .../apache/hadoop/hive/ql/io/TestRCFile.java | 8 +- .../hive/ql/io/orc/TestInputOutputFormat.java | 28 +- .../ql/io/orc/TestOrcSplitElimination.java | 18 +- .../TestVectorizedOrcAcidRowBatchReader.java | 6 +- .../hive/ql/lockmgr/ITestDbTxnManager.java | 2 +- .../hive/ql/lockmgr/TestDbTxnManager2.java | 8 +- .../hadoop/hive/ql/metadata/TestHive.java | 4 +- ...etastoreClientListPartitionsTempTable.java | 2 +- .../physical/TestNullScanTaskDispatcher.java | 2 +- .../ql/parse/TestDMLSemanticAnalyzer.java | 2 +- .../TestConditionalResolverCommonJoin.java | 7 +- .../ql/plan/mapping/TestCounterMapping.java | 4 +- .../hive/ql/plan/mapping/TestOperatorCmp.java | 4 +- .../ql/plan/mapping/TestReOptimization.java | 2 +- .../hive/ql/processors/TestSetProcessor.java | 4 +- ...stHivePrivilegeObjectOwnerNameAndType.java | 4 +- .../TestSQLStdHiveAccessControllerCLI.java | 2 +- .../TestSQLStdHiveAccessControllerHS2.java | 2 +- .../hive/ql/session/TestSessionState.java | 5 +- .../hive/ql/stats/TestStatsUpdaterThread.java | 8 +- .../util/TestHiveStrictManagedMigration.java | 4 +- .../hadoop/hive/ql/util/TestUpgradeTool.java | 2 +- .../groupby_cube_multi_gby.q.out | 2 +- .../hive/serde2/DefaultFetchFormatter.java | 2 +- .../service/cli/session/HiveSessionImpl.java | 4 +- .../hive/service/server/HiveServer2.java | 4 +- .../cli/session/TestSessionCleanup.java | 2 +- .../session/TestSessionManagerMetrics.java | 4 +- .../service/server/TestHS2HttpServer.java | 4 +- .../service/server/TestHS2HttpServerPam.java | 13 +- .../TestHS2HttpServerPamConfiguration.java | 2 +- .../DummyRawStoreForJdoConnection.java | 2 +- .../hive/metastore/TestObjectStore.java | 4 +- .../TestRetriesInRetryingHMSHandler.java | 2 +- .../hive/streaming/AbstractRecordWriter.java | 2 +- .../streaming/HiveStreamingConnection.java | 2 +- .../TestStreamingDynamicPartitioning.java | 8 +- 345 files changed, 1539 insertions(+), 1650 deletions(-) diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 49b5fad11be7..33d13013049d 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -1831,7 +1831,7 @@ String getPromptForCli() { String prompt; // read prompt configuration and substitute variables. HiveConf conf = getCommands().getHiveConf(true); - prompt = conf.getVar(HiveConf.ConfVars.CLIPROMPT); + prompt = conf.getVar(HiveConf.ConfVars.CLI_PROMPT); prompt = getCommands().substituteVariables(conf, prompt); return prompt + getFormattedDb() + "> "; } diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java index 17af4e2cd714..04ebab7df2e4 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java @@ -291,7 +291,7 @@ public void updateBeeLineOptsFromConf() { if (conf == null) { conf = beeLine.getCommands().getHiveConf(false); } - setForce(HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS)); + setForce(HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLI_IGNORE_ERRORS)); } } @@ -529,7 +529,7 @@ public boolean getShowDbInPrompt() { return showDbInPrompt; } else { HiveConf conf = beeLine.getCommands().getHiveConf(true); - return HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB); + return HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLI_PRINT_CURRENT_DB); } } diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java index 5ea4d11b7abd..048ca59becb0 100644 --- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java +++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java @@ -280,7 +280,7 @@ private void verifyCMD(String CMD, String keywords, OutputStream os, String[] op public static void init(){ // something changed scratch dir permissions, so test can't execute HiveConf hiveConf = new HiveConf(); - String scratchDir = hiveConf.get(HiveConf.ConfVars.SCRATCHDIR.varname); + String scratchDir = hiveConf.get(HiveConf.ConfVars.SCRATCH_DIR.varname); File file = new File(scratchDir); if (file.exists()) { file.setWritable(true, false); diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java index d0d37b506ab2..6eb679ddebc1 100644 --- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java +++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java @@ -425,7 +425,7 @@ public void handle(Signal signal) { ret = processCmd(command.toString()); lastRet = ret; } catch (CommandProcessorException e) { - boolean ignoreErrors = HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS); + boolean ignoreErrors = HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLI_IGNORE_ERRORS); if (!ignoreErrors) { throw e; } @@ -773,7 +773,7 @@ public int run(String[] args) throws Exception { } // read prompt configuration and substitute variables. - prompt = conf.getVar(HiveConf.ConfVars.CLIPROMPT); + prompt = conf.getVar(HiveConf.ConfVars.CLI_PROMPT); prompt = new VariableSubstitution(new HiveVariableSource() { @Override public Map getHiveVariable() { @@ -936,7 +936,7 @@ protected void setupConsoleReader() throws IOException { * @return String to show user for current db value */ private static String getFormattedDb(HiveConf conf, CliSessionState ss) { - if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB)) { + if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLI_PRINT_CURRENT_DB)) { return ""; } //BUG: This will not work in remote mode - HIVE-5153 diff --git a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java index afe6607298a9..806ed9be66cf 100644 --- a/common/src/java/org/apache/hadoop/hive/common/LogUtils.java +++ b/common/src/java/org/apache/hadoop/hive/common/LogUtils.java @@ -117,11 +117,11 @@ public static String initHiveLog4jCommon(HiveConf conf, ConfVars confVarName) // property specified file found in local file system // use the specified file if (confVarName == HiveConf.ConfVars.HIVE_EXEC_LOG4J_FILE) { - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); if(queryId == null || (queryId = queryId.trim()).isEmpty()) { queryId = "unknown-" + System.currentTimeMillis(); } - System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), queryId); + System.setProperty(HiveConf.ConfVars.HIVE_QUERY_ID.toString(), queryId); } final boolean async = checkAndSetAsyncLogging(conf); // required for MDC based routing appender so that child threads can inherit the MDC context @@ -157,8 +157,8 @@ private static String initHiveLog4jDefault( if (hive_l4j == null) { hive_l4j = LogUtils.class.getClassLoader().getResource(HIVE_L4J); } - System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), - HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID)); + System.setProperty(HiveConf.ConfVars.HIVE_QUERY_ID.toString(), + HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID)); break; case HIVE_LOG4J_FILE: hive_l4j = LogUtils.class.getClassLoader().getResource(HIVE_L4J); @@ -216,8 +216,8 @@ public static String maskIfPassword(String key, String value) { */ public static void registerLoggingContext(Configuration conf) { if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { - MDC.put(SESSIONID_LOG_KEY, HiveConf.getVar(conf, HiveConf.ConfVars.HIVESESSIONID)); - MDC.put(QUERYID_LOG_KEY, HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID)); + MDC.put(SESSIONID_LOG_KEY, HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SESSION_ID)); + MDC.put(QUERYID_LOG_KEY, HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID)); MDC.put(OPERATIONLOG_LEVEL_KEY, HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL)); MDC.put(OPERATIONLOG_LOCATION_KEY, HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION)); l4j.info("Thread context registration is done."); diff --git a/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java b/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java index d7f4b146ed52..f4f7a9cb051f 100644 --- a/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java +++ b/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java @@ -38,7 +38,7 @@ public class ServerUtils { public static void cleanUpScratchDir(HiveConf hiveConf) { if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_START_CLEANUP_SCRATCHDIR)) { - String hiveScratchDir = hiveConf.get(HiveConf.ConfVars.SCRATCHDIR.varname); + String hiveScratchDir = hiveConf.get(HiveConf.ConfVars.SCRATCH_DIR.varname); try { Path jobScratchDir = new Path(hiveScratchDir); LOG.info("Cleaning scratchDir : " + hiveScratchDir); diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 1fa63ae3821a..cbe91a509ff2 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -156,7 +156,7 @@ public static ResultFileFormat from(String value) { } public ResultFileFormat getResultFileFormat() { - return ResultFileFormat.from(this.getVar(ConfVars.HIVEQUERYRESULTFILEFORMAT)); + return ResultFileFormat.from(this.getVar(ConfVars.HIVE_QUERY_RESULT_FILEFORMAT)); } public interface EncoderDecoder { @@ -265,21 +265,21 @@ private static URL checkConfigFile(File f) { * be recreated so that the change will take effect. */ public static final HiveConf.ConfVars[] metaVars = { - HiveConf.ConfVars.METASTOREWAREHOUSE, - HiveConf.ConfVars.REPLDIR, - HiveConf.ConfVars.METASTOREURIS, - HiveConf.ConfVars.METASTORESELECTION, + HiveConf.ConfVars.METASTORE_WAREHOUSE, + HiveConf.ConfVars.REPL_DIR, + HiveConf.ConfVars.METASTORE_URIS, + HiveConf.ConfVars.METASTORE_SELECTION, HiveConf.ConfVars.METASTORE_SERVER_PORT, - HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, - HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, + HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, + HiveConf.ConfVars.METASTORE_THRIFT_FAILURE_RETRIES, HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME, - HiveConf.ConfVars.METASTOREPWD, - HiveConf.ConfVars.METASTORECONNECTURLHOOK, - HiveConf.ConfVars.METASTORECONNECTURLKEY, - HiveConf.ConfVars.METASTORESERVERMINTHREADS, - HiveConf.ConfVars.METASTORESERVERMAXTHREADS, + HiveConf.ConfVars.METASTORE_PWD, + HiveConf.ConfVars.METASTORE_CONNECT_URL_HOOK, + HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY, + HiveConf.ConfVars.METASTORE_SERVER_MIN_THREADS, + HiveConf.ConfVars.METASTORE_SERVER_MAX_THREADS, HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE, HiveConf.ConfVars.METASTORE_INT_ORIGINAL, HiveConf.ConfVars.METASTORE_INT_ARCHIVED, @@ -315,9 +315,9 @@ private static URL checkConfigFile(File f) { HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX, HiveConf.ConfVars.METASTORE_INIT_HOOKS, HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS, - HiveConf.ConfVars.HMSHANDLERATTEMPTS, - HiveConf.ConfVars.HMSHANDLERINTERVAL, - HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF, + HiveConf.ConfVars.HMS_HANDLER_ATTEMPTS, + HiveConf.ConfVars.HMS_HANDLER_INTERVAL, + HiveConf.ConfVars.HMS_HANDLER_FORCE_RELOAD_CONF, HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN, HiveConf.ConfVars.METASTORE_ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS, HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES, @@ -376,16 +376,16 @@ private static URL checkConfigFile(File f) { * for different databases. */ public static final HiveConf.ConfVars[] dbVars = { - HiveConf.ConfVars.HADOOPBIN, - HiveConf.ConfVars.METASTOREWAREHOUSE, - HiveConf.ConfVars.SCRATCHDIR + HiveConf.ConfVars.HADOOP_BIN, + HiveConf.ConfVars.METASTORE_WAREHOUSE, + HiveConf.ConfVars.SCRATCH_DIR }; /** * encoded parameter values are ;-) encoded. Use decoder to get ;-) decoded string */ static final HiveConf.ConfVars[] ENCODED_CONF = { - ConfVars.HIVEQUERYSTRING + ConfVars.HIVE_QUERY_STRING }; /** @@ -489,31 +489,31 @@ public static enum ConfVars { // QL execution stuff DFS_XATTR_ONLY_SUPPORTED_ON_RESERVED_NAMESPACE("dfs.xattr.supported.only.on.reserved.namespace", false, "DFS supports xattr only on Reserved Name space (/.reserved/raw)"), - SCRIPTWRAPPER("hive.exec.script.wrapper", null, ""), + SCRIPT_WRAPPER("hive.exec.script.wrapper", null, ""), PLAN("hive.exec.plan", "", ""), - STAGINGDIR("hive.exec.stagingdir", ".hive-staging", + STAGING_DIR("hive.exec.stagingdir", ".hive-staging", "Directory name that will be created inside table locations in order to support HDFS encryption. " + "This is replaces ${hive.exec.scratchdir} for query results with the exception of read-only tables. " + "In all cases ${hive.exec.scratchdir} is still used for other temporary files, such as job plans."), - SCRATCHDIR("hive.exec.scratchdir", "/tmp/hive", + SCRATCH_DIR("hive.exec.scratchdir", "/tmp/hive", "HDFS root scratch dir for Hive jobs which gets created with write all (733) permission. " + "For each connecting user, an HDFS scratch dir: ${hive.exec.scratchdir}/ is created, " + "with ${hive.scratch.dir.permission}."), - REPLDIR("hive.repl.rootdir","/user/${system:user.name}/repl/", + REPL_DIR("hive.repl.rootdir","/user/${system:user.name}/repl/", "HDFS root dir for all replication dumps."), - REPLCMENABLED("hive.repl.cm.enabled", false, + REPL_CM_ENABLED("hive.repl.cm.enabled", false, "Turn on ChangeManager, so delete files will go to cmrootdir."), - REPLCMDIR("hive.repl.cmrootdir","/user/${system:user.name}/cmroot/", + REPL_CM_DIR("hive.repl.cmrootdir","/user/${system:user.name}/cmroot/", "Root dir for ChangeManager, used for deleted files."), - REPLCMRETIAN("hive.repl.cm.retain","10d", + REPL_CM_RETAIN("hive.repl.cm.retain","10d", new TimeValidator(TimeUnit.DAYS), "Time to retain removed files in cmrootdir."), - REPLCMENCRYPTEDDIR("hive.repl.cm.encryptionzone.rootdir", ".cmroot", + REPL_CM_ENCRYPTED_DIR("hive.repl.cm.encryptionzone.rootdir", ".cmroot", "Root dir for ChangeManager if encryption zones are enabled, used for deleted files."), - REPLCMFALLBACKNONENCRYPTEDDIR("hive.repl.cm.nonencryptionzone.rootdir", + REPL_CM_FALLBACK_NONENCRYPTED_DIR("hive.repl.cm.nonencryptionzone.rootdir", "", "Root dir for ChangeManager for non encrypted paths if hive.repl.cmrootdir is encrypted."), - REPLCMINTERVAL("hive.repl.cm.interval","3600s", + REPL_CM_INTERVAL("hive.repl.cm.interval","3600s", new TimeValidator(TimeUnit.SECONDS), "Interval for cmroot cleanup thread."), REPL_HA_DATAPATH_REPLACE_REMOTE_NAMESERVICE("hive.repl.ha.datapath.replace.remote.nameservice", false, @@ -694,93 +694,93 @@ public static enum ConfVars { REPL_STATS_TOP_EVENTS_COUNTS("hive.repl.stats.events.count", 5, "Number of topmost expensive events that needs to be maintained per event type for the replication statistics." + " Maximum permissible limit is 10."), - LOCALSCRATCHDIR("hive.exec.local.scratchdir", + LOCAL_SCRATCH_DIR("hive.exec.local.scratchdir", "${system:java.io.tmpdir}" + File.separator + "${system:user.name}", "Local scratch space for Hive jobs"), DOWNLOADED_RESOURCES_DIR("hive.downloaded.resources.dir", "${system:java.io.tmpdir}" + File.separator + "${hive.session.id}_resources", "Temporary local directory for added resources in the remote file system."), - SCRATCHDIRPERMISSION("hive.scratch.dir.permission", "700", + SCRATCH_DIR_PERMISSION("hive.scratch.dir.permission", "700", "The permission for the user specific scratch directories that get created."), - SUBMITVIACHILD("hive.exec.submitviachild", false, ""), - SUBMITLOCALTASKVIACHILD("hive.exec.submit.local.task.via.child", true, + SUBMIT_VIA_CHILD("hive.exec.submitviachild", false, ""), + SUBMIT_LOCAL_TASK_VIA_CHILD("hive.exec.submit.local.task.via.child", true, "Determines whether local tasks (typically mapjoin hashtable generation phase) runs in \n" + "separate JVM (true recommended) or not. \n" + "Avoids the overhead of spawning new JVM, but can lead to out-of-memory issues."), - SCRIPTERRORLIMIT("hive.exec.script.maxerrsize", 100000, + SCRIPT_ERROR_LIMIT("hive.exec.script.maxerrsize", 100000, "Maximum number of bytes a script is allowed to emit to standard error (per map-reduce task). \n" + "This prevents runaway scripts from filling logs partitions to capacity"), - ALLOWPARTIALCONSUMP("hive.exec.script.allow.partial.consumption", false, + ALLOW_PARTIAL_CONSUMP("hive.exec.script.allow.partial.consumption", false, "When enabled, this option allows a user script to exit successfully without consuming \n" + "all the data from the standard input."), - STREAMREPORTERPERFIX("stream.stderr.reporter.prefix", "reporter:", + STREAM_REPORTER_PREFIX("stream.stderr.reporter.prefix", "reporter:", "Streaming jobs that log to standard error with this prefix can log counter or status information."), - STREAMREPORTERENABLED("stream.stderr.reporter.enabled", true, + STREAM_REPORTER_ENABLED("stream.stderr.reporter.enabled", true, "Enable consumption of status and counter messages for streaming jobs."), - COMPRESSRESULT("hive.exec.compress.output", false, + COMPRESS_RESULT("hive.exec.compress.output", false, "This controls whether the final outputs of a query (to a local/HDFS file or a Hive table) is compressed. \n" + "The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"), - COMPRESSINTERMEDIATE("hive.exec.compress.intermediate", false, + COMPRESS_INTERMEDIATE("hive.exec.compress.intermediate", false, "This controls whether intermediate files produced by Hive between multiple map-reduce jobs are compressed. \n" + "The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"), - COMPRESSINTERMEDIATECODEC("hive.intermediate.compression.codec", "", ""), - COMPRESSINTERMEDIATETYPE("hive.intermediate.compression.type", "", ""), - BYTESPERREDUCER("hive.exec.reducers.bytes.per.reducer", (long) (256 * 1000 * 1000), + COMPRESS_INTERMEDIATE_CODEC("hive.intermediate.compression.codec", "", ""), + COMPRESS_INTERMEDIATE_TYPE("hive.intermediate.compression.type", "", ""), + BYTES_PER_REDUCER("hive.exec.reducers.bytes.per.reducer", (long) (256 * 1000 * 1000), "size per reducer.The default is 256Mb, i.e if the input size is 1G, it will use 4 reducers."), - MAXREDUCERS("hive.exec.reducers.max", 1009, + MAX_REDUCERS("hive.exec.reducers.max", 1009, "max number of reducers will be used. If the one specified in the configuration parameter mapred.reduce.tasks is\n" + "negative, Hive will use this one as the max number of reducers when automatically determine number of reducers."), - PREEXECHOOKS("hive.exec.pre.hooks", "", + PRE_EXEC_HOOKS("hive.exec.pre.hooks", "", "Comma-separated list of pre-execution hooks to be invoked for each statement. \n" + "A pre-execution hook is specified as the name of a Java class which implements the \n" + "org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."), - POSTEXECHOOKS("hive.exec.post.hooks", "", + POST_EXEC_HOOKS("hive.exec.post.hooks", "", "Comma-separated list of post-execution hooks to be invoked for each statement. \n" + "A post-execution hook is specified as the name of a Java class which implements the \n" + "org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."), - ONFAILUREHOOKS("hive.exec.failure.hooks", "", + ON_FAILURE_HOOKS("hive.exec.failure.hooks", "", "Comma-separated list of on-failure hooks to be invoked for each statement. \n" + "An on-failure hook is specified as the name of Java class which implements the \n" + "org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."), - QUERYREDACTORHOOKS("hive.exec.query.redactor.hooks", "", + QUERY_REDACTOR_HOOKS("hive.exec.query.redactor.hooks", "", "Comma-separated list of hooks to be invoked for each query which can \n" + "transform the query before it's placed in the job.xml file. Must be a Java class which \n" + "extends from the org.apache.hadoop.hive.ql.hooks.Redactor abstract class."), - CLIENTSTATSPUBLISHERS("hive.client.stats.publishers", "", + CLIENT_STATS_PUBLISHERS("hive.client.stats.publishers", "", "Comma-separated list of statistics publishers to be invoked on counters on each job. \n" + "A client stats publisher is specified as the name of a Java class which implements the \n" + "org.apache.hadoop.hive.ql.stats.ClientStatsPublisher interface."), - BASICSTATSTASKSMAXTHREADSFACTOR("hive.basic.stats.max.threads.factor", 2, "Determines the maximum number of " + BASIC_STATS_TASKS_MAX_THREADS_FACTOR("hive.basic.stats.max.threads.factor", 2, "Determines the maximum number of " + "threads that can be used for collection of file level statistics. If the value configured is x, then the " + "maximum number of threads that can be used is x multiplied by the number of available processors. A value" + " of less than 1, makes stats collection sequential."), - EXECPARALLEL("hive.exec.parallel", false, "Whether to execute jobs in parallel"), - EXECPARALLETHREADNUMBER("hive.exec.parallel.thread.number", 8, + EXEC_PARALLEL("hive.exec.parallel", false, "Whether to execute jobs in parallel"), + EXEC_PARALLEL_THREAD_NUMBER("hive.exec.parallel.thread.number", 8, "How many jobs at most can be executed in parallel"), @Deprecated - HIVESPECULATIVEEXECREDUCERS("hive.mapred.reduce.tasks.speculative.execution", false, + HIVE_SPECULATIVE_EXEC_REDUCERS("hive.mapred.reduce.tasks.speculative.execution", false, "(Deprecated) Whether speculative execution for reducers should be turned on. "), - HIVECOUNTERSPULLINTERVAL("hive.exec.counters.pull.interval", 1000L, + HIVE_COUNTERS_PULL_INTERVAL("hive.exec.counters.pull.interval", 1000L, "The interval with which to poll the JobTracker for the counters the running job. \n" + "The smaller it is the more load there will be on the jobtracker, the higher it is the less granular the caught will be."), - DYNAMICPARTITIONING("hive.exec.dynamic.partition", true, + DYNAMIC_PARTITIONING("hive.exec.dynamic.partition", true, "Whether or not to allow dynamic partitions in DML/DDL."), - DYNAMICPARTITIONINGMODE("hive.exec.dynamic.partition.mode", "nonstrict", + DYNAMIC_PARTITIONING_MODE("hive.exec.dynamic.partition.mode", "nonstrict", new StringSet("strict", "nonstrict"), "In strict mode, the user must specify at least one static partition\n" + "in case the user accidentally overwrites all partitions.\n" + "In nonstrict mode all partitions are allowed to be dynamic."), - DYNAMICPARTITIONMAXPARTS("hive.exec.max.dynamic.partitions", 1000, + DYNAMIC_PARTITION_MAX_PARTS("hive.exec.max.dynamic.partitions", 1000, "Maximum number of dynamic partitions allowed to be created in total."), - DYNAMICPARTITIONMAXPARTSPERNODE("hive.exec.max.dynamic.partitions.pernode", 100, + DYNAMIC_PARTITION_MAX_PARTS_PER_NODE("hive.exec.max.dynamic.partitions.pernode", 100, "Maximum number of dynamic partitions allowed to be created in each mapper/reducer node."), - DYNAMICPARTITIONCONVERT("hive.exec.dynamic.partition.type.conversion", true, + DYNAMIC_PARTITION_CONVERT("hive.exec.dynamic.partition.type.conversion", true, "Whether to check and cast a dynamic partition column before creating the partition " + "directory. For example, if partition p is type int and we insert string '001', then if " + "this value is true, directory p=1 will be created; if false, p=001"), - MAXCREATEDFILES("hive.exec.max.created.files", 100000L, + MAX_CREATED_FILES("hive.exec.max.created.files", 100000L, "Maximum number of HDFS files created by all mappers/reducers in a MapReduce job."), - DEFAULTPARTITIONNAME("hive.exec.default.partition.name", "__HIVE_DEFAULT_PARTITION__", + DEFAULT_PARTITION_NAME("hive.exec.default.partition.name", "__HIVE_DEFAULT_PARTITION__", "The default partition name in case the dynamic partition column value is null/empty string or any other values that cannot be escaped. \n" + "This value must not contain any special character used in HDFS URI (e.g., ':', '%', '/' etc). \n" + "The user has to be aware that the dynamic partition value should not contain this value to avoid confusions."), @@ -841,17 +841,17 @@ public static enum ConfVars { HIVE_IN_REPL_TEST_FILES_SORTED("hive.in.repl.test.files.sorted", false, "internal usage only, set to true if the file listing is required in sorted order during bootstrap load", true), - LOCALMODEAUTO("hive.exec.mode.local.auto", false, + LOCAL_MODE_AUTO("hive.exec.mode.local.auto", false, "Let Hive determine whether to run in local mode automatically"), - LOCALMODEMAXBYTES("hive.exec.mode.local.auto.inputbytes.max", 134217728L, + LOCAL_MODE_MAX_BYTES("hive.exec.mode.local.auto.inputbytes.max", 134217728L, "When hive.exec.mode.local.auto is true, input bytes should less than this for local mode."), - LOCALMODEMAXINPUTFILES("hive.exec.mode.local.auto.input.files.max", 4, + LOCAL_MODE_MAX_INPUT_FILES("hive.exec.mode.local.auto.input.files.max", 4, "When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode."), DROP_IGNORES_NON_EXISTENT("hive.exec.drop.ignorenonexistent", true, "Do not report an error if DROP TABLE/VIEW/Index/Function specifies a nonexistent table/view/function"), - HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"), + HIVE_IGNORE_MAPJOIN_HINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"), HIVE_CONF_LOCKED_LIST("hive.conf.locked.list", "", "Comma separated " + "list of configuration options which are locked and can not be changed at runtime. Warning is logged and the " + @@ -886,28 +886,28 @@ public static enum ConfVars { // Properties with null values are ignored and exist only for the purpose of giving us // a symbolic name to reference in the Hive source code. Properties with non-null // values will override any values set in the underlying Hadoop configuration. - HADOOPBIN("hadoop.bin.path", findHadoopBinary(), "", true), - YARNBIN("yarn.bin.path", findYarnBinary(), "", true), - MAPREDBIN("mapred.bin.path", findMapRedBinary(), "", true), + HADOOP_BIN("hadoop.bin.path", findHadoopBinary(), "", true), + YARN_BIN("yarn.bin.path", findYarnBinary(), "", true), + MAPRED_BIN("mapred.bin.path", findMapRedBinary(), "", true), HIVE_FS_HAR_IMPL("fs.har.impl", "org.apache.hadoop.hive.shims.HiveHarFileSystem", "The implementation for accessing Hadoop Archives. Note that this won't be applicable to Hadoop versions less than 0.20"), - MAPREDMAXSPLITSIZE(FileInputFormat.SPLIT_MAXSIZE, 256000000L, "", true), - MAPREDMINSPLITSIZE(FileInputFormat.SPLIT_MINSIZE, 1L, "", true), - MAPREDMINSPLITSIZEPERNODE(CombineFileInputFormat.SPLIT_MINSIZE_PERNODE, 1L, "", true), - MAPREDMINSPLITSIZEPERRACK(CombineFileInputFormat.SPLIT_MINSIZE_PERRACK, 1L, "", true), + MAPRED_MAX_SPLIT_SIZE(FileInputFormat.SPLIT_MAXSIZE, 256000000L, "", true), + MAPRED_MIN_SPLIT_SIZE(FileInputFormat.SPLIT_MINSIZE, 1L, "", true), + MAPRED_MIN_SPLIT_SIZE_PER_NODE(CombineFileInputFormat.SPLIT_MINSIZE_PERNODE, 1L, "", true), + MAPRED_MIN_SPLIT_SIZE_PER_RACK(CombineFileInputFormat.SPLIT_MINSIZE_PERRACK, 1L, "", true), // The number of reduce tasks per job. Hadoop sets this value to 1 by default // By setting this property to -1, Hive will automatically determine the correct // number of reducers. - HADOOPNUMREDUCERS("mapreduce.job.reduces", -1, "", true), + HADOOP_NUM_REDUCERS("mapreduce.job.reduces", -1, "", true), // Metastore stuff. Be sure to update HiveConf.metaVars when you add something here! - METASTOREDBTYPE("hive.metastore.db.type", "DERBY", new StringSet("DERBY", "ORACLE", "MYSQL", "MSSQL", "POSTGRES"), + METASTORE_DB_TYPE("hive.metastore.db.type", "DERBY", new StringSet("DERBY", "ORACLE", "MYSQL", "MSSQL", "POSTGRES"), "Type of database used by the metastore. Information schema & JDBCStorageHandler depend on it."), /** * @deprecated Use MetastoreConf.WAREHOUSE */ @Deprecated - METASTOREWAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse", + METASTORE_WAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse", "location of default database for the warehouse"), HIVE_METASTORE_WAREHOUSE_EXTERNAL("hive.metastore.warehouse.external.dir", null, @@ -918,14 +918,14 @@ public static enum ConfVars { * @deprecated Use MetastoreConf.THRIFT_URIS */ @Deprecated - METASTOREURIS("hive.metastore.uris", "", + METASTORE_URIS("hive.metastore.uris", "", "Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore."), /** * @deprecated Use MetastoreConf.THRIFT_URI_SELECTION */ @Deprecated - METASTORESELECTION("hive.metastore.uri.selection", "RANDOM", + METASTORE_SELECTION("hive.metastore.uri.selection", "RANDOM", new StringSet("SEQUENTIAL", "RANDOM"), "Determines the selection mechanism used by metastore client to connect to remote " + "metastore. SEQUENTIAL implies that the first valid metastore from the URIs specified " + @@ -976,13 +976,13 @@ public static enum ConfVars { * @deprecated Use MetastoreConf.THRIFT_CONNECTION_RETRIES */ @Deprecated - METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3, + METASTORE_THRIFT_CONNECTION_RETRIES("hive.metastore.connect.retries", 3, "Number of retries while opening a connection to metastore"), /** * @deprecated Use MetastoreConf.THRIFT_FAILURE_RETRIES */ @Deprecated - METASTORETHRIFTFAILURERETRIES("hive.metastore.failure.retries", 1, + METASTORE_THRIFT_FAILURE_RETRIES("hive.metastore.failure.retries", 1, "Number of retries upon failure of Thrift metastore calls"), /** * @deprecated Use MetastoreConf.SERVER_PORT @@ -1016,25 +1016,25 @@ public static enum ConfVars { * @deprecated Use MetastoreConf.PWD */ @Deprecated - METASTOREPWD("javax.jdo.option.ConnectionPassword", "mine", + METASTORE_PWD("javax.jdo.option.ConnectionPassword", "mine", "password to use against metastore database"), /** * @deprecated Use MetastoreConf.CONNECT_URL_HOOK */ @Deprecated - METASTORECONNECTURLHOOK("hive.metastore.ds.connection.url.hook", "", + METASTORE_CONNECT_URL_HOOK("hive.metastore.ds.connection.url.hook", "", "Name of the hook to use for retrieving the JDO connection URL. If empty, the value in javax.jdo.option.ConnectionURL is used"), /** * @deprecated Use MetastoreConf.MULTITHREADED */ @Deprecated - METASTOREMULTITHREADED("javax.jdo.option.Multithreaded", true, + METASTORE_MULTI_THREADED("javax.jdo.option.Multithreaded", true, "Set this to true if multiple threads access metastore through JDO concurrently."), /** * @deprecated Use MetastoreConf.CONNECT_URL_KEY */ @Deprecated - METASTORECONNECTURLKEY("javax.jdo.option.ConnectionURL", + METASTORE_CONNECT_URL_KEY("javax.jdo.option.ConnectionURL", "jdbc:derby:;databaseName=metastore_db;create=true", "JDBC connect string for a JDBC metastore.\n" + "To use SSL to encrypt/authenticate the connection, provide database-specific SSL flag in the connection URL.\n" + @@ -1050,19 +1050,19 @@ public static enum ConfVars { * @deprecated Use MetastoreConf.HMS_HANDLER_ATTEMPTS */ @Deprecated - HMSHANDLERATTEMPTS("hive.hmshandler.retry.attempts", 10, + HMS_HANDLER_ATTEMPTS("hive.hmshandler.retry.attempts", 10, "The number of times to retry a HMSHandler call if there were a connection error."), /** * @deprecated Use MetastoreConf.HMS_HANDLER_INTERVAL */ @Deprecated - HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", "2000ms", + HMS_HANDLER_INTERVAL("hive.hmshandler.retry.interval", "2000ms", new TimeValidator(TimeUnit.MILLISECONDS), "The time between HMSHandler retry attempts on failure."), /** * @deprecated Use MetastoreConf.HMS_HANDLER_FORCE_RELOAD_CONF */ @Deprecated - HMSHANDLERFORCERELOADCONF("hive.hmshandler.force.reload.conf", false, + HMS_HANDLER_FORCE_RELOAD_CONF("hive.hmshandler.force.reload.conf", false, "Whether to force reloading of the HMSHandler configuration (including\n" + "the connection URL, before the next metastore query that accesses the\n" + "datastore. Once reloaded, this value is reset to false. Used for\n" + @@ -1071,19 +1071,19 @@ public static enum ConfVars { * @deprecated Use MetastoreConf.SERVER_MAX_MESSAGE_SIZE */ @Deprecated - METASTORESERVERMAXMESSAGESIZE("hive.metastore.server.max.message.size", 100*1024*1024L, + METASTORE_SERVER_MAX_MESSAGE_SIZE("hive.metastore.server.max.message.size", 100*1024*1024L, "Maximum message size in bytes a HMS will accept."), /** * @deprecated Use MetastoreConf.SERVER_MIN_THREADS */ @Deprecated - METASTORESERVERMINTHREADS("hive.metastore.server.min.threads", 200, + METASTORE_SERVER_MIN_THREADS("hive.metastore.server.min.threads", 200, "Minimum number of worker threads in the Thrift server's pool."), /** * @deprecated Use MetastoreConf.SERVER_MAX_THREADS */ @Deprecated - METASTORESERVERMAXTHREADS("hive.metastore.server.max.threads", 1000, + METASTORE_SERVER_MAX_THREADS("hive.metastore.server.max.threads", 1000, "Maximum number of worker threads in the Thrift server's pool."), /** * @deprecated Use MetastoreConf.TCP_KEEP_ALIVE @@ -1729,10 +1729,10 @@ public static enum ConfVars { "alongside the dropped table data. This ensures that the metadata will be cleaned up along with the dropped table data."), // CLI - CLIIGNOREERRORS("hive.cli.errors.ignore", false, ""), - CLIPRINTCURRENTDB("hive.cli.print.current.db", false, + CLI_IGNORE_ERRORS("hive.cli.errors.ignore", false, ""), + CLI_PRINT_CURRENT_DB("hive.cli.print.current.db", false, "Whether to include the current database in the Hive prompt."), - CLIPROMPT("hive.cli.prompt", "hive", + CLI_PROMPT("hive.cli.prompt", "hive", "Command line prompt configuration value. Other hiveconf can be used in this configuration value. \n" + "Variable substitution will only be invoked at the Hive CLI startup."), /** @@ -1744,9 +1744,9 @@ public static enum ConfVars { // Things we log in the jobconf // session identifier - HIVESESSIONID("hive.session.id", "", ""), + HIVE_SESSION_ID("hive.session.id", "", ""), // whether session is running in silent mode or not - HIVESESSIONSILENT("hive.session.silent", false, ""), + HIVE_SESSION_SILENT("hive.session.silent", false, ""), HIVE_LOCAL_TIME_ZONE("hive.local.time.zone", "LOCAL", "Sets the time-zone for displaying and interpreting time stamps. If this property value is set to\n" + @@ -1757,35 +1757,35 @@ public static enum ConfVars { HIVE_SESSION_HISTORY_ENABLED("hive.session.history.enabled", false, "Whether to log Hive query, query plan, runtime statistics etc."), - HIVEQUERYSTRING("hive.query.string", "", + HIVE_QUERY_STRING("hive.query.string", "", "Query being executed (might be multiple per a session)"), - HIVEQUERYID("hive.query.id", "", + HIVE_QUERY_ID("hive.query.id", "", "ID for query being executed (might be multiple per a session)"), - HIVEQUERYTAG("hive.query.tag", null, "Tag for the queries in the session. User can kill the queries with the tag " + + HIVE_QUERY_TAG("hive.query.tag", null, "Tag for the queries in the session. User can kill the queries with the tag " + "in another session. Currently there is no tag duplication check, user need to make sure his tag is unique. " + "Also 'kill query' needs to be issued to all HiveServer2 instances to proper kill the queries"), - HIVEJOBNAMELENGTH("hive.jobname.length", 50, "max jobname length"), + HIVE_JOBNAME_LENGTH("hive.jobname.length", 50, "max jobname length"), // hive jar - HIVEJAR("hive.jar.path", "", + HIVE_JAR("hive.jar.path", "", "The location of hive_cli.jar that is used when submitting jobs in a separate jvm."), - HIVEAUXJARS("hive.aux.jars.path", "", + HIVE_AUX_JARS("hive.aux.jars.path", "", "The location of the plugin jars that contain implementations of user defined functions and serdes."), // reloadable jars - HIVERELOADABLEJARS("hive.reloadable.aux.jars.path", "", + HIVE_RELOADABLE_JARS("hive.reloadable.aux.jars.path", "", "The locations of the plugin jars, which can be a comma-separated folders or jars. Jars can be renewed\n" + "by executing reload command. And these jars can be " + "used as the auxiliary classes like creating a UDF or SerDe."), // hive added files and jars - HIVEADDEDFILES("hive.added.files.path", "", "This an internal parameter."), - HIVEADDEDJARS("hive.added.jars.path", "", "This an internal parameter."), - HIVEADDEDARCHIVES("hive.added.archives.path", "", "This an internal parameter."), - HIVEADDFILESUSEHDFSLOCATION("hive.resource.use.hdfs.location", true, "Reference HDFS based files/jars directly instead of " + HIVE_ADDED_FILES("hive.added.files.path", "", "This an internal parameter."), + HIVE_ADDED_JARS("hive.added.jars.path", "", "This an internal parameter."), + HIVE_ADDED_ARCHIVES("hive.added.archives.path", "", "This an internal parameter."), + HIVE_ADD_FILES_USE_HDFS_LOCATION("hive.resource.use.hdfs.location", true, "Reference HDFS based files/jars directly instead of " + "copy to session based HDFS scratch directory, to make distributed cache more useful."), HIVE_CURRENT_DATABASE("hive.current.database", "", "Database name used by current session. Internal usage only.", true), @@ -1795,17 +1795,17 @@ public static enum ConfVars { new TimeValidator(TimeUnit.SECONDS), "How long to run autoprogressor for the script/UDTF operators.\n" + "Set to 0 for forever."), - HIVESCRIPTAUTOPROGRESS("hive.script.auto.progress", false, + HIVE_SCRIPT_AUTO_PROGRESS("hive.script.auto.progress", false, "Whether Hive Transform/Map/Reduce Clause should automatically send progress information to TaskTracker \n" + "to avoid the task getting killed because of inactivity. Hive sends progress information when the script is \n" + "outputting to stderr. This option removes the need of periodically producing stderr messages, \n" + "but users should be cautious because this may prevent infinite loops in the scripts to be killed by TaskTracker."), - HIVESCRIPTIDENVVAR("hive.script.operator.id.env.var", "HIVE_SCRIPT_OPERATOR_ID", + HIVE_SCRIPT_ID_ENV_VAR("hive.script.operator.id.env.var", "HIVE_SCRIPT_OPERATOR_ID", "Name of the environment variable that holds the unique script operator ID in the user's \n" + "transform function (the custom mapper/reducer that the user has specified in the query)"), - HIVESCRIPTTRUNCATEENV("hive.script.operator.truncate.env", false, + HIVE_SCRIPT_TRUNCATE_ENV("hive.script.operator.truncate.env", false, "Truncate each environment variable for external script in scripts operator to 20KB (to fit system limits)"), - HIVESCRIPT_ENV_BLACKLIST("hive.script.operator.env.blacklist", + HIVE_SCRIPT_ENV_BLACKLIST("hive.script.operator.env.blacklist", "hive.txn.valid.txns,hive.txn.tables.valid.writeids,hive.txn.valid.writeids,hive.script.operator.env.blacklist,hive.repl.current.table.write.id", "Comma separated list of keys from the configuration file not to convert to environment " + "variables when invoking the script operator"), @@ -1838,16 +1838,16 @@ public static enum ConfVars { "Set the owner of files loaded using load data in managed tables."), @Deprecated - HIVEMAPREDMODE("hive.mapred.mode", null, + HIVE_MAPRED_MODE("hive.mapred.mode", null, "Deprecated; use hive.strict.checks.* settings instead."), - HIVEALIAS("hive.alias", "", ""), - HIVEMAPSIDEAGGREGATE("hive.map.aggr", true, "Whether to use map-side aggregation in Hive Group By queries"), - HIVEGROUPBYSKEW("hive.groupby.skewindata", false, "Whether there is skew in data to optimize group by queries"), + HIVE_ALIAS("hive.alias", "", ""), + HIVE_MAPSIDE_AGGREGATE("hive.map.aggr", true, "Whether to use map-side aggregation in Hive Group By queries"), + HIVE_GROUPBY_SKEW("hive.groupby.skewindata", false, "Whether there is skew in data to optimize group by queries"), HIVE_JOIN_SHORTCUT_UNMATCHED_ROWS("hive.join.shortcut.unmatched.rows", true, "Enables to shortcut processing of known filtered rows in merge joins. internal use only. may affect correctness"), - HIVEJOINEMITINTERVAL("hive.join.emit.interval", 1000, + HIVE_JOIN_EMIT_INTERVAL("hive.join.emit.interval", 1000, "How many rows in the right-most join operand Hive should buffer before emitting the join result."), - HIVEJOINCACHESIZE("hive.join.cache.size", 25000, + HIVE_JOIN_CACHE_SIZE("hive.join.cache.size", 25000, "How many rows in the joining tables (except the streaming table) should be cached in memory."), HIVE_PUSH_RESIDUAL_INNER("hive.join.inner.residual", false, "Whether to push non-equi filter predicates within inner joins. This can improve efficiency in " @@ -1963,31 +1963,31 @@ public static enum ConfVars { // hive.mapjoin.bucket.cache.size has been replaced by hive.smbjoin.cache.row, // need to remove by hive .13. Also, do not change default (see SMB operator) - HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100, ""), + HIVE_MAPJOIN_BUCKET_CACHE_SIZE("hive.mapjoin.bucket.cache.size", 100, ""), - HIVEMAPJOINUSEOPTIMIZEDTABLE("hive.mapjoin.optimized.hashtable", true, + HIVE_MAPJOIN_USE_OPTIMIZED_TABLE("hive.mapjoin.optimized.hashtable", true, "Whether Hive should use memory-optimized hash table for MapJoin.\n" + "Only works on Tez because memory-optimized hashtable cannot be serialized."), - HIVEMAPJOINOPTIMIZEDTABLEPROBEPERCENT("hive.mapjoin.optimized.hashtable.probe.percent", + HIVE_MAPJOIN_OPTIMIZED_TABLE_PROBE_PERCENT("hive.mapjoin.optimized.hashtable.probe.percent", (float) 0.5, "Probing space percentage of the optimized hashtable"), - HIVEMAPJOINPARALELHASHTABLETHREADS("hive.mapjoin.hashtable.load.threads", 2, + HIVE_MAPJOIN_PARALEL_HASHTABLE_THREADS("hive.mapjoin.hashtable.load.threads", 2, "Number of threads used to load records from a broadcast edge in HT used for MJ"), - HIVEUSEHYBRIDGRACEHASHJOIN("hive.mapjoin.hybridgrace.hashtable", false, "Whether to use hybrid" + + HIVE_USE_HYBRIDGRACE_HASHJOIN("hive.mapjoin.hybridgrace.hashtable", false, "Whether to use hybrid" + "grace hash join as the join method for mapjoin. Tez only."), - HIVEHYBRIDGRACEHASHJOINMEMCHECKFREQ("hive.mapjoin.hybridgrace.memcheckfrequency", 1024, "For " + + HIVE_HYBRIDGRACE_HASHJOIN_MEMCHECK_FREQ("hive.mapjoin.hybridgrace.memcheckfrequency", 1024, "For " + "hybrid grace hash join, how often (how many rows apart) we check if memory is full. " + "This number should be power of 2."), - HIVEHYBRIDGRACEHASHJOINMINWBSIZE("hive.mapjoin.hybridgrace.minwbsize", 524288, "For hybrid grace" + + HIVE_HYBRIDGRACE_HASHJOIN_MIN_WB_SIZE("hive.mapjoin.hybridgrace.minwbsize", 524288, "For hybrid grace" + "Hash join, the minimum write buffer size used by optimized hashtable. Default is 512 KB."), - HIVEHYBRIDGRACEHASHJOINMINNUMPARTITIONS("hive.mapjoin.hybridgrace.minnumpartitions", 16, "For" + + HIVE_HYBRIDGRACE_HASHJOIN_MIN_NUM_PARTITIONS("hive.mapjoin.hybridgrace.minnumpartitions", 16, "For" + "Hybrid grace hash join, the minimum number of partitions to create."), - HIVEHASHTABLEWBSIZE("hive.mapjoin.optimized.hashtable.wbsize", 8 * 1024 * 1024, + HIVE_HASHTABLE_WB_SIZE("hive.mapjoin.optimized.hashtable.wbsize", 8 * 1024 * 1024, "Optimized hashtable (see hive.mapjoin.optimized.hashtable) uses a chain of buffers to\n" + "store data. This is one buffer size. HT may be slightly faster if this is larger, but for small\n" + "joins unnecessary memory will be allocated and then trimmed."), - HIVEHYBRIDGRACEHASHJOINBLOOMFILTER("hive.mapjoin.hybridgrace.bloomfilter", true, "Whether to " + + HIVE_HYBRIDGRACE_HASHJOIN_BLOOMFILTER("hive.mapjoin.hybridgrace.bloomfilter", true, "Whether to " + "use BloomFilter in Hybrid grace hash join to minimize unnecessary spilling."), - HIVEMAPJOINFULLOUTER("hive.mapjoin.full.outer", true, + HIVE_MAPJOIN_FULL_OUTER("hive.mapjoin.full.outer", true, "Whether to use MapJoin for FULL OUTER JOINs."), HIVE_TEST_MAPJOINFULLOUTER_OVERRIDE( "hive.test.mapjoin.full.outer.override", @@ -1996,27 +1996,27 @@ public static enum ConfVars { "setting. Using enable will force it on and disable will force it off.\n" + "The default none is do nothing, of course", true), - HIVESMBJOINCACHEROWS("hive.smbjoin.cache.rows", 10000, + HIVE_SMBJOIN_CACHE_ROWS("hive.smbjoin.cache.rows", 10000, "How many rows with the same key value should be cached in memory per smb joined table."), - HIVEGROUPBYMAPINTERVAL("hive.groupby.mapaggr.checkinterval", 100000, + HIVE_GROUPBY_MAP_INTERVAL("hive.groupby.mapaggr.checkinterval", 100000, "Number of rows after which size of the grouping keys/aggregation classes is performed"), - HIVEMAPAGGRHASHMEMORY("hive.map.aggr.hash.percentmemory", (float) 0.5, + HIVE_MAP_AGGR_HASH_MEMORY("hive.map.aggr.hash.percentmemory", (float) 0.5, "Portion of total memory to be used by map-side group aggregation hash table"), - HIVEMAPJOINFOLLOWEDBYMAPAGGRHASHMEMORY("hive.mapjoin.followby.map.aggr.hash.percentmemory", (float) 0.3, + HIVE_MAPJOIN_FOLLOWEDBY_MAP_AGGR_HASH_MEMORY("hive.mapjoin.followby.map.aggr.hash.percentmemory", (float) 0.3, "Portion of total memory to be used by map-side group aggregation hash table, when this group by is followed by map join"), - HIVEMAPAGGRMEMORYTHRESHOLD("hive.map.aggr.hash.force.flush.memory.threshold", (float) 0.9, + HIVE_MAP_AGGR_MEMORY_THRESHOLD("hive.map.aggr.hash.force.flush.memory.threshold", (float) 0.9, "The max memory to be used by map-side group aggregation hash table.\n" + "If the memory usage is higher than this number, force to flush data"), - HIVEMAPAGGRHASHMINREDUCTION("hive.map.aggr.hash.min.reduction", (float) 0.99, + HIVE_MAP_AGGR_HASH_MIN_REDUCTION("hive.map.aggr.hash.min.reduction", (float) 0.99, "Hash aggregation will be turned off if the ratio between hash table size and input rows is bigger than this number. \n" + "Set to 1 to make sure hash aggregation is never turned off."), - HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND("hive.map.aggr.hash.min.reduction.lower.bound", (float) 0.4, + HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND("hive.map.aggr.hash.min.reduction.lower.bound", (float) 0.4, "Lower bound of Hash aggregate reduction filter. See also: hive.map.aggr.hash.min.reduction"), - HIVEMAPAGGRHASHMINREDUCTIONSTATSADJUST("hive.map.aggr.hash.min.reduction.stats", true, + HIVE_MAP_AGGR_HASH_MIN_REDUCTION_STATS_ADJUST("hive.map.aggr.hash.min.reduction.stats", true, "Whether the value for hive.map.aggr.hash.min.reduction should be set statically using stats estimates. \n" + "If this is enabled, the default value for hive.map.aggr.hash.min.reduction is only used as an upper-bound\n" + "for the value set in the map-side group by operators."), - HIVEMULTIGROUPBYSINGLEREDUCER("hive.multigroupby.singlereducer", true, + HIVE_MULTI_GROUPBY_SINGLE_REDUCER("hive.multigroupby.singlereducer", true, "Whether to optimize multi group by query to generate single M/R job plan. If the multi group by query has \n" + "common group by keys, it will be optimized to generate single M/R job."), HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", false, @@ -2056,14 +2056,14 @@ public static enum ConfVars { "Distributed copies (distcp) will be used instead for bigger files so that copies can be done faster."), // for hive udtf operator - HIVEUDTFAUTOPROGRESS("hive.udtf.auto.progress", false, + HIVE_UDTF_AUTO_PROGRESS("hive.udtf.auto.progress", false, "Whether Hive should automatically send progress information to TaskTracker \n" + "when using UDTF's to prevent the task getting killed because of inactivity. Users should be cautious \n" + "because this may prevent TaskTracker from killing tasks with infinite loops."), - HIVEDEFAULTFILEFORMAT("hive.default.fileformat", "TextFile", new StringSet("TextFile", "SequenceFile", "RCfile", "ORC", "parquet"), + HIVE_DEFAULT_FILEFORMAT("hive.default.fileformat", "TextFile", new StringSet("TextFile", "SequenceFile", "RCfile", "ORC", "parquet"), "Default file format for CREATE TABLE statement. Users can explicitly override it by CREATE TABLE ... STORED AS [FORMAT]"), - HIVEDEFAULTMANAGEDFILEFORMAT("hive.default.fileformat.managed", "none", + HIVE_DEFAULT_MANAGED_FILEFORMAT("hive.default.fileformat.managed", "none", new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC", "parquet"), "Default file format for CREATE TABLE statement applied to managed tables only. External tables will be \n" + "created with format specified by hive.default.fileformat. Leaving this null will result in using hive.default.fileformat \n" + @@ -2072,17 +2072,17 @@ public static enum ConfVars { "Default storage handler class for CREATE TABLE statements. If this is set to a valid class, a 'CREATE TABLE ... STORED AS ... LOCATION ...' command will " + "be equivalent to 'CREATE TABLE ... STORED BY [default.storage.handler.class] LOCATION ...'. Any STORED AS clauses will be ignored, given that STORED BY and STORED AS are " + "incompatible within the same command. Users can explicitly override the default class by issuing 'CREATE TABLE ... STORED BY [overriding.storage.handler.class] ...'"), - HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", ResultFileFormat.SEQUENCEFILE.toString(), + HIVE_QUERY_RESULT_FILEFORMAT("hive.query.result.fileformat", ResultFileFormat.SEQUENCEFILE.toString(), new StringSet(ResultFileFormat.getValidSet()), "Default file format for storing result of the query."), - HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"), + HIVE_CHECK_FILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"), // default serde for rcfile - HIVEDEFAULTRCFILESERDE("hive.default.rcfile.serde", + HIVE_DEFAULT_RCFILE_SERDE("hive.default.rcfile.serde", "org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe", "The default SerDe Hive will use for the RCFile format"), - HIVEDEFAULTSERDE("hive.default.serde", + HIVE_DEFAULT_SERDE("hive.default.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", "The default SerDe Hive will use for storage formats that do not specify a SerDe."), @@ -2090,7 +2090,7 @@ public static enum ConfVars { * @deprecated Use MetastoreConf.SERDES_USING_METASTORE_FOR_SCHEMA */ @Deprecated - SERDESUSINGMETASTOREFORSCHEMA("hive.serdes.using.metastore.for.schema", + SERDES_USING_METASTORE_FOR_SCHEMA("hive.serdes.using.metastore.for.schema", "org.apache.hadoop.hive.ql.io.orc.OrcSerde," + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe," + "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe," + @@ -2105,9 +2105,9 @@ public static enum ConfVars { HIVE_LEGACY_SCHEMA_FOR_ALL_SERDES("hive.legacy.schema.for.all.serdes", false, "A backward compatibility setting for external metastore users that do not handle \n" + - SERDESUSINGMETASTOREFORSCHEMA.varname + " correctly. This may be removed at any time."), + SERDES_USING_METASTORE_FOR_SCHEMA.varname + " correctly. This may be removed at any time."), - HIVEHISTORYFILELOC("hive.querylog.location", + HIVE_HISTORY_FILE_LOC("hive.querylog.location", "${system:java.io.tmpdir}" + File.separator + "${system:user.name}", "Location of Hive run time structured log file"), @@ -2126,26 +2126,26 @@ public static enum ConfVars { "logged less frequently than specified.\n" + "This only has an effect if hive.querylog.enable.plan.progress is set to true."), - HIVESCRIPTSERDE("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", + HIVE_SCRIPT_SERDE("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", "The default SerDe for transmitting input data to and reading output data from the user scripts. "), - HIVESCRIPTRECORDREADER("hive.script.recordreader", + HIVE_SCRIPT_RECORD_READER("hive.script.recordreader", "org.apache.hadoop.hive.ql.exec.TextRecordReader", "The default record reader for reading data from the user scripts. "), - HIVESCRIPTRECORDWRITER("hive.script.recordwriter", + HIVE_SCRIPT_RECORD_WRITER("hive.script.recordwriter", "org.apache.hadoop.hive.ql.exec.TextRecordWriter", "The default record writer for writing data to the user scripts. "), - HIVESCRIPTESCAPE("hive.transform.escape.input", false, + HIVE_SCRIPT_ESCAPE("hive.transform.escape.input", false, "This adds an option to escape special chars (newlines, carriage returns and\n" + "tabs) when they are passed to the user script. This is useful if the Hive tables\n" + "can contain data that contains special characters."), - HIVEBINARYRECORDMAX("hive.binary.record.max.length", 1000, + HIVE_BINARY_RECORD_MAX("hive.binary.record.max.length", 1000, "Read from a binary stream and treat each hive.binary.record.max.length bytes as a record. \n" + "The last record before the end of stream can have less than hive.binary.record.max.length bytes"), - HIVEHADOOPMAXMEM("hive.mapred.local.mem", 0, "mapper/reducer memory in local mode"), + HIVE_HADOOP_MAX_MEM("hive.mapred.local.mem", 0, "mapper/reducer memory in local mode"), //small table file size - HIVESMALLTABLESFILESIZE("hive.mapjoin.smalltable.filesize", 25000000L, + HIVE_SMALL_TABLES_FILESIZE("hive.mapjoin.smalltable.filesize", 25000000L, "The threshold for the input file size of the small tables; if the file size is smaller \n" + "than this threshold, it will try to convert the common join into map join"), @@ -2169,53 +2169,53 @@ public static enum ConfVars { "If true, VectorizedOrcAcidRowBatchReader will compute min/max " + "ROW__ID for the split and only load delete events in that range.\n" ), - HIVESAMPLERANDOMNUM("hive.sample.seednumber", 0, + HIVE_SAMPLE_RANDOM_NUM("hive.sample.seednumber", 0, "A number used to percentage sampling. By changing this number, user will change the subsets of data sampled."), // test mode in hive mode - HIVETESTMODE("hive.test.mode", false, + HIVE_TEST_MODE("hive.test.mode", false, "Whether Hive is running in test mode. If yes, it turns on sampling and prefixes the output tablename.", false), - HIVEEXIMTESTMODE("hive.exim.test.mode", false, + HIVE_EXIM_TEST_MODE("hive.exim.test.mode", false, "The subset of test mode that only enables custom path handling for ExIm.", false), - HIVETESTMODEPREFIX("hive.test.mode.prefix", "test_", + HIVE_TEST_MODE_PREFIX("hive.test.mode.prefix", "test_", "In test mode, specifies prefixes for the output table", false), - HIVETESTMODESAMPLEFREQ("hive.test.mode.samplefreq", 32, + HIVE_TEST_MODE_SAMPLE_FREQ("hive.test.mode.samplefreq", 32, "In test mode, specifies sampling frequency for table, which is not bucketed,\n" + "For example, the following query:\n" + " INSERT OVERWRITE TABLE dest SELECT col1 from src\n" + "would be converted to\n" + " INSERT OVERWRITE TABLE test_dest\n" + " SELECT col1 from src TABLESAMPLE (BUCKET 1 out of 32 on rand(1))", false), - HIVETESTMODENOSAMPLE("hive.test.mode.nosamplelist", "", + HIVE_TEST_MODE_NOSAMPLE("hive.test.mode.nosamplelist", "", "In test mode, specifies comma separated table names which would not apply sampling", false), - HIVETESTMODEDUMMYSTATAGGR("hive.test.dummystats.aggregator", "", "internal variable for test", false), - HIVETESTMODEDUMMYSTATPUB("hive.test.dummystats.publisher", "", "internal variable for test", false), - HIVETESTCURRENTTIMESTAMP("hive.test.currenttimestamp", null, "current timestamp for test", false), - HIVETESTMODEROLLBACKTXN("hive.test.rollbacktxn", false, "For testing only. Will mark every ACID transaction aborted", false), - HIVETESTMODEFAILCOMPACTION("hive.test.fail.compaction", false, "For testing only. Will cause CompactorMR to fail.", false), - HIVETESTMODEFAILLOADDYNAMICPARTITION("hive.test.fail.load.dynamic.partition", false, "For testing only. Will cause loadDynamicPartition to fail.", false), - HIVETESTMODEFAILHEARTBEATER("hive.test.fail.heartbeater", false, "For testing only. Will cause Heartbeater to fail.", false), + HIVE_TEST_MODE_DUMMY_STAT_AGGR("hive.test.dummystats.aggregator", "", "internal variable for test", false), + HIVE_TEST_MODE_DUMMY_STAT_PUB("hive.test.dummystats.publisher", "", "internal variable for test", false), + HIVE_TEST_CURRENT_TIMESTAMP("hive.test.currenttimestamp", null, "current timestamp for test", false), + HIVE_TEST_MODE_ROLLBACK_TXN("hive.test.rollbacktxn", false, "For testing only. Will mark every ACID transaction aborted", false), + HIVE_TEST_MODE_FAIL_COMPACTION("hive.test.fail.compaction", false, "For testing only. Will cause CompactorMR to fail.", false), + HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION("hive.test.fail.load.dynamic.partition", false, "For testing only. Will cause loadDynamicPartition to fail.", false), + HIVE_TEST_MODE_FAIL_HEARTBEATER("hive.test.fail.heartbeater", false, "For testing only. Will cause Heartbeater to fail.", false), TESTMODE_BUCKET_CODEC_VERSION("hive.test.bucketcodec.version", 1, "For testing only. Will make ACID subsystem write RecordIdentifier.bucketId in specified\n" + "format", false), HIVE_EXTEND_BUCKET_ID_RANGE("hive.extend.bucketid.range", true, "Dynamically allocate some bits from statement id when bucket id overflows. This allows having more than 4096 buckets."), - HIVETESTMODEACIDKEYIDXSKIP("hive.test.acid.key.index.skip", false, "For testing only. OrcRecordUpdater will skip " + HIVE_TEST_MODE_ACID_KEY_IDX_SKIP("hive.test.acid.key.index.skip", false, "For testing only. OrcRecordUpdater will skip " + "generation of the hive.acid.key.index", false), - HIVEMERGEMAPFILES("hive.merge.mapfiles", true, + HIVE_MERGE_MAPFILES("hive.merge.mapfiles", true, "Merge small files at the end of a map-only job"), - HIVEMERGEMAPREDFILES("hive.merge.mapredfiles", false, + HIVE_MERGE_MAPRED_FILES("hive.merge.mapredfiles", false, "Merge small files at the end of a map-reduce job"), - HIVEMERGETEZFILES("hive.merge.tezfiles", false, "Merge small files at the end of a Tez DAG"), - HIVEMERGEMAPFILESSIZE("hive.merge.size.per.task", (long) (256 * 1000 * 1000), + HIVE_MERGE_TEZ_FILES("hive.merge.tezfiles", false, "Merge small files at the end of a Tez DAG"), + HIVE_MERGE_MAP_FILES_SIZE("hive.merge.size.per.task", (long) (256 * 1000 * 1000), "Size of merged files at the end of the job"), - HIVEMERGEMAPFILESAVGSIZE("hive.merge.smallfiles.avgsize", (long) (16 * 1000 * 1000), + HIVE_MERGE_MAP_FILES_AVG_SIZE("hive.merge.smallfiles.avgsize", (long) (16 * 1000 * 1000), "When the average output file size of a job is less than this number, Hive will start an additional \n" + "map-reduce job to merge the output files into bigger files. This is only done for map-only jobs \n" + "if hive.merge.mapfiles is true, and for map-reduce jobs if hive.merge.mapredfiles is true."), - HIVEMERGERCFILEBLOCKLEVEL("hive.merge.rcfile.block.level", true, ""), - HIVEMERGEORCFILESTRIPELEVEL("hive.merge.orcfile.stripe.level", true, + HIVE_MERGE_RCFILE_BLOCK_LEVEL("hive.merge.rcfile.block.level", true, ""), + HIVE_MERGE_ORC_FILE_STRIPE_LEVEL("hive.merge.orcfile.stripe.level", true, "When hive.merge.mapfiles, hive.merge.mapredfiles or hive.merge.tezfiles is enabled\n" + "while writing a table with ORC file format, enabling this config will do stripe-level\n" + "fast merge for small ORC files. Note that enabling this config will not honor the\n" + @@ -2233,11 +2233,11 @@ public static enum ConfVars { HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY("hive.iceberg.allow.datafiles.in.table.location.only", false, "If this is set to true, then all the data files being read should be withing the table location"), - HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true, + HIVE_USE_EXPLICIT_RCFILE_HEADER("hive.exec.rcfile.use.explicit.header", true, "If this is set the header for RCFiles will simply be RCF. If this is not\n" + "set the header will be that borrowed from sequence files, e.g. SEQ- followed\n" + "by the input and output RCFile formats."), - HIVEUSERCFILESYNCCACHE("hive.exec.rcfile.use.sync.cache", true, ""), + HIVE_USE_RCFILE_SYNC_CACHE("hive.exec.rcfile.use.sync.cache", true, ""), HIVE_RCFILE_RECORD_INTERVAL("hive.io.rcfile.record.interval", Integer.MAX_VALUE, ""), HIVE_RCFILE_COLUMN_NUMBER_CONF("hive.io.rcfile.column.number.conf", 0, ""), @@ -2373,109 +2373,109 @@ public static enum ConfVars { "The default is false, which means only 'TRUE' and 'FALSE' are treated as legal\n" + "boolean literal."), - HIVESKEWJOIN("hive.optimize.skewjoin", false, + HIVE_SKEW_JOIN("hive.optimize.skewjoin", false, "Whether to enable skew join optimization. \n" + "The algorithm is as follows: At runtime, detect the keys with a large skew. Instead of\n" + "processing those keys, store them temporarily in an HDFS directory. In a follow-up map-reduce\n" + "job, process those skewed keys. The same key need not be skewed for all the tables, and so,\n" + "the follow-up map-reduce job (for the skewed keys) would be much faster, since it would be a\n" + "map-join."), - HIVEDYNAMICPARTITIONHASHJOIN("hive.optimize.dynamic.partition.hashjoin", false, + HIVE_DYNAMIC_PARTITION_HASHJOIN("hive.optimize.dynamic.partition.hashjoin", false, "Whether to enable dynamically partitioned hash join optimization. \n" + "This setting is also dependent on enabling hive.auto.convert.join"), - HIVECONVERTJOIN("hive.auto.convert.join", true, + HIVE_CONVERT_JOIN("hive.auto.convert.join", true, "Whether Hive enables the optimization about converting common join into mapjoin based on the input file size"), - HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true, + HIVE_CONVERT_JOIN_NOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true, "Whether Hive enables the optimization about converting common join into mapjoin based on the input file size. \n" + "If this parameter is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the\n" + "specified size, the join is directly converted to a mapjoin (there is no conditional task)."), HIVE_CONVERT_ANTI_JOIN("hive.auto.convert.anti.join", true, "Whether Hive enables the optimization about converting join with null filter to anti join."), - HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD("hive.auto.convert.join.noconditionaltask.size", + HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD("hive.auto.convert.join.noconditionaltask.size", 10000000L, "If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" + "However, if it is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than this size, \n" + "the join is directly converted to a mapjoin(there is no conditional task). The default is 10MB"), - HIVECONVERTJOINUSENONSTAGED("hive.auto.convert.join.use.nonstaged", false, + HIVE_CONVERT_JOIN_USE_NONSTAGED("hive.auto.convert.join.use.nonstaged", false, "For conditional joins, if input stream from a small alias can be directly applied to join operator without \n" + "filtering or projection, the alias need not to be pre-staged in distributed cache via mapred local task.\n" + "Currently, this is not working with vectorization or tez execution engine."), - HIVESKEWJOINKEY("hive.skewjoin.key", 100000, + HIVE_SKEWJOIN_KEY("hive.skewjoin.key", 100000, "Determine if we get a skew key in join. If we see more than the specified number of rows with the same key in join operator,\n" + "we think the key as a skew join key. "), - HIVESKEWJOINMAPJOINNUMMAPTASK("hive.skewjoin.mapjoin.map.tasks", 10000, + HIVE_SKEWJOIN_MAPJOIN_NUM_MAP_TASK("hive.skewjoin.mapjoin.map.tasks", 10000, "Determine the number of map task used in the follow up map join job for a skew join.\n" + "It should be used together with hive.skewjoin.mapjoin.min.split to perform a fine-grained control."), - HIVESKEWJOINMAPJOINMINSPLIT("hive.skewjoin.mapjoin.min.split", 33554432L, + HIVE_SKEWJOIN_MAPJOIN_MIN_SPLIT("hive.skewjoin.mapjoin.min.split", 33554432L, "Determine the number of map task at most used in the follow up map join job for a skew join by specifying \n" + "the minimum split size. It should be used together with hive.skewjoin.mapjoin.map.tasks to perform a fine-grained control."), - HIVESENDHEARTBEAT("hive.heartbeat.interval", 1000, + HIVE_SEND_HEARTBEAT("hive.heartbeat.interval", 1000, "Send a heartbeat after this interval - used by mapjoin and filter operators"), - HIVELIMITMAXROWSIZE("hive.limit.row.max.size", 100000L, + HIVE_LIMIT_MAX_ROW_SIZE("hive.limit.row.max.size", 100000L, "When trying a smaller subset of data for simple LIMIT, how much size we need to guarantee each row to have at least."), - HIVELIMITOPTLIMITFILE("hive.limit.optimize.limit.file", 10, - "When trying a smaller subset of data for simple LIMIT, maximum number of files we can sample."), - HIVELIMITOPTENABLE("hive.limit.optimize.enable", false, + HIVE_LIMIT_OPT_LIMIT_FILE("hive.limit.optimize.limit.file", 10, + "When trying a smaller subset of data for simple LIMIT, maximum number of fil:wq:es we can sample."), + HIVE_LIMIT_OPT_ENABLE("hive.limit.optimize.enable", false, "Whether to enable to optimization to trying a smaller subset of data for simple LIMIT first."), - HIVELIMITOPTMAXFETCH("hive.limit.optimize.fetch.max", 50000, + HIVE_LIMIT_OPT_MAX_FETCH("hive.limit.optimize.fetch.max", 50000, "Maximum number of rows allowed for a smaller subset of data for simple LIMIT, if it is a fetch query. \n" + "Insert queries are not restricted by this limit."), - HIVELIMITPUSHDOWNMEMORYUSAGE("hive.limit.pushdown.memory.usage", 0.1f, new RatioValidator(), + HIVE_LIMIT_PUSHDOWN_MEMORY_USAGE("hive.limit.pushdown.memory.usage", 0.1f, new RatioValidator(), "The fraction of available memory to be used for buffering rows in Reducesink operator for limit pushdown optimization."), - HIVECONVERTJOINMAXENTRIESHASHTABLE("hive.auto.convert.join.hashtable.max.entries", 21000000L, + HIVE_CONVERT_JOIN_MAX_ENTRIES_HASHTABLE("hive.auto.convert.join.hashtable.max.entries", 21000000L, "If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" + "However, if it is on, and the predicted number of entries in hashtable for a given join \n" + "input is larger than this number, the join will not be converted to a mapjoin. \n" + "The value \"-1\" means no limit."), - XPRODSMALLTABLEROWSTHRESHOLD("hive.xprod.mapjoin.small.table.rows", 1,"Maximum number of rows on build side" + XPROD_SMALL_TABLE_ROWS_THRESHOLD("hive.xprod.mapjoin.small.table.rows", 1,"Maximum number of rows on build side" + " of map join before it switches over to cross product edge"), - HIVECONVERTJOINMAXSHUFFLESIZE("hive.auto.convert.join.shuffle.max.size", 10000000000L, + HIVE_CONVERT_JOIN_MAX_SHUFFLE_SIZE("hive.auto.convert.join.shuffle.max.size", 10000000000L, "If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" + "However, if it is on, and the predicted size of the larger input for a given join is greater \n" + "than this number, the join will not be converted to a dynamically partitioned hash join. \n" + "The value \"-1\" means no limit."), - HIVEHASHTABLEKEYCOUNTADJUSTMENT("hive.hashtable.key.count.adjustment", 0.99f, + HIVE_HASHTABLE_KEY_COUNT_ADJUSTMENT("hive.hashtable.key.count.adjustment", 0.99f, "Adjustment to mapjoin hashtable size derived from table and column statistics; the estimate" + " of the number of keys is divided by this value. If the value is 0, statistics are not used" + "and hive.hashtable.initialCapacity is used instead."), - HIVEHASHTABLETHRESHOLD("hive.hashtable.initialCapacity", 100000, "Initial capacity of " + + HIVE_HASHTABLE_THRESHOLD("hive.hashtable.initialCapacity", 100000, "Initial capacity of " + "mapjoin hashtable if statistics are absent, or if hive.hashtable.key.count.adjustment is set to 0"), - HIVEHASHTABLELOADFACTOR("hive.hashtable.loadfactor", (float) 0.75, ""), - HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE("hive.mapjoin.followby.gby.localtask.max.memory.usage", (float) 0.55, + HIVE_HASHTABLE_LOAD_FACTOR("hive.hashtable.loadfactor", (float) 0.75, ""), + HIVE_HASHTABLE_FOLLOWBY_GBY_MAX_MEMORY_USAGE("hive.mapjoin.followby.gby.localtask.max.memory.usage", (float) 0.55, "This number means how much memory the local task can take to hold the key/value into an in-memory hash table \n" + "when this map join is followed by a group by. If the local task's memory usage is more than this number, \n" + "the local task will abort by itself. It means the data of the small table is too large " + "to be held in memory."), - HIVEHASHTABLEMAXMEMORYUSAGE("hive.mapjoin.localtask.max.memory.usage", (float) 0.90, + HIVE_HASHTABLE_MAX_MEMORY_USAGE("hive.mapjoin.localtask.max.memory.usage", (float) 0.90, "This number means how much memory the local task can take to hold the key/value into an in-memory hash table. \n" + "If the local task's memory usage is more than this number, the local task will abort by itself. \n" + "It means the data of the small table is too large to be held in memory."), - HIVEHASHTABLESCALE("hive.mapjoin.check.memory.rows", (long)100000, + HIVE_HASHTABLE_SCALE("hive.mapjoin.check.memory.rows", (long)100000, "The number means after how many rows processed it needs to check the memory usage"), - HIVEINPUTFORMAT("hive.input.format", "org.apache.hadoop.hive.ql.io.CombineHiveInputFormat", + HIVE_INPUT_FORMAT("hive.input.format", "org.apache.hadoop.hive.ql.io.CombineHiveInputFormat", "The default input format. Set this to HiveInputFormat if you encounter problems with CombineHiveInputFormat."), - HIVETEZINPUTFORMAT("hive.tez.input.format", "org.apache.hadoop.hive.ql.io.HiveInputFormat", + HIVE_TEZ_INPUT_FORMAT("hive.tez.input.format", "org.apache.hadoop.hive.ql.io.HiveInputFormat", "The default input format for tez. Tez groups splits in the AM."), - HIVETEZCONTAINERSIZE("hive.tez.container.size", -1, + HIVE_TEZ_CONTAINER_SIZE("hive.tez.container.size", -1, "By default Tez will spawn containers of the size of a mapper. This can be used to overwrite."), - HIVETEZCPUVCORES("hive.tez.cpu.vcores", -1, + HIVE_TEZ_CPU_VCORES("hive.tez.cpu.vcores", -1, "By default Tez will ask for however many cpus map-reduce is configured to use per container.\n" + "This can be used to overwrite."), - HIVETEZJAVAOPTS("hive.tez.java.opts", null, + HIVE_TEZ_JAVA_OPTS("hive.tez.java.opts", null, "By default Tez will use the Java options from map tasks. This can be used to overwrite."), - HIVETEZLOGLEVEL("hive.tez.log.level", "INFO", + HIVE_TEZ_LOG_LEVEL("hive.tez.log.level", "INFO", "The log level to use for tasks executing as part of the DAG.\n" + "Used only if hive.tez.java.opts is used to configure Java options."), - HIVETEZHS2USERACCESS("hive.tez.hs2.user.access", true, + HIVE_TEZ_HS2_USER_ACCESS("hive.tez.hs2.user.access", true, "Whether to grant access to the hs2/hive user for queries"), - HIVEQUERYNAME ("hive.query.name", null, + HIVE_QUERY_NAME("hive.query.name", null, "This named is used by Tez to set the dag name. This name in turn will appear on \n" + "the Tez UI representing the work that was done."), - HIVETEZJOBNAME("tez.job.name", "HIVE-%s", + HIVE_TEZ_JOB_NAME("tez.job.name", "HIVE-%s", "This named is used by Tez to set the job name. This name in turn will appear on \n" + "the Yarn UI representing the Yarn Application Name. And The job name may be a \n" + "Java String.format() string, to which the session ID will be supplied as the single parameter."), @@ -2491,15 +2491,15 @@ public static enum ConfVars { "This time slice should align with the flush interval of the logging library else file pruning may\n" + "incorrectly prune files leading to incorrect results from sys.logs table."), - HIVEOPTIMIZEBUCKETINGSORTING("hive.optimize.bucketingsorting", true, + HIVE_OPTIMIZE_BUCKETING_SORTING("hive.optimize.bucketingsorting", true, "Don't create a reducer for enforcing \n" + "bucketing/sorting for queries of the form: \n" + "insert overwrite table T2 select * from T1;\n" + "where T1 and T2 are bucketed/sorted by the same keys into the same number of buckets."), - HIVEPARTITIONER("hive.mapred.partitioner", "org.apache.hadoop.hive.ql.io.DefaultHivePartitioner", ""), - HIVEENFORCESORTMERGEBUCKETMAPJOIN("hive.enforce.sortmergebucketmapjoin", false, + HIVE_PARTITIONER("hive.mapred.partitioner", "org.apache.hadoop.hive.ql.io.DefaultHivePartitioner", ""), + HIVE_ENFORCE_SORT_MERGE_BUCKET_MAPJOIN("hive.enforce.sortmergebucketmapjoin", false, "If the user asked for sort-merge bucketed map-side join, and it cannot be performed, should the query fail or not ?"), - HIVEENFORCEBUCKETMAPJOIN("hive.enforce.bucketmapjoin", false, + HIVE_ENFORCE_BUCKET_MAPJOIN("hive.enforce.bucketmapjoin", false, "If the user asked for bucketed map-side join, and it cannot be performed, \n" + "should the query fail or not ? For example, if the buckets in the tables being joined are\n" + "not a multiple of each other, bucketed map-side join cannot be performed, and the\n" + @@ -2539,24 +2539,24 @@ public static enum ConfVars { "with few files (10 files), the sort-merge join will only use 10 mappers, and a simple map-only join might be faster\n" + "if the complete small table can fit in memory, and a map-join can be performed."), - HIVESCRIPTOPERATORTRUST("hive.exec.script.trust", false, ""), - HIVEROWOFFSET("hive.exec.rowoffset", false, + HIVE_SCRIPT_OPERATOR_TRUST("hive.exec.script.trust", false, ""), + HIVE_ROW_OFFSET("hive.exec.rowoffset", false, "Whether to provide the row offset virtual column"), // Optimizer - HIVEOPTINDEXFILTER("hive.optimize.index.filter", true, "Whether to enable automatic use of indexes"), + HIVE_OPT_INDEX_FILTER("hive.optimize.index.filter", true, "Whether to enable automatic use of indexes"), - HIVEOPTPPD("hive.optimize.ppd", true, + HIVE_OPT_PPD("hive.optimize.ppd", true, "Whether to enable predicate pushdown"), - HIVEOPTPPD_WINDOWING("hive.optimize.ppd.windowing", true, + HIVE_OPT_PPD_WINDOWING("hive.optimize.ppd.windowing", true, "Whether to enable predicate pushdown through windowing"), - HIVEPPDRECOGNIZETRANSITIVITY("hive.ppd.recognizetransivity", true, + HIVE_PPD_RECOGNIZE_TRANSITIVITY("hive.ppd.recognizetransivity", true, "Whether to transitively replicate predicate filters over equijoin conditions."), - HIVEPPD_RECOGNIZE_COLUMN_EQUALITIES("hive.ppd.recognize.column.equalities", true, + HIVE_PPD_RECOGNIZE_COLUMN_EQUALITIES("hive.ppd.recognize.column.equalities", true, "Whether we should traverse the join branches to discover transitive propagation opportunities over" + " equijoin conditions. \n" + "Requires hive.ppd.recognizetransivity to be set to true."), - HIVEPPDREMOVEDUPLICATEFILTERS("hive.ppd.remove.duplicatefilters", true, + HIVE_PPD_REMOVE_DUPLICATE_FILTERS("hive.ppd.remove.duplicatefilters", true, "During query optimization, filters may be pushed down in the operator tree. \n" + "If this config is true only pushed down filters remain in the operator tree, \n" + "and the original filter is removed. If this config is false, the original filter \n" + @@ -2565,44 +2565,44 @@ public static enum ConfVars { true, "Whether to transitively infer disjunctive predicates across joins. \n" + "Disjunctive predicates are hard to simplify and pushing them down might lead to infinite rule matching " + "causing stackoverflow and OOM errors"), - HIVEPOINTLOOKUPOPTIMIZER("hive.optimize.point.lookup", true, + HIVE_POINT_LOOKUP_OPTIMIZER("hive.optimize.point.lookup", true, "Whether to transform OR clauses in Filter operators into IN clauses"), - HIVEPOINTLOOKUPOPTIMIZERMIN("hive.optimize.point.lookup.min", 2, + HIVE_POINT_LOOKUP_OPTIMIZER_MIN("hive.optimize.point.lookup.min", 2, "Minimum number of OR clauses needed to transform into IN clauses"), HIVEOPT_TRANSFORM_IN_MAXNODES("hive.optimize.transform.in.maxnodes", 16, "Maximum number of IN expressions beyond which IN will not be transformed into OR clause"), - HIVECOUNTDISTINCTOPTIMIZER("hive.optimize.countdistinct", true, + HIVE_COUNT_DISTINCT_OPTIMIZER("hive.optimize.countdistinct", true, "Whether to transform count distinct into two stages"), - HIVEPARTITIONCOLUMNSEPARATOR("hive.optimize.partition.columns.separate", true, + HIVE_PARTITION_COLUMN_SEPARATOR("hive.optimize.partition.columns.separate", true, "Extract partition columns from IN clauses"), // Constant propagation optimizer - HIVEOPTCONSTANTPROPAGATION("hive.optimize.constant.propagation", true, "Whether to enable constant propagation optimizer"), - HIVEIDENTITYPROJECTREMOVER("hive.optimize.remove.identity.project", true, "Removes identity project from operator tree"), - HIVEMETADATAONLYQUERIES("hive.optimize.metadataonly", false, + HIVE_OPT_CONSTANT_PROPAGATION("hive.optimize.constant.propagation", true, "Whether to enable constant propagation optimizer"), + HIVE_IDENTITY_PROJECT_REMOVER("hive.optimize.remove.identity.project", true, "Removes identity project from operator tree"), + HIVE_METADATA_ONLY_QUERIES("hive.optimize.metadataonly", false, "Whether to eliminate scans of the tables from which no columns are selected. Note\n" + "that, when selecting from empty tables with data files, this can produce incorrect\n" + "results, so it's disabled by default. It works correctly for normal tables."), - HIVENULLSCANOPTIMIZE("hive.optimize.null.scan", true, "Don't scan relations which are guaranteed to not generate any rows"), - HIVEOPTPPD_STORAGE("hive.optimize.ppd.storage", true, + HIVE_NULL_SCAN_OPTIMIZE("hive.optimize.null.scan", true, "Don't scan relations which are guaranteed to not generate any rows"), + HIVE_OPT_PPD_STORAGE("hive.optimize.ppd.storage", true, "Whether to push predicates down to storage handlers"), - HIVEOPTGROUPBY("hive.optimize.groupby", true, + HIVE_OPT_GROUPBY("hive.optimize.groupby", true, "Whether to enable the bucketed group by from bucketed partitions/tables."), - HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false, + HIVE_OPT_BUCKET_MAPJOIN("hive.optimize.bucketmapjoin", false, "Whether to try bucket mapjoin"), - HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false, + HIVE_OPT_SORT_MERGE_BUCKET_MAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false, "Whether to try sorted bucket merge map join"), - HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true, + HIVE_OPT_REDUCE_DEDUPLICATION("hive.optimize.reducededuplication", true, "Remove extra map-reduce jobs if the data is already clustered by the same key which needs to be used again. \n" + "This should always be set to true. Since it is a new feature, it has been made configurable."), - HIVEOPTREDUCEDEDUPLICATIONMINREDUCER("hive.optimize.reducededuplication.min.reducer", 4, + HIVE_OPT_REDUCE_DEDUPLICATION_MIN_REDUCER("hive.optimize.reducededuplication.min.reducer", 4, "Reduce deduplication merges two RSs by moving key/parts/reducer-num of the child RS to parent RS. \n" + "That means if reducer-num of the child RS is fixed (order by or forced bucketing) and small, it can make very slow, single MR.\n" + "The optimization will be automatically disabled if number of reducers would be less than specified value."), - HIVEOPTJOINREDUCEDEDUPLICATION("hive.optimize.joinreducededuplication", true, + HIVE_OPT_JOIN_REDUCE_DEDUPLICATION("hive.optimize.joinreducededuplication", true, "Remove extra shuffle/sorting operations after join algorithm selection has been executed. \n" + "Currently it only works with Apache Tez. This should always be set to true. \n" + "Since it is a new feature, it has been made configurable."), - HIVEOPTSORTDYNAMICPARTITIONTHRESHOLD("hive.optimize.sort.dynamic.partition.threshold", 0, + HIVE_OPT_SORT_DYNAMIC_PARTITION_THRESHOLD("hive.optimize.sort.dynamic.partition.threshold", 0, "When enabled dynamic partitioning column will be globally sorted.\n" + "This way we can keep only one record writer open for each partition value\n" + "in the reducer thereby reducing the memory pressure on reducers.\n" + @@ -2613,13 +2613,13 @@ public static enum ConfVars { "Setting it to any other positive integer will make Hive use this as threshold for number of writers."), - HIVESAMPLINGFORORDERBY("hive.optimize.sampling.orderby", false, "Uses sampling on order-by clause for parallel execution."), - HIVESAMPLINGNUMBERFORORDERBY("hive.optimize.sampling.orderby.number", 1000, "Total number of samples to be obtained."), - HIVESAMPLINGPERCENTFORORDERBY("hive.optimize.sampling.orderby.percent", 0.1f, new RatioValidator(), + HIVE_SAMPLING_FOR_ORDERBY("hive.optimize.sampling.orderby", false, "Uses sampling on order-by clause for parallel execution."), + HIVE_SAMPLING_NUMBER_FOR_ORDERBY("hive.optimize.sampling.orderby.number", 1000, "Total number of samples to be obtained."), + HIVE_SAMPLING_PERCENT_FOR_ORDERBY("hive.optimize.sampling.orderby.percent", 0.1f, new RatioValidator(), "Probability with which a row will be chosen."), HIVE_REMOVE_ORDERBY_IN_SUBQUERY("hive.remove.orderby.in.subquery", true, "If set to true, order/sort by without limit in sub queries will be removed."), - HIVEOPTIMIZEDISTINCTREWRITE("hive.optimize.distinct.rewrite", true, "When applicable this " + HIVE_OPTIMIZE_DISTINCT_REWRITE("hive.optimize.distinct.rewrite", true, "When applicable this " + "optimization rewrites distinct aggregates from a single stage to multi-stage " + "aggregation. This may not be optimal in all cases. Ideally, whether to trigger it or " + "not should be cost based decision. Until Hive formalizes cost model for this, this is config driven."), @@ -2640,7 +2640,7 @@ public static enum ConfVars { "By default, when writing data into a table and UNION ALL is the last step of the query, Hive on Tez will\n" + "create a subdirectory for each branch of the UNION ALL. When this property is enabled,\n" + "the subdirectories are removed, and the files are renamed and moved to the parent directory"), - HIVEOPTCORRELATION("hive.optimize.correlation", false, "exploit intra-query correlations."), + HIVE_OPT_CORRELATION("hive.optimize.correlation", false, "exploit intra-query correlations."), HIVE_OPTIMIZE_LIMIT_TRANSPOSE("hive.optimize.limittranspose", false, "Whether to push a limit through left/right outer join or union. If the value is true and the size of the outer\n" + @@ -3549,9 +3549,9 @@ public static enum ConfVars { "This value is only used for a given table if the kudu.master_addresses table property is not set."), // For har files - HIVEARCHIVEENABLED("hive.archive.enabled", false, "Whether archiving operations are permitted"), + HIVE_ARCHIVE_ENABLED("hive.archive.enabled", false, "Whether archiving operations are permitted"), - HIVEFETCHTASKCONVERSION("hive.fetch.task.conversion", "more", new StringSet("none", "minimal", "more"), + HIVE_FETCH_TASK_CONVERSION("hive.fetch.task.conversion", "more", new StringSet("none", "minimal", "more"), "Some select queries can be converted to single FETCH task minimizing latency.\n" + "Currently the query should be single sourced not having any subquery and should not have\n" + "any aggregations or distincts (which incurs RS), lateral views and joins.\n" + @@ -3559,30 +3559,30 @@ public static enum ConfVars { "1. minimal : SELECT STAR, FILTER on partition columns, LIMIT only\n" + "2. more : SELECT, FILTER, LIMIT only (support TABLESAMPLE and virtual columns)" ), - HIVEFETCHTASKCACHING("hive.fetch.task.caching", true, + HIVE_FETCH_TASK_CACHING("hive.fetch.task.caching", true, "Enabling the caching of the result of fetch tasks eliminates the chance of running into a failing read." + " On the other hand, if enabled, the hive.fetch.task.conversion.threshold must be adjusted accordingly. That" + " is 200MB by default which must be lowered in case of enabled caching to prevent the consumption of too much memory."), - HIVEFETCHTASKCONVERSIONTHRESHOLD("hive.fetch.task.conversion.threshold", 209715200L, + HIVE_FETCH_TASK_CONVERSION_THRESHOLD("hive.fetch.task.conversion.threshold", 209715200L, "Input threshold for applying hive.fetch.task.conversion. If target table is native, input length\n" + "is calculated by summation of file lengths. If it's not native, storage handler for the table\n" + "can optionally implement org.apache.hadoop.hive.ql.metadata.InputEstimator interface."), - HIVEFETCHTASKAGGR("hive.fetch.task.aggr", false, + HIVE_FETCH_TASK_AGGR("hive.fetch.task.aggr", false, "Aggregation queries with no group-by clause (for example, select count(*) from src) execute\n" + "final aggregations in single reduce task. If this is set true, Hive delegates final aggregation\n" + "stage to fetch task, possibly decreasing the query time."), - HIVEOPTIMIZEMETADATAQUERIES("hive.compute.query.using.stats", true, + HIVE_OPTIMIZE_METADATA_QUERIES("hive.compute.query.using.stats", true, "When set to true Hive will answer a few queries like count(1) purely using stats\n" + "stored in metastore. For basic stats collection turn on the config hive.stats.autogather to true.\n" + "For more advanced stats collection need to run analyze table queries."), // Serde for FetchTask - HIVEFETCHOUTPUTSERDE("hive.fetch.output.serde", "org.apache.hadoop.hive.serde2.DelimitedJSONSerDe", + HIVE_FETCH_OUTPUT_SERDE("hive.fetch.output.serde", "org.apache.hadoop.hive.serde2.DelimitedJSONSerDe", "The SerDe used by FetchTask to serialize the fetch output."), - HIVEEXPREVALUATIONCACHE("hive.cache.expr.evaluation", true, + HIVE_EXPR_EVALUATION_CACHE("hive.cache.expr.evaluation", true, "If true, the evaluation result of a deterministic expression referenced twice or more\n" + "will be cached.\n" + "For example, in a filter condition like '.. where key + 10 = 100 or key + 10 = 0'\n" + @@ -3591,12 +3591,12 @@ public static enum ConfVars { "or filter operators."), // Hive Variables - HIVEVARIABLESUBSTITUTE("hive.variable.substitute", true, + HIVE_VARIABLE_SUBSTITUTE("hive.variable.substitute", true, "This enables substitution using syntax like ${var} ${system:var} and ${env:var}."), - HIVEVARIABLESUBSTITUTEDEPTH("hive.variable.substitute.depth", 40, + HIVE_VARIABLE_SUBSTITUTE_DEPTH("hive.variable.substitute.depth", 40, "The maximum replacements the substitution engine will do."), - HIVECONFVALIDATION("hive.conf.validation", true, + HIVE_CONF_VALIDATION("hive.conf.validation", true, "Enables type checking for registered Hive configurations"), SEMANTIC_ANALYZER_HOOK("hive.semantic.analyzer.hook", "", ""), @@ -4551,7 +4551,7 @@ public static enum ConfVars { " it will now take 512 reducers, similarly if the max number of reducers is 511,\n" + " and a job was going to use this many, it will now use 256 reducers."), - HIVEOPTLISTBUCKETING("hive.optimize.listbucketing", false, + HIVE_OPT_LIST_BUCKETING("hive.optimize.listbucketing", false, "Enable list bucketing optimizer. Default value is false so that we disable it by default."), // Allow TCP Keep alive socket option for for HiveServer or a maximum timeout for the socket. @@ -4741,11 +4741,11 @@ public static enum ConfVars { "Whether to generate consistent split locations when generating splits in the AM"), HIVE_PREWARM_ENABLED("hive.prewarm.enabled", false, "Enables container prewarm for Tez(Hadoop 2 only)"), HIVE_PREWARM_NUM_CONTAINERS("hive.prewarm.numcontainers", 10, "Controls the number of containers to prewarm for Tez (Hadoop 2 only)"), - HIVESTAGEIDREARRANGE("hive.stageid.rearrange", "none", new StringSet("none", "idonly", "traverse", "execution"), ""), - HIVEEXPLAINDEPENDENCYAPPENDTASKTYPES("hive.explain.dependency.append.tasktype", false, ""), - HIVEUSEGOOGLEREGEXENGINE("hive.use.googleregex.engine",false,"whether to use google regex engine or not, default regex engine is java.util.regex"), + HIVE_STAGE_ID_REARRANGE("hive.stageid.rearrange", "none", new StringSet("none", "idonly", "traverse", "execution"), ""), + HIVE_EXPLAIN_DEPENDENCY_APPEND_TASK_TYPES("hive.explain.dependency.append.tasktype", false, ""), + HIVE_USE_GOOGLE_REGEX_ENGINE("hive.use.googleregex.engine",false,"whether to use google regex engine or not, default regex engine is java.util.regex"), - HIVECOUNTERGROUP("hive.counters.group.name", "HIVE", + HIVE_COUNTER_GROUP("hive.counters.group.name", "HIVE", "The name of counter group for internal Hive variables (CREATED_FILE, FATAL_ERROR, etc.)"), HIVE_QUOTEDID_SUPPORT("hive.support.quoted.identifiers", "column", @@ -5497,7 +5497,7 @@ public static enum ConfVars { "validated for all SQL operations after every defined interval (default: 500ms) and corresponding action\n" + "defined in the trigger will be taken"), - NWAYJOINREORDER("hive.reorder.nway.joins", true, + N_WAY_JOIN_REORDER("hive.reorder.nway.joins", true, "Runs reordering of tables within single n-way join (i.e.: picks streamtable)"), HIVE_MERGE_NWAY_JOINS("hive.merge.nway.joins", false, "Merge adjacent joins into a single n-way join"), @@ -5583,7 +5583,7 @@ public static enum ConfVars { "hive.rewrite.data.policy", "Comma separated list of configuration options which are immutable at runtime"), HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list", - METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname + METASTORE_PWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname + "," + HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PASSWORD.varname + "," + DRUID_METADATA_DB_PASSWORD.varname // Adding the S3 credentials from Hadoop config to be hidden @@ -6366,11 +6366,11 @@ public String getQueryString() { } public static String getQueryString(Configuration conf) { - return getVar(conf, ConfVars.HIVEQUERYSTRING, EncoderDecoderFactory.URL_ENCODER_DECODER); + return getVar(conf, ConfVars.HIVE_QUERY_STRING, EncoderDecoderFactory.URL_ENCODER_DECODER); } public static String getQueryId(Configuration conf) { - return getVar(conf, HiveConf.ConfVars.HIVEQUERYID, ""); + return getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID, ""); } public void setQueryString(String query) { @@ -6378,7 +6378,7 @@ public void setQueryString(String query) { } public static void setQueryString(Configuration conf, String query) { - setVar(conf, ConfVars.HIVEQUERYSTRING, query, EncoderDecoderFactory.URL_ENCODER_DECODER); + setVar(conf, ConfVars.HIVE_QUERY_STRING, query, EncoderDecoderFactory.URL_ENCODER_DECODER); } public void logVars(PrintStream ps) { for (ConfVars one : ConfVars.values()) { @@ -6482,7 +6482,7 @@ private void initialize(Class cls) { // if embedded metastore is to be used as per config so far // then this is considered like the metastore server case - String msUri = this.getVar(HiveConf.ConfVars.METASTOREURIS); + String msUri = this.getVar(HiveConf.ConfVars.METASTORE_URIS); // This is hackery, but having hive-common depend on standalone-metastore is really bad // because it will pull all of the metastore code into every module. We need to check that // we aren't using the standalone metastore. If we are, we should treat it the same as a @@ -6539,18 +6539,18 @@ private void initialize(Class cls) { // if the running class was loaded directly (through eclipse) rather than through a // jar then this would be needed if (hiveJar == null) { - hiveJar = this.get(ConfVars.HIVEJAR.varname); + hiveJar = this.get(ConfVars.HIVE_JAR.varname); } if (auxJars == null) { - auxJars = StringUtils.join(FileUtils.getJarFilesByPath(this.get(ConfVars.HIVEAUXJARS.varname), this), ','); + auxJars = StringUtils.join(FileUtils.getJarFilesByPath(this.get(ConfVars.HIVE_AUX_JARS.varname), this), ','); } if (getBoolVar(ConfVars.METASTORE_SCHEMA_VERIFICATION)) { setBoolVar(ConfVars.METASTORE_AUTO_CREATE_ALL, false); } - if (getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) { + if (getBoolVar(HiveConf.ConfVars.HIVE_CONF_VALIDATION)) { List trimmed = new ArrayList(); for (Map.Entry entry : this) { String key = entry.getKey(); @@ -6633,28 +6633,28 @@ private static String[] convertVarsToRegex(String[] paramList) { */ private static final String[] SQL_STD_AUTH_SAFE_VAR_NAMES = new String[] { ConfVars.AGGR_JOIN_TRANSPOSE.varname, - ConfVars.BYTESPERREDUCER.varname, + ConfVars.BYTES_PER_REDUCER.varname, ConfVars.CLIENT_STATS_COUNTERS.varname, ConfVars.CREATE_TABLES_AS_ACID.varname, ConfVars.CREATE_TABLE_AS_EXTERNAL.varname, - ConfVars.DEFAULTPARTITIONNAME.varname, + ConfVars.DEFAULT_PARTITION_NAME.varname, ConfVars.DROP_IGNORES_NON_EXISTENT.varname, - ConfVars.HIVECOUNTERGROUP.varname, - ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT.varname, - ConfVars.HIVEENFORCEBUCKETMAPJOIN.varname, - ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN.varname, - ConfVars.HIVEEXPREVALUATIONCACHE.varname, - ConfVars.HIVEQUERYRESULTFILEFORMAT.varname, - ConfVars.HIVEHASHTABLELOADFACTOR.varname, - ConfVars.HIVEHASHTABLETHRESHOLD.varname, - ConfVars.HIVEIGNOREMAPJOINHINT.varname, - ConfVars.HIVELIMITMAXROWSIZE.varname, - ConfVars.HIVEMAPREDMODE.varname, - ConfVars.HIVEMAPSIDEAGGREGATE.varname, - ConfVars.HIVEOPTIMIZEMETADATAQUERIES.varname, - ConfVars.HIVEROWOFFSET.varname, - ConfVars.HIVEVARIABLESUBSTITUTE.varname, - ConfVars.HIVEVARIABLESUBSTITUTEDEPTH.varname, + ConfVars.HIVE_COUNTER_GROUP.varname, + ConfVars.HIVE_DEFAULT_MANAGED_FILEFORMAT.varname, + ConfVars.HIVE_ENFORCE_BUCKET_MAPJOIN.varname, + ConfVars.HIVE_ENFORCE_SORT_MERGE_BUCKET_MAPJOIN.varname, + ConfVars.HIVE_EXPR_EVALUATION_CACHE.varname, + ConfVars.HIVE_QUERY_RESULT_FILEFORMAT.varname, + ConfVars.HIVE_HASHTABLE_LOAD_FACTOR.varname, + ConfVars.HIVE_HASHTABLE_THRESHOLD.varname, + ConfVars.HIVE_IGNORE_MAPJOIN_HINT.varname, + ConfVars.HIVE_LIMIT_MAX_ROW_SIZE.varname, + ConfVars.HIVE_MAPRED_MODE.varname, + ConfVars.HIVE_MAPSIDE_AGGREGATE.varname, + ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES.varname, + ConfVars.HIVE_ROW_OFFSET.varname, + ConfVars.HIVE_VARIABLE_SUBSTITUTE.varname, + ConfVars.HIVE_VARIABLE_SUBSTITUTE_DEPTH.varname, ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_INCLUDEFUNCNAME.varname, ConfVars.HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL.varname, ConfVars.HIVE_CHECK_CROSS_PRODUCT.varname, @@ -6695,14 +6695,14 @@ private static String[] convertVarsToRegex(String[] paramList) { ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS.varname, ConfVars.LLAP_ENABLE_GRACE_JOIN_IN_LLAP.varname, ConfVars.LLAP_ALLOW_PERMANENT_FNS.varname, - ConfVars.MAXCREATEDFILES.varname, - ConfVars.MAXREDUCERS.varname, - ConfVars.NWAYJOINREORDER.varname, + ConfVars.MAX_CREATED_FILES.varname, + ConfVars.MAX_REDUCERS.varname, + ConfVars.N_WAY_JOIN_REORDER.varname, ConfVars.OUTPUT_FILE_EXTENSION.varname, ConfVars.SHOW_JOB_FAIL_DEBUG_INFO.varname, ConfVars.TASKLOG_DEBUG_TIMEOUT.varname, - ConfVars.HIVEQUERYID.varname, - ConfVars.HIVEQUERYTAG.varname, + ConfVars.HIVE_QUERY_ID.varname, + ConfVars.HIVE_QUERY_TAG.varname, }; /** @@ -6892,7 +6892,7 @@ public String getAuxJars() { */ public void setAuxJars(String auxJars) { this.auxJars = auxJars; - setVar(this, ConfVars.HIVEAUXJARS, auxJars); + setVar(this, ConfVars.HIVE_AUX_JARS, auxJars); } public URL getHiveDefaultLocation() { @@ -7074,7 +7074,7 @@ public static class StrictChecks { private static String makeMessage(String what, ConfVars setting) { return what + " are disabled for safety reasons. If you know what you are doing, please set " - + setting.varname + " to false and make sure that " + ConfVars.HIVEMAPREDMODE.varname + + + setting.varname + " to false and make sure that " + ConfVars.HIVE_MAPRED_MODE.varname + " is not set to 'strict' to proceed. Note that you may get errors or incorrect " + "results if you make a mistake while using some of the unsafe features."; } @@ -7107,7 +7107,7 @@ public static void checkOffsetWithoutOrderBy(Configuration conf) throws Semantic } private static boolean isAllowed(Configuration conf, ConfVars setting) { - String mode = HiveConf.getVar(conf, ConfVars.HIVEMAPREDMODE, (String)null); + String mode = HiveConf.getVar(conf, ConfVars.HIVE_MAPRED_MODE, (String)null); return (mode != null) ? !"strict".equals(mode) : !HiveConf.getBoolVar(conf, setting); } } diff --git a/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java b/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java index 973b9acae278..4ea28227628a 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java +++ b/common/src/java/org/apache/hadoop/hive/conf/VariableSubstitution.java @@ -59,12 +59,12 @@ public String substitute(HiveConf conf, String expr) { if (expr == null) { return expr; } - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEVARIABLESUBSTITUTE)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_VARIABLE_SUBSTITUTE)) { l4j.debug("Substitution is on: " + expr); } else { return expr; } - int depth = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVEVARIABLESUBSTITUTEDEPTH); + int depth = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_VARIABLE_SUBSTITUTE_DEPTH); return substitute(conf, expr, depth); } } diff --git a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 0968227fff01..4d94e6dae5aa 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -421,7 +421,7 @@ public enum ErrorMsg { REPLACE_UNSUPPORTED_TYPE_CONVERSION(10314, "Replacing columns with unsupported type conversion (from {0} to {1}) for column {2}. SerDe may be incompatible", true), HIVE_GROUPING_SETS_AGGR_NOMAPAGGR_MULTIGBY(10315, "Grouping sets aggregations (with rollups or cubes) are not allowed when " + - "HIVEMULTIGROUPBYSINGLEREDUCER is turned on. Set hive.multigroupby.singlereducer=false if you want to use grouping sets"), + "HIVE_MULTI_GROUPBY_SINGLE_REDUCER is turned on. Set hive.multigroupby.singlereducer=false if you want to use grouping sets"), CANNOT_RETRIEVE_TABLE_METADATA(10316, "Error while retrieving table metadata"), INVALID_AST_TREE(10318, "Internal error : Invalid AST"), ERROR_SERIALIZE_METASTORE(10319, "Error while serializing the metastore objects"), diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java index bff79a98faa5..0e8e4c35fc5d 100644 --- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java +++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java @@ -71,19 +71,19 @@ public void testConfProperties() throws Exception { // checkHiveConf(ConfVars.HADOOPFS.varname, "core-site.xml"); // Make sure non-null-valued ConfVar properties *do* override the Hadoop Configuration - checkHadoopConf(ConfVars.HADOOPNUMREDUCERS.varname, "1"); - checkConfVar(ConfVars.HADOOPNUMREDUCERS, "-1"); - checkHiveConf(ConfVars.HADOOPNUMREDUCERS.varname, "-1"); + checkHadoopConf(ConfVars.HADOOP_NUM_REDUCERS.varname, "1"); + checkConfVar(ConfVars.HADOOP_NUM_REDUCERS, "-1"); + checkHiveConf(ConfVars.HADOOP_NUM_REDUCERS.varname, "-1"); // Non-null ConfVar only defined in ConfVars - checkHadoopConf(ConfVars.HIVESKEWJOINKEY.varname, null); - checkConfVar(ConfVars.HIVESKEWJOINKEY, "100000"); - checkHiveConf(ConfVars.HIVESKEWJOINKEY.varname, "100000"); + checkHadoopConf(ConfVars.HIVE_SKEWJOIN_KEY.varname, null); + checkConfVar(ConfVars.HIVE_SKEWJOIN_KEY, "100000"); + checkHiveConf(ConfVars.HIVE_SKEWJOIN_KEY.varname, "100000"); // ConfVar overridden in in hive-site.xml - checkHadoopConf(ConfVars.HIVETESTMODEDUMMYSTATAGGR.varname, null); - checkConfVar(ConfVars.HIVETESTMODEDUMMYSTATAGGR, ""); - checkHiveConf(ConfVars.HIVETESTMODEDUMMYSTATAGGR.varname, "value2"); + checkHadoopConf(ConfVars.HIVE_TEST_MODE_DUMMY_STAT_AGGR.varname, null); + checkConfVar(ConfVars.HIVE_TEST_MODE_DUMMY_STAT_AGGR, ""); + checkHiveConf(ConfVars.HIVE_TEST_MODE_DUMMY_STAT_AGGR.varname, "value2"); //Property defined for hive masking algorithm checkConfVar(ConfVars.HIVE_MASKING_ALGO, "sha256"); @@ -94,7 +94,7 @@ public void testConfProperties() throws Exception { checkHiveConf("test.property1", "value1"); // Test HiveConf property variable substitution in hive-site.xml - checkHiveConf("test.var.hiveconf.property", ConfVars.DEFAULTPARTITIONNAME.getDefaultValue()); + checkHiveConf("test.var.hiveconf.property", ConfVars.DEFAULT_PARTITION_NAME.getDefaultValue()); } @Test @@ -150,7 +150,7 @@ public void testHiddenConfig() throws Exception { } ArrayList hiddenList = Lists.newArrayList( - HiveConf.ConfVars.METASTOREPWD.varname, + HiveConf.ConfVars.METASTORE_PWD.varname, HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname, HiveConf.ConfVars.HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PASSWORD.varname, "fs.s3.awsSecretAccessKey", @@ -196,7 +196,7 @@ public void testEncodingDecoding() throws UnsupportedEncodingException { HiveConf conf = new HiveConf(); String query = "select blah, '\u0001' from random_table"; conf.setQueryString(query); - Assert.assertEquals(URLEncoder.encode(query, "UTF-8"), conf.get(ConfVars.HIVEQUERYSTRING.varname)); + Assert.assertEquals(URLEncoder.encode(query, "UTF-8"), conf.get(ConfVars.HIVE_QUERY_STRING.varname)); Assert.assertEquals(query, conf.getQueryString()); } diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java index 1d0beaf58678..76b825b26f5c 100644 --- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java +++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java @@ -36,7 +36,7 @@ public class TestHiveConfRestrictList { public void setUp() throws Exception { System.setProperty(ConfVars.HIVE_CONF_RESTRICTED_LIST.varname, - ConfVars.HIVETESTMODEPREFIX.varname); + ConfVars.HIVE_TEST_MODE_PREFIX.varname); conf = new HiveConf(); } @@ -46,7 +46,7 @@ public void setUp() throws Exception { */ @Test public void testRestriction() throws Exception { - verifyRestriction(ConfVars.HIVETESTMODEPREFIX.varname, "foo"); + verifyRestriction(ConfVars.HIVE_TEST_MODE_PREFIX.varname, "foo"); conf.verifyAndSet(ConfVars.HIVE_AM_SPLIT_GENERATION.varname, "false"); } @@ -56,7 +56,7 @@ public void testRestriction() throws Exception { */ @Test public void testMultipleRestrictions() throws Exception { - verifyRestriction(ConfVars.HIVETESTMODEPREFIX.varname, "foo"); + verifyRestriction(ConfVars.HIVE_TEST_MODE_PREFIX.varname, "foo"); verifyRestriction(ConfVars.HIVE_IN_TEST.varname, "true"); } @@ -75,25 +75,25 @@ public void testRestrictList() throws Exception { */ @Test public void testAppendRestriction() throws Exception { - String appendListStr = ConfVars.SCRATCHDIR.varname + "," + - ConfVars.LOCALSCRATCHDIR.varname + "," + - ConfVars.METASTOREURIS.varname; + String appendListStr = ConfVars.SCRATCH_DIR.varname + "," + + ConfVars.LOCAL_SCRATCH_DIR.varname + "," + + ConfVars.METASTORE_URIS.varname; conf.addToRestrictList(appendListStr); // check if the new configs are added to HIVE_CONF_RESTRICTED_LIST String newRestrictList = conf.getVar(ConfVars.HIVE_CONF_RESTRICTED_LIST); - assertTrue(newRestrictList.contains(ConfVars.SCRATCHDIR.varname)); - assertTrue(newRestrictList.contains(ConfVars.LOCALSCRATCHDIR.varname)); - assertTrue(newRestrictList.contains(ConfVars.METASTOREURIS.varname)); + assertTrue(newRestrictList.contains(ConfVars.SCRATCH_DIR.varname)); + assertTrue(newRestrictList.contains(ConfVars.LOCAL_SCRATCH_DIR.varname)); + assertTrue(newRestrictList.contains(ConfVars.METASTORE_URIS.varname)); // check if the old values are still there in HIVE_CONF_RESTRICTED_LIST - assertTrue(newRestrictList.contains(ConfVars.HIVETESTMODEPREFIX.varname)); + assertTrue(newRestrictList.contains(ConfVars.HIVE_TEST_MODE_PREFIX.varname)); // verify that the new configs are in effect - verifyRestriction(ConfVars.HIVETESTMODEPREFIX.varname, "foo"); + verifyRestriction(ConfVars.HIVE_TEST_MODE_PREFIX.varname, "foo"); verifyRestriction(ConfVars.HIVE_CONF_RESTRICTED_LIST.varname, "foo"); - verifyRestriction(ConfVars.LOCALSCRATCHDIR.varname, "foo"); - verifyRestriction(ConfVars.METASTOREURIS.varname, "foo"); + verifyRestriction(ConfVars.LOCAL_SCRATCH_DIR.varname, "foo"); + verifyRestriction(ConfVars.METASTORE_URIS.varname, "foo"); } private void verifyRestriction(String varName, String newVal) { diff --git a/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/SampleURLHook.java b/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/SampleURLHook.java index 7e09faf44b8e..4a553d5f9df9 100644 --- a/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/SampleURLHook.java +++ b/contrib/src/java/org/apache/hadoop/hive/contrib/metastore/hooks/SampleURLHook.java @@ -33,7 +33,7 @@ public class SampleURLHook implements JDOConnectionURLHook { @Override public String getJdoConnectionUrl(Configuration conf) throws Exception { if (originalUrl == null) { - originalUrl = conf.get(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, ""); + originalUrl = conf.get(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, ""); return "jdbc:derby:;databaseName=target/tmp/junit_metastore_db_blank;create=true"; } else { return originalUrl; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java index cd35ebd81f71..656fa40c03fa 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java @@ -819,7 +819,7 @@ private List fetchSegmentsMetadata(Path segmentDescriptorDir) throw private String getUniqueId() { if (uniqueId == null) { uniqueId = - Preconditions.checkNotNull(Strings.emptyToNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVEQUERYID)), + Preconditions.checkNotNull(Strings.emptyToNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVE_QUERY_ID)), "Hive query id is null"); } return uniqueId; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java index fc50e0b9216d..2a2be067125f 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java @@ -101,7 +101,7 @@ public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) protected HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException { String address = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS); - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); if (StringUtils.isEmpty(address)) { throw new IOException("Druid broker address not specified in configuration"); } diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java index 7d94f1afc996..5040831b9a94 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java @@ -111,7 +111,7 @@ private DataSegment createSegment(String location, Interval interval, String ver Mockito.when(tableMock.getDbName()).thenReturn(DB_NAME); Mockito.when(tableMock.getTableName()).thenReturn(TABLE_NAME); config = new Configuration(); - config.set(String.valueOf(HiveConf.ConfVars.HIVEQUERYID), "hive-" + UUID.randomUUID().toString()); + config.set(String.valueOf(HiveConf.ConfVars.HIVE_QUERY_ID), "hive-" + UUID.randomUUID().toString()); config.set(String.valueOf(HiveConf.ConfVars.DRUID_WORKING_DIR), tableWorkingPath); config.set(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY), new Path(tableWorkingPath, "finalSegmentDir").toString()); diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseQueries.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseQueries.java index da69f0887f77..16bdd8cf3d27 100644 --- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseQueries.java +++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseQueries.java @@ -105,7 +105,7 @@ public void testRollbackDoesNotDeleteOriginTableWhenCTLTFails() throws CommandPr conf.setBoolVar(HiveConf.ConfVars.HIVE_STRICT_MANAGED_TABLES, true); conf.setBoolVar(HiveConf.ConfVars.CREATE_TABLES_AS_ACID, true); conf.setBoolVar(HiveConf.ConfVars.HIVE_CREATE_TABLES_AS_INSERT_ONLY, true); - conf.setVar(HiveConf.ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT, "ORC"); + conf.setVar(HiveConf.ConfVars.HIVE_DEFAULT_MANAGED_FILEFORMAT, "ORC"); driver = DriverFactory.newDriver(conf); diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java index 1e319b4e7dea..c42e9936f47a 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java @@ -81,7 +81,7 @@ private HCatConstants() { // restrict instantiation public static final String HCAT_TABLE_SCHEMA = "hcat.table.schema"; - public static final String HCAT_METASTORE_URI = HiveConf.ConfVars.METASTOREURIS.varname; + public static final String HCAT_METASTORE_URI = HiveConf.ConfVars.METASTORE_URIS.varname; public static final String HCAT_PERMS = "hcat.perms"; diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java index d786e3c4822e..11e53d94a48b 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.annotation.NoReconnect; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hive.common.util.ShutdownHookManager; @@ -327,7 +326,7 @@ static class HiveClientCacheKey { final private int threadId; private HiveClientCacheKey(HiveConf hiveConf, final int threadId) throws IOException, LoginException { - this.metaStoreURIs = hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS); + this.metaStoreURIs = hiveConf.getVar(HiveConf.ConfVars.METASTORE_URIS); ugi = Utils.getUGI(); this.hiveConf = hiveConf; this.threadId = threadId; diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/DynamicPartitionFileRecordWriterContainer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/DynamicPartitionFileRecordWriterContainer.java index cda8770a2c98..ceafabaa6eb0 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/DynamicPartitionFileRecordWriterContainer.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/DynamicPartitionFileRecordWriterContainer.java @@ -88,7 +88,7 @@ public DynamicPartitionFileRecordWriterContainer( this.dynamicContexts = new HashMap(); this.dynamicObjectInspectors = new HashMap(); this.dynamicOutputJobInfo = new HashMap(); - this.HIVE_DEFAULT_PARTITION_VALUE = HiveConf.getVar(context.getConfiguration(), HiveConf.ConfVars.DEFAULTPARTITIONNAME); + this.HIVE_DEFAULT_PARTITION_VALUE = HiveConf.getVar(context.getConfiguration(), HiveConf.ConfVars.DEFAULT_PARTITION_NAME); } @Override @@ -149,7 +149,7 @@ protected LocalFileWriter getLocalFileWriter(HCatRecord value) throws IOExceptio throw new HCatException(ErrorType.ERROR_TOO_MANY_DYNAMIC_PTNS, "Number of dynamic partitions being created " + "exceeds configured max allowable partitions[" + maxDynamicPartitions - + "], increase parameter [" + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname + + "], increase parameter [" + HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.varname + "] if needed."); } diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java index 476c60e53af5..de9ad252ff24 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java @@ -716,7 +716,7 @@ private void discoverPartitions(JobContext context) throws IOException { + "exceeds configured max allowable partitions[" + maxDynamicPartitions + "], increase parameter [" - + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname + + HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.varname + "] if needed."); } diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatOutputFormat.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatOutputFormat.java index 7167bd913e4d..9aaf67e40d84 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatOutputFormat.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatOutputFormat.java @@ -280,14 +280,14 @@ private static int getMaxDynamicPartitions(HiveConf hConf) { if (HCatConstants.HCAT_IS_DYNAMIC_MAX_PTN_CHECK_ENABLED) { maxDynamicPartitions = hConf.getIntVar( - HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS); + HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS); } return maxDynamicPartitions; } private static boolean getHarRequested(HiveConf hConf) { - return hConf.getBoolVar(HiveConf.ConfVars.HIVEARCHIVEENABLED); + return hConf.getBoolVar(HiveConf.ConfVars.HIVE_ARCHIVE_ENABLED); } } diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceInputFormat.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceInputFormat.java index 084185a7ed10..7cbc29d1ec39 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceInputFormat.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceInputFormat.java @@ -45,7 +45,7 @@ public RecordReader createRecordReader(Inpu @Override public List getSplits(JobContext job) throws IOException { HiveConf.setLongVar(job.getConfiguration(), - HiveConf.ConfVars.MAPREDMINSPLITSIZE, SequenceFile.SYNC_INTERVAL); + HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, SequenceFile.SYNC_INTERVAL); return super.getSplits(job); } } diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java index 2b57d8d8ae69..749409a03620 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java @@ -100,15 +100,15 @@ public void setUp() throws Exception { System.setSecurityManager(new NoExitSecurityManager()); Policy.setPolicy(new DerbyPolicy()); - hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); - hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_FAILURE_RETRIES, 3); hcatConf.setTimeVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 60, TimeUnit.SECONDS); hcatConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - hcatConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + hcatConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.WAREHOUSE)); - hcatConf.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + hcatConf.set(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.CONNECT_URL_KEY)); - hcatConf.set(HiveConf.ConfVars.METASTOREURIS.varname, + hcatConf.set(HiveConf.ConfVars.METASTORE_URIS.varname, MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.THRIFT_URIS)); clientWH = new Warehouse(hcatConf); msc = new HiveMetaStoreClient(hcatConf); @@ -200,18 +200,18 @@ private int callHCatCli(String[] args) throws Exception { argsList.add("-Dhive.support.concurrency=false"); argsList .add("-Dhive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); - argsList.add("-D" + HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES.varname + "=3"); - argsList.add("-D" + HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES.varname + "=3"); + argsList.add("-D" + HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES.varname + "=3"); + argsList.add("-D" + HiveConf.ConfVars.METASTORE_THRIFT_FAILURE_RETRIES.varname + "=3"); argsList.add("-D" + HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT.varname + "=60"); - argsList.add("-D" + HiveConf.ConfVars.METASTOREWAREHOUSE.varname + "=" + argsList.add("-D" + HiveConf.ConfVars.METASTORE_WAREHOUSE.varname + "=" + MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.WAREHOUSE)); - argsList.add("-D" + HiveConf.ConfVars.METASTORECONNECTURLKEY.varname + "=" + argsList.add("-D" + HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname + "=" + MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.CONNECT_URL_KEY)); - argsList.add("-D" + HiveConf.ConfVars.METASTOREURIS.varname + "=" + argsList.add("-D" + HiveConf.ConfVars.METASTORE_URIS.varname + "=" + MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.THRIFT_URIS)); argsList.add("-D" + HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname + "=" + HCatSemanticAnalyzer.class.getName()); - argsList.add("-D" + HiveConf.ConfVars.PREEXECHOOKS.varname + "="); - argsList.add("-D" + HiveConf.ConfVars.POSTEXECHOOKS.varname + "="); + argsList.add("-D" + HiveConf.ConfVars.PRE_EXEC_HOOKS.varname + "="); + argsList.add("-D" + HiveConf.ConfVars.POST_EXEC_HOOKS.varname + "="); argsList.add("-D" + HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname + "=false"); argsList.add("-D" + "test.warehouse.dir=" + System.getProperty("test.warehouse.dir")); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java index b5f29f5e40fd..546317ab00ec 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java @@ -65,7 +65,7 @@ public void setUpHCatDriver() throws IOException { hcatConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); - hcatConf.set(HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE.varname, + hcatConf.set(HiveConf.ConfVars.HIVE_DEFAULT_RCFILE_SERDE.varname, "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"); hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java index 63715432b721..73558f92cd71 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java @@ -44,8 +44,8 @@ public void setUp() throws Exception { HiveConf hcatConf = new HiveConf(this.getClass()); hcatConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); - hcatConf.set(ConfVars.PREEXECHOOKS.varname, ""); - hcatConf.set(ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.PRE_EXEC_HOOKS.varname, ""); + hcatConf.set(ConfVars.POST_EXEC_HOOKS.varname, ""); hcatConf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHiveClientCache.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHiveClientCache.java index fe1d8afdc8bd..58772179a82a 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHiveClientCache.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHiveClientCache.java @@ -83,7 +83,7 @@ public void testCacheHit() throws IOException, MetaException, LoginException { client.close(); // close shouldn't matter // Setting a non important configuration should return the same client only - hiveConf.setIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS, 10); + hiveConf.setIntVar(HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS, 10); HiveClientCache.ICacheableMetaStoreClient client2 = (HiveClientCache.ICacheableMetaStoreClient) cache.get(hiveConf); assertNotNull(client2); assertSame(client, client2); @@ -98,7 +98,7 @@ public void testCacheMiss() throws IOException, MetaException, LoginException { assertNotNull(client); // Set different uri as it is one of the criteria deciding whether to return the same client or not - hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, " "); // URIs are checked for string equivalence, even spaces make them different + hiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, " "); // URIs are checked for string equivalence, even spaces make them different IMetaStoreClient client2 = cache.get(hiveConf); assertNotNull(client2); assertNotSame(client, client2); @@ -157,7 +157,7 @@ public IMetaStoreClient call() throws IOException, MetaException, LoginException public void testCloseAllClients() throws IOException, MetaException, LoginException { final HiveClientCache cache = new HiveClientCache(1000); HiveClientCache.ICacheableMetaStoreClient client1 = (HiveClientCache.ICacheableMetaStoreClient) cache.get(hiveConf); - hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, " "); // URIs are checked for string equivalence, even spaces make them different + hiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, " "); // URIs are checked for string equivalence, even spaces make them different HiveClientCache.ICacheableMetaStoreClient client2 = (HiveClientCache.ICacheableMetaStoreClient) cache.get(hiveConf); cache.closeAllClientsQuietly(); assertTrue(client1.isClosed()); @@ -227,18 +227,18 @@ public LocalMetaServer() { securityManager = System.getSecurityManager(); System.setSecurityManager(new NoExitSecurityManager()); hiveConf = new HiveConf(TestHiveClientCache.class); - hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + hiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + MS_PORT); - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_FAILURE_RETRIES, 3); hiveConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, " "); } public void start() throws InterruptedException { diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java index cf5ef56b5201..08a941c5e1f1 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/HCatDataCheckUtil.java @@ -46,8 +46,8 @@ public static IDriver instantiateDriver(MiniCluster cluster) { for (Entry e : cluster.getProperties().entrySet()) { hiveConf.set(e.getKey().toString(), e.getValue().toString()); } - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); LOG.debug("Hive conf : {}", hiveConf.getAllProperties()); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java index a304e49ae879..25cb75ec41d2 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/HCatBaseTest.java @@ -89,12 +89,12 @@ protected void setUpHiveConf() { + File.separator + "mapred" + File.separator + "staging"); hiveConf.set("mapred.temp.dir", workDir + File.separator + this.getClass().getSimpleName() + File.separator + "mapred" + File.separator + "temp"); - hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, ""); - hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.POST_EXEC_HOOKS, ""); hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, TEST_WAREHOUSE_DIR); - hiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, true); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, TEST_WAREHOUSE_DIR); + hiveConf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, true); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java index 424e428be545..a97162de993a 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java @@ -38,7 +38,6 @@ import org.apache.hive.hcatalog.data.schema.HCatFieldSchema; import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils; -import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; @@ -196,7 +195,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, public void _testHCatDynamicPartitionMaxPartitions() throws Exception { HiveConf hc = new HiveConf(this.getClass()); - int maxParts = hiveConf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS); + int maxParts = hiveConf.getIntVar(HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS); LOG.info("Max partitions allowed = {}", maxParts); IOException exc = null; diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java index e601992fc40b..d87158b23fae 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java @@ -156,7 +156,7 @@ public static void setup() throws Exception { warehousedir = new Path(System.getProperty("test.warehouse.dir")); HiveConf metastoreConf = new HiveConf(); - metastoreConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, warehousedir.toString()); + metastoreConf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, warehousedir.toString()); // Run hive metastore server MetaStoreTestUtils.startMetaStoreWithRetry(metastoreConf); @@ -183,23 +183,23 @@ public static void setup() throws Exception { private static void initializeSetup(HiveConf metastoreConf) throws Exception { hiveConf = new HiveConf(metastoreConf, TestHCatMultiOutputFormat.class); - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_FAILURE_RETRIES, 3); hiveConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + System.setProperty(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.WAREHOUSE)); - System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.CONNECT_URL_KEY)); - System.setProperty(HiveConf.ConfVars.METASTOREURIS.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_URIS.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.THRIFT_URIS)); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehousedir.toString()); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, warehousedir.toString()); try { hmsc = new HiveMetaStoreClient(hiveConf); initalizeTables(); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java index 22a0d3f7d134..2b28f4f0d752 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.ql.io.RCFileInputFormat; import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; import org.apache.hadoop.hive.serde.serdeConstants; @@ -120,23 +119,23 @@ public static void setup() throws Exception { System.setSecurityManager(new NoExitSecurityManager()); Policy.setPolicy(new DerbyPolicy()); - hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); - hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_FAILURE_RETRIES, 3); hcatConf.setTimeVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 120, TimeUnit.SECONDS); hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); - hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); msc = new HiveMetaStoreClient(hcatConf); - System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + System.setProperty(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.WAREHOUSE)); - System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.CONNECT_URL_KEY)); - System.setProperty(HiveConf.ConfVars.METASTOREURIS.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_URIS.varname, MetastoreConf.getVar(hcatConf, MetastoreConf.ConfVars.THRIFT_URIS)); } diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java index c955aa502126..a787f409eb3f 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestPassProperties.java @@ -58,10 +58,10 @@ public class TestPassProperties { public void Initialize() throws Exception { hiveConf = new HiveConf(this.getClass()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, TEST_WAREHOUSE_DIR); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java index 67193d4e50f1..759a73b49988 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; @@ -233,7 +232,7 @@ private void writeThenReadByRecordReader(int intervalRecordCount, jonconf.set("mapred.input.dir", testDir.toString()); JobContext context = new Job(jonconf); HiveConf.setLongVar(context.getConfiguration(), - HiveConf.ConfVars.MAPREDMAXSPLITSIZE, maxSplitSize); + HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, maxSplitSize); List splits = inputFormat.getSplits(context); assertEquals("splits length should be " + splitNumber, splitNumber, splits.size()); int readCount = 0; diff --git a/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/PigHCatUtil.java b/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/PigHCatUtil.java index afe6e92163b8..dbf7ac3f1a32 100644 --- a/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/PigHCatUtil.java +++ b/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/PigHCatUtil.java @@ -132,7 +132,7 @@ static public Pair getDBTableNames(String location) throws IOExc static public String getHCatServerUri(Job job) { - return job.getConfiguration().get(HiveConf.ConfVars.METASTOREURIS.varname); + return job.getConfiguration().get(HiveConf.ConfVars.METASTORE_URIS.varname); } static public String getHCatServerPrincipal(Job job) { @@ -153,7 +153,7 @@ private static IMetaStoreClient getHiveMetaClient(String serverUri, HiveConf hiveConf = new HiveConf(job.getConfiguration(), clazz); if (serverUri != null) { - hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, serverUri.trim()); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, serverUri.trim()); } if (serverKerberosPrincipal != null) { diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java index ae292eb78c16..d5e5fc311973 100644 --- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java +++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestE2EScenarios.java @@ -84,10 +84,10 @@ public void setUp() throws Exception { } HiveConf hiveConf = new HiveConf(this.getClass()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, TEST_WAREHOUSE_DIR); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java index b96479b826a0..b16d5c183d50 100644 --- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java +++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java @@ -116,8 +116,8 @@ public static void setUpBeforeClass() throws Exception { + File.separator + "mapred" + File.separator + "staging"); hiveConf.set("mapred.temp.dir", workDir + File.separator + "TestHCatLoaderComplexSchema" + File.separator + "mapred" + File.separator + "temp"); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java index beb4fe9f4b92..0e5691a66543 100644 --- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java +++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderEncryption.java @@ -159,10 +159,10 @@ public void setup() throws Exception { } HiveConf hiveConf = new HiveConf(this.getClass()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, TEST_WAREHOUSE_DIR); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java index a0c5ce93ff27..3a2b3c15b5fc 100644 --- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java +++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java @@ -95,11 +95,11 @@ public void setUp() throws Exception { if (driver == null) { HiveConf hiveConf = new HiveConf(this.getClass()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); - hiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, TEST_WAREHOUSE_DIR); + hiveConf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java index 782fffb516b9..c53e5afa094d 100644 --- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java +++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java @@ -54,7 +54,7 @@ public abstract class MessageFactory { + HCAT_MESSAGE_FORMAT, DEFAULT_MESSAGE_FACTORY_IMPL); - protected static final String HCAT_SERVER_URL = hiveConf.get(HiveConf.ConfVars.METASTOREURIS.name(), ""); + protected static final String HCAT_SERVER_URL = hiveConf.get(HiveConf.ConfVars.METASTORE_URIS.name(), ""); protected static final String HCAT_SERVICE_PRINCIPAL = hiveConf.get(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL.name(), ""); /** diff --git a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java index 0420c506136d..07c9ca57a2f7 100644 --- a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java +++ b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java @@ -54,9 +54,7 @@ import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe; -import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.mapred.TextInputFormat; -import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hive.hcatalog.DerbyPolicy; import org.apache.hive.hcatalog.api.repl.Command; import org.apache.hive.hcatalog.api.repl.ReplicationTask; @@ -109,9 +107,9 @@ public static void tearDown() throws Exception { public static void startMetaStoreServer() throws Exception { hcatConf = new HiveConf(TestHCatClient.class); - String metastoreUri = System.getProperty("test."+HiveConf.ConfVars.METASTOREURIS.varname); + String metastoreUri = System.getProperty("test."+HiveConf.ConfVars.METASTORE_URIS.varname); if (metastoreUri != null) { - hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, metastoreUri); + hcatConf.setVar(HiveConf.ConfVars.METASTORE_URIS, metastoreUri); useExternalMS = true; return; } @@ -130,17 +128,17 @@ public static void startMetaStoreServer() throws Exception { System.setSecurityManager(new NoExitSecurityManager()); Policy.setPolicy(new DerbyPolicy()); - hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + hcatConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + msPort); - hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); - hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, " "); } public static HiveConf getConf(){ @@ -819,7 +817,7 @@ private void startReplicationTargetMetaStoreIfRequired() throws Exception { .replace("metastore", "target_metastore")); replicationTargetHCatPort = MetaStoreTestUtils.startMetaStoreWithRetry(conf); replicationTargetHCatConf = new HiveConf(hcatConf); - replicationTargetHCatConf.setVar(HiveConf.ConfVars.METASTOREURIS, + replicationTargetHCatConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + replicationTargetHCatPort); isReplicationTargetHCatRunning = true; } diff --git a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java index b94c7d715530..d13adf97cb90 100644 --- a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java +++ b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java @@ -252,7 +252,7 @@ private void init() { private void handleHiveProperties() { HiveConf hiveConf = new HiveConf();//load hive-site.xml from classpath List interestingPropNames = Arrays.asList( - HiveConf.ConfVars.METASTOREURIS.varname, + HiveConf.ConfVars.METASTORE_URIS.varname, HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname, HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI.varname, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname, diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java index c93ce5455e9f..90eea7618df6 100644 --- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java +++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/CachedClientPool.java @@ -128,7 +128,7 @@ public R run(Action action, boolean retry) static Key extractKey(String cacheKeys, Configuration conf) { // generate key elements in a certain order, so that the Key instances are comparable List elements = Lists.newArrayList(); - elements.add(conf.get(HiveConf.ConfVars.METASTOREURIS.varname, "")); + elements.add(conf.get(HiveConf.ConfVars.METASTORE_URIS.varname, "")); elements.add(conf.get(HiveCatalog.HIVE_CONF_CATALOG, "hive")); if (cacheKeys == null || cacheKeys.isEmpty()) { return Key.of(elements); diff --git a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index 6c98cee6a528..de859a508672 100644 --- a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -95,11 +95,11 @@ public void initialize(String inputName, Map properties) { } if (properties.containsKey(CatalogProperties.URI)) { - this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, properties.get(CatalogProperties.URI)); + this.conf.set(HiveConf.ConfVars.METASTORE_URIS.varname, properties.get(CatalogProperties.URI)); } if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) { - this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + this.conf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, LocationUtil.stripTrailingSlash(properties.get(CatalogProperties.WAREHOUSE_LOCATION))); } @@ -489,7 +489,7 @@ protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { } private String databaseLocation(String databaseName) { - String warehouseLocation = conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname); + String warehouseLocation = conf.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname); Preconditions.checkNotNull( warehouseLocation, "Warehouse location is not set: hive.metastore.warehouse.dir=null"); warehouseLocation = LocationUtil.stripTrailingSlash(warehouseLocation); @@ -563,7 +563,7 @@ Database convertToDatabase(Namespace namespace, Map meta) { public String toString() { return MoreObjects.toStringHelper(this) .add("name", name) - .add("uri", this.conf == null ? "" : this.conf.get(HiveConf.ConfVars.METASTOREURIS.varname)) + .add("uri", this.conf == null ? "" : this.conf.get(HiveConf.ConfVars.METASTORE_URIS.varname)) .toString(); } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index d65d38085a1d..aa55094f12b8 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -1106,7 +1106,7 @@ public void testConstructorWarehousePathWithEndSlash() { catalogWithSlash.initialize( "hive_catalog", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, wareHousePath + "/")); - assertThat(catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) + assertThat(catalogWithSlash.getConf().get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname)) .as("Should have trailing slash stripped") .isEqualTo(wareHousePath); } diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java index 3a1b92a12c7f..280dc02554ba 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java @@ -70,13 +70,13 @@ public void after() { @Test public void testConf() { HiveConf conf = createHiveConf(); - conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file:/mywarehouse/"); + conf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "file:/mywarehouse/"); HiveClientPool clientPool = new HiveClientPool(10, conf); HiveConf clientConf = clientPool.hiveConf(); - assertThat(clientConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) - .isEqualTo(conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + assertThat(clientConf.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname)) + .isEqualTo(conf.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname)); assertThat(clientPool.poolSize()).isEqualTo(10); // 'hive.metastore.sasl.enabled' should be 'true' as defined in xml diff --git a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java index 6fc54de9ec2a..d4ceeca67d2b 100644 --- a/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java +++ b/iceberg/iceberg-catalog/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java @@ -151,7 +151,7 @@ public void start(HiveConf conf, int poolSize) { this.executorService.submit(() -> server.serve()); // in Hive3, setting this as a system prop ensures that it will be picked up whenever a new HiveConf is created - System.setProperty(HiveConf.ConfVars.METASTOREURIS.varname, hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS)); + System.setProperty(HiveConf.ConfVars.METASTORE_URIS.varname, hiveConf.getVar(HiveConf.ConfVars.METASTORE_URIS)); this.clientPool = new HiveClientPool(1, hiveConf); } catch (Exception e) { @@ -229,7 +229,7 @@ public R run(ClientPool.Action action) thro private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf conf) throws Exception { HiveConf serverConf = new HiveConf(conf); - serverConf.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, "jdbc:derby:" + DERBY_PATH + ";create=true"); + serverConf.set(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, "jdbc:derby:" + DERBY_PATH + ";create=true"); baseHandler = HMS_HANDLER_CTOR.newInstance("new db based metaserver", serverConf); IHMSHandler handler = GET_BASE_HMS_HANDLER.invoke(serverConf, baseHandler, false); @@ -244,8 +244,8 @@ private TServer newThriftServer(TServerSocket socket, int poolSize, HiveConf con } private void initConf(HiveConf conf, int port) { - conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://localhost:" + port); - conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file:" + HIVE_WAREHOUSE_DIR.getAbsolutePath()); + conf.set(HiveConf.ConfVars.METASTORE_URIS.varname, "thrift://localhost:" + port); + conf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "file:" + HIVE_WAREHOUSE_DIR.getAbsolutePath()); conf.set(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname, "file:" + HIVE_EXTERNAL_WAREHOUSE_DIR.getAbsolutePath()); conf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, "false"); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index c4a6005818ae..ba64faa6188a 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -687,7 +687,7 @@ private static FilesForCommit collectResults(int numTasks, ExecutorService execu */ @VisibleForTesting static String generateJobLocation(String location, Configuration conf, JobID jobId) { - String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname); + String queryId = conf.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname); return location + "/temp/" + queryId + "-" + jobId; } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java index 7c625543b780..c356898c65eb 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java @@ -72,7 +72,7 @@ private static HiveIcebergWriter writer(JobConf jc) { setWriterLevelConfiguration(jc, table); return WriterBuilder.builderFor(table) - .queryId(jc.get(HiveConf.ConfVars.HIVEQUERYID.varname)) + .queryId(jc.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname)) .tableName(tableName) .attemptID(taskAttemptID) .poolSize(poolSize) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java index 130b7186221d..548d33f7d93d 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java @@ -146,8 +146,8 @@ public void initialize(@Nullable Configuration configuration, Properties serDePr // Currently ClusteredWriter is used which requires that records are ordered by partition keys. // Here we ensure that SortedDynPartitionOptimizer will kick in and do the sorting. // TODO: remove once we have both Fanout and ClusteredWriter available: HIVE-25948 - HiveConf.setIntVar(configuration, HiveConf.ConfVars.HIVEOPTSORTDYNAMICPARTITIONTHRESHOLD, 1); - HiveConf.setVar(configuration, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + HiveConf.setIntVar(configuration, HiveConf.ConfVars.HIVE_OPT_SORT_DYNAMIC_PARTITION_THRESHOLD, 1); + HiveConf.setVar(configuration, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); try { this.inspector = IcebergObjectInspector.create(projectedSchema); } catch (Exception e) { diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 985e7d48f617..d07b820e6d04 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -671,8 +671,8 @@ public DynamicPartitionCtx createDPContext( Table table = IcebergTableUtil.getTable(conf, tableDesc.getProperties()); DynamicPartitionCtx dpCtx = new DynamicPartitionCtx(Maps.newLinkedHashMap(), - hiveConf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME), - hiveConf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTSPERNODE)); + hiveConf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME), + hiveConf.getIntVar(HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS_PER_NODE)); List, ExprNodeDesc>> customSortExprs = Lists.newLinkedList(); dpCtx.setCustomSortExpressions(customSortExprs); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java index a453e5ea723a..510f562922ba 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java @@ -234,7 +234,7 @@ public static RemoteIterator getFilesIterator(Path path, Conf } static String generateTableObjectLocation(String tableLocation, Configuration conf) { - return tableLocation + "/temp/" + conf.get(HiveConf.ConfVars.HIVEQUERYID.varname) + TABLE_EXTENSION; + return tableLocation + "/temp/" + conf.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname) + TABLE_EXTENSION; } static void createFileForTableObject(Table table, Configuration conf) { @@ -305,7 +305,7 @@ static JobConf getPartJobConf(Configuration confs, org.apache.hadoop.hive.ql.met job.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, Constants.ICEBERG_PARTITION_TABLE_SCHEMA); job.set(InputFormatConfig.TABLE_LOCATION, tbl.getPath().toString()); job.set(InputFormatConfig.TABLE_IDENTIFIER, tbl.getFullyQualifiedName() + ".partitions"); - HiveConf.setVar(job, HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, Constants.DELIMITED_JSON_SERDE); + HiveConf.setVar(job, HiveConf.ConfVars.HIVE_FETCH_OUTPUT_SERDE, Constants.DELIMITED_JSON_SERDE); HiveConf.setBoolVar(job, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false); return job; } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java index 0853124b6025..b5c9e2942b45 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java @@ -186,9 +186,9 @@ public void before() throws IOException { // Fetch task conversion might kick in for certain queries preventing vectorization code path to be used, so // we turn it off explicitly to achieve better coverage. if (isVectorized) { - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); } else { - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "more"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "more"); } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java index acfe94126dc6..bc3c948c4ad4 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java @@ -618,7 +618,7 @@ public void testConcurrent2Deletes() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); shell.executeStatement(sql); @@ -649,7 +649,7 @@ public void testConcurrent2Updates() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); shell.executeStatement(sql); @@ -684,7 +684,7 @@ public void testConcurrentUpdateAndDelete() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); shell.executeStatement(sql[i]); @@ -719,7 +719,7 @@ public void testConcurrent2MergeInserts() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); shell.executeStatement(sql); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java index d0eb3ebc8f09..45f82b5a28aa 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java @@ -224,7 +224,7 @@ private JobConf jobConf(Table table, int taskNum) { JobConf conf = new JobConf(); conf.setNumMapTasks(taskNum); conf.setNumReduceTasks(0); - conf.set(HiveConf.ConfVars.HIVEQUERYID.varname, QUERY_ID); + conf.set(HiveConf.ConfVars.HIVE_QUERY_ID.varname, QUERY_ID); conf.set(InputFormatConfig.OUTPUT_TABLES, table.name()); conf.set(InputFormatConfig.OPERATION_TYPE_PREFIX + table.name(), Context.Operation.OTHER.name()); conf.set(InputFormatConfig.TABLE_CATALOG_PREFIX + table.name(), diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 48da53e2b685..c1bbeb039893 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -1978,7 +1978,7 @@ public void checkIcebergTableLocation() throws TException, InterruptedException, String dBName = "testdb"; String tableName = "tbl"; String dbWithSuffix = "/" + dBName + ".db"; - String dbManagedLocation = shell.getHiveConf().get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname) + dbWithSuffix; + String dbManagedLocation = shell.getHiveConf().get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname) + dbWithSuffix; String dbExternalLocation = shell.getHiveConf().get(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname) + dbWithSuffix; Path noExistedTblPath = new Path(dbManagedLocation + "/" + tableName); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java index 79e477bbe59f..8d2a9a294c7c 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java @@ -85,9 +85,9 @@ public void setHiveSessionValue(String key, boolean value) { public void start() { // Create a copy of the HiveConf for the metastore metastore.start(new HiveConf(hs2Conf), 20); - hs2Conf.setVar(HiveConf.ConfVars.METASTOREURIS, metastore.hiveConf().getVar(HiveConf.ConfVars.METASTOREURIS)); - hs2Conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, - metastore.hiveConf().getVar(HiveConf.ConfVars.METASTOREWAREHOUSE)); + hs2Conf.setVar(HiveConf.ConfVars.METASTORE_URIS, metastore.hiveConf().getVar(HiveConf.ConfVars.METASTORE_URIS)); + hs2Conf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, + metastore.hiveConf().getVar(HiveConf.ConfVars.METASTORE_WAREHOUSE)); hs2Conf.setVar(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL, metastore.hiveConf().getVar(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL)); @@ -195,13 +195,13 @@ private HiveConf initializeConf() { // Switch off optimizers in order to contain the map reduction within this JVM hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_CBO_ENABLED, true); hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_INFER_BUCKET_SORT, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVESKEWJOIN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_METADATA_ONLY_QUERIES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SKEW_JOIN, false); // Speed up test execution - hiveConf.setLongVar(HiveConf.ConfVars.HIVECOUNTERSPULLINTERVAL, 1L); + hiveConf.setLongVar(HiveConf.ConfVars.HIVE_COUNTERS_PULL_INTERVAL, 1L); hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false); // Resource configuration diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java index cd4aa88e7807..c9d0bf7e3bc1 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java @@ -55,7 +55,7 @@ public void testConcurrentOverlappingUpdates() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES); shell.executeStatement(sql); @@ -89,7 +89,7 @@ public void testNonOverlappingConcurrent2Updates() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES); shell.executeStatement(sql[i]); @@ -126,7 +126,7 @@ public void testConcurrent2MergeInserts() { .run(i -> { init(shell, testTables, temp, executionEngine); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES); shell.executeStatement(sql); diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java index 155735c6072e..8c28c63b62b4 100644 --- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/ManyMiniCluster.java @@ -299,12 +299,12 @@ private void setUpMetastore() throws Exception { //The default org.apache.hadoop.hive.ql.hooks.PreExecutePrinter hook //is present only in the ql/test directory - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hiveConf.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + hiveConf.set(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, "jdbc:derby:" + new File(workDir + "/metastore_db") + ";create=true"); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.toString(), + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.toString(), new File(workDir, "warehouse").toString()); //set where derby logs File derbyLogFile = new File(workDir + "/derby.log"); diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java index fe33f47793fb..acb37344972c 100644 --- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/hbase/TestPigHBaseStorageHandler.java @@ -75,9 +75,9 @@ public void Initialize() throws Exception { Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(), getTestDir()); hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); - hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString()); + hcatConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); + hcatConf.set(ConfVars.METASTORE_WAREHOUSE.varname, whPath.toString()); hcatConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java index c093055ecff1..eb093e105143 100644 --- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java +++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/mapreduce/TestSequenceFileReadWrite.java @@ -73,10 +73,10 @@ public void setup() throws Exception { warehouseDir = HCatUtil.makePathASafeFileName(dataDir + File.separator + "warehouse"); inputFileName = HCatUtil.makePathASafeFileName(dataDir + File.separator + "input.data"); hiveConf = new HiveConf(this.getClass()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouseDir); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, warehouseDir); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/AbstractHTLoadBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/AbstractHTLoadBench.java index b33c7da545e2..b597bed53849 100644 --- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/AbstractHTLoadBench.java +++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/AbstractHTLoadBench.java @@ -123,8 +123,8 @@ protected void setupMapJoinHT(HiveConf hiveConf, long seed, int rowCount, TypeInfo[] smallTableValueTypeInfos, int[] smallTableRetainKeyColumnNums, SmallTableGenerationParameters smallTableGenerationParameters) throws Exception { - hiveConf.set(HiveConf.ConfVars.HIVEMAPJOINPARALELHASHTABLETHREADS.varname, LOAD_THREADS_NUM + ""); - LOG.info("Number of threads: " + hiveConf.get(HiveConf.ConfVars.HIVEMAPJOINPARALELHASHTABLETHREADS.varname)); + hiveConf.set(HiveConf.ConfVars.HIVE_MAPJOIN_PARALEL_HASHTABLE_THREADS.varname, LOAD_THREADS_NUM + ""); + LOG.info("Number of threads: " + hiveConf.get(HiveConf.ConfVars.HIVE_MAPJOIN_PARALEL_HASHTABLE_THREADS.varname)); this.rowCount = rowCount; diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/LegacyVectorMapJoinFastHashTableLoader.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/LegacyVectorMapJoinFastHashTableLoader.java index 7af9380b619d..f292cf09481e 100644 --- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/LegacyVectorMapJoinFastHashTableLoader.java +++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/load/LegacyVectorMapJoinFastHashTableLoader.java @@ -56,7 +56,7 @@ public LegacyVectorMapJoinFastHashTableLoader(TezContext context, Configuration this.hconf = hconf; this.desc = (MapJoinDesc)joinOp.getConf(); this.cacheKey = joinOp.getCacheKey(); - this.htLoadCounter = this.tezContext.getTezProcessorContext().getCounters().findCounter(HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP), hconf.get("__hive.context.name", "")); + this.htLoadCounter = this.tezContext.getTezProcessorContext().getCounters().findCounter(HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_COUNTER_GROUP), hconf.get("__hive.context.name", "")); } @Override @@ -66,7 +66,7 @@ public void init(ExecMapperContext context, MapredContext mrContext, this.hconf = hconf; this.desc = joinOp.getConf(); this.cacheKey = joinOp.getCacheKey(); - String counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP); + String counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); String vertexName = hconf.get(Operator.CONTEXT_NAME_KEY, ""); String counterName = Utilities.getVertexCounterName(HashTableLoaderCounters.HASHTABLE_LOAD_TIME_MS.name(), vertexName); this.htLoadCounter = tezContext.getTezProcessorContext().getCounters().findCounter(counterGroup, counterName); diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/JdbcWithMiniKdcSQLAuthTest.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/JdbcWithMiniKdcSQLAuthTest.java index fccf3e0209f8..ce5518f0e4a6 100644 --- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/JdbcWithMiniKdcSQLAuthTest.java +++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/JdbcWithMiniKdcSQLAuthTest.java @@ -59,7 +59,7 @@ public static void beforeTestBase(String transportMode) throws Exception { hiveConf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true); hiveConf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false); - hiveConf.setBoolVar(ConfVars.HIVEFETCHTASKCACHING, false); + hiveConf.setBoolVar(ConfVars.HIVE_FETCH_TASK_CACHING, false); miniHS2 = MiniHiveKdc.getMiniHS2WithKerb(miniHiveKdc, hiveConf); miniHS2.start(new HashMap()); diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHs2HooksWithMiniKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHs2HooksWithMiniKdc.java index 890e4092ea4f..e370810e5e82 100644 --- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHs2HooksWithMiniKdc.java +++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestHs2HooksWithMiniKdc.java @@ -50,12 +50,12 @@ public class TestHs2HooksWithMiniKdc { @BeforeClass public static void setUpBeforeClass() throws Exception { Class.forName(MiniHS2.getJdbcDriverName()); - confOverlay.put(ConfVars.POSTEXECHOOKS.varname, PostExecHook.class.getName()); - confOverlay.put(ConfVars.PREEXECHOOKS.varname, PreExecHook.class.getName()); + confOverlay.put(ConfVars.POST_EXEC_HOOKS.varname, PostExecHook.class.getName()); + confOverlay.put(ConfVars.PRE_EXEC_HOOKS.varname, PreExecHook.class.getName()); confOverlay.put(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, SemanticAnalysisHook.class.getName()); confOverlay.put(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "" + Boolean.FALSE); - confOverlay.put(ConfVars.HIVEFETCHTASKCACHING.varname, "" + false); + confOverlay.put(ConfVars.HIVE_FETCH_TASK_CACHING.varname, "" + false); miniHiveKdc = new MiniHiveKdc(); HiveConf hiveConf = new HiveConf(); diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStore.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStore.java index 58bcac694215..3da7b16a0eaf 100644 --- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStore.java +++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStore.java @@ -48,9 +48,9 @@ public static void beforeTest() throws Exception { String hs2Principal = miniHS2.getConfProperty(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL.varname); String hs2KeyTab = miniHS2.getConfProperty(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB.varname); System.out.println("HS2 principal : " + hs2Principal + " HS2 keytab : " + hs2KeyTab + " Metastore principal : " + metastorePrincipal); - System.setProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.WAREHOUSE)); - System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.CONNECT_URL_KEY)); System.setProperty(ConfVars.METASTORE_USE_THRIFT_SASL.varname, String.valueOf(MetastoreConf.getBoolVar(hiveConf, MetastoreConf.ConfVars.USE_THRIFT_SASL))); diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStoreNoDoAs.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStoreNoDoAs.java index 74d8e777597e..f666077de886 100644 --- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStoreNoDoAs.java +++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithDBTokenStoreNoDoAs.java @@ -48,11 +48,11 @@ public static void beforeTest() throws Exception { String hs2Principal = miniHS2.getConfProperty(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL.varname); String hs2KeyTab = miniHS2.getConfProperty(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB.varname); System.out.println("HS2 principal : " + hs2Principal + " HS2 keytab : " + hs2KeyTab + " Metastore principal : " + metastorePrincipal); - System.setProperty(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.WAREHOUSE)); - System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.CONNECT_URL_KEY)); - System.setProperty(HiveConf.ConfVars.METASTOREURIS.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_URIS.varname, MetastoreConf.getVar(hiveConf, MetastoreConf.ConfVars.THRIFT_URIS)); System.setProperty(ConfVars.METASTORE_USE_THRIFT_SASL.varname, String.valueOf(MetastoreConf.getBoolVar(hiveConf, MetastoreConf.ConfVars.USE_THRIFT_SASL))); diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdcCookie.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdcCookie.java index 883d333dd48d..77dabb42bd27 100644 --- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdcCookie.java +++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestJdbcWithMiniKdcCookie.java @@ -81,7 +81,7 @@ public void setUp() throws Exception { hiveConf.setTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE, 1, TimeUnit.SECONDS); hiveConf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - hiveConf.setBoolVar(ConfVars.HIVEFETCHTASKCACHING, false); + hiveConf.setBoolVar(ConfVars.HIVE_FETCH_TASK_CACHING, false); miniHS2 = MiniHiveKdc.getMiniHS2WithKerb(miniHiveKdc, hiveConf); miniHS2.start(new HashMap()); diff --git a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestSSLWithMiniKdc.java b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestSSLWithMiniKdc.java index 7ca74efb648b..4ee239a7cbff 100644 --- a/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestSSLWithMiniKdc.java +++ b/itests/hive-minikdc/src/test/java/org/apache/hive/minikdc/TestSSLWithMiniKdc.java @@ -58,7 +58,7 @@ public static void beforeTest() throws Exception { SSLTestUtils.setMetastoreSslConf(hiveConf); hiveConf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - hiveConf.setBoolVar(ConfVars.HIVEFETCHTASKCACHING, false); + hiveConf.setBoolVar(ConfVars.HIVE_FETCH_TASK_CACHING, false); setHMSSaslConf(miniHiveKdc, hiveConf); diff --git a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java index 61d2e920be7b..5f952a356951 100644 --- a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java +++ b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java @@ -26,7 +26,6 @@ import java.lang.reflect.Method; import java.net.URI; -import java.security.PrivilegedExceptionAction; import java.util.Arrays; import java.util.List; @@ -91,14 +90,14 @@ protected HiveConf createHiveConf() throws Exception { warehouseDir = new Path(new Path(fs.getUri()), "/warehouse"); fs.mkdirs(warehouseDir); - conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, warehouseDir.toString()); + conf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, warehouseDir.toString()); extWarehouseDir = new Path(new Path(fs.getUri()), "/external"); fs.mkdirs(extWarehouseDir); conf.setVar(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL, extWarehouseDir.toString()); // Set up scratch directory Path scratchDir = new Path(new Path(fs.getUri()), "/scratchdir"); - conf.setVar(HiveConf.ConfVars.SCRATCHDIR, scratchDir.toString()); + conf.setVar(HiveConf.ConfVars.SCRATCH_DIR, scratchDir.toString()); return conf; } diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/UtilsForTest.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/UtilsForTest.java index b3dfa961a6c3..c25aa1df5262 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/UtilsForTest.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/UtilsForTest.java @@ -39,7 +39,7 @@ public class UtilsForTest { public static void setNewDerbyDbLocation(HiveConf conf, String newloc) { String newDbLoc = System.getProperty("test.tmp.dir") + newloc + "metastore_db"; - conf.setVar(ConfVars.METASTORECONNECTURLKEY, "jdbc:derby:;databaseName=" + newDbLoc + conf.setVar(ConfVars.METASTORE_CONNECT_URL_KEY, "jdbc:derby:;databaseName=" + newDbLoc + ";create=true"); } diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java index 7d441b6acc03..add09aec5da2 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java @@ -19,7 +19,6 @@ //The tests here are heavily based on some timing, so there is some chance to fail. package org.apache.hadoop.hive.hooks; -import java.io.Serializable; import java.lang.Override; import java.sql.Statement; import java.util.List; @@ -140,9 +139,9 @@ public void postAnalyze(HiveSemanticAnalyzerHookContext context, @BeforeClass public static void setUpBeforeClass() throws Exception { HiveConf hiveConf = new HiveConf(); - hiveConf.setVar(ConfVars.PREEXECHOOKS, + hiveConf.setVar(ConfVars.PRE_EXEC_HOOKS, PreExecHook.class.getName()); - hiveConf.setVar(ConfVars.POSTEXECHOOKS, + hiveConf.setVar(ConfVars.POST_EXEC_HOOKS, PostExecHook.class.getName()); hiveConf.setVar(ConfVars.SEMANTIC_ANALYZER_HOOK, SemanticAnalysisHook.class.getName()); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/AbstractTestAuthorizationApiAuthorizer.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/AbstractTestAuthorizationApiAuthorizer.java index c10060f8171d..a0d5bd2c99b3 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/AbstractTestAuthorizationApiAuthorizer.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/AbstractTestAuthorizationApiAuthorizer.java @@ -59,9 +59,9 @@ protected static void setup() throws Exception { if (isRemoteMetastoreMode) { MetaStoreTestUtils.startMetaStoreWithRetry(hiveConf); } - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); msc = new HiveMetaStoreClient(hiveConf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java index 9ddad9922652..06ba4dae0c1a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java @@ -44,7 +44,7 @@ public class TestMetaStoreAuthorization { public void setup() throws Exception { conf.setBoolVar(HiveConf.ConfVars.METASTORE_AUTHORIZATION_STORAGE_AUTH_CHECKS, true); - conf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + conf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); conf.setTimeVar(ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, 60, TimeUnit.SECONDS); } diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java index 2f7a2601627c..a94d1da8cc1c 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java @@ -42,7 +42,7 @@ public class TestMetaStoreMetrics { @BeforeClass public static void before() throws Exception { hiveConf = new HiveConf(TestMetaStoreMetrics.class); - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_METRICS, true); hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); hiveConf diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMultipleEncryptionZones.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMultipleEncryptionZones.java index 4dcfb2266a0c..a901d87c75eb 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMultipleEncryptionZones.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMultipleEncryptionZones.java @@ -84,18 +84,18 @@ public static void setUp() throws Exception { DFSTestUtil.createKey("test_key_cm", miniDFSCluster, conf); DFSTestUtil.createKey("test_key_db", miniDFSCluster, conf); hiveConf = new HiveConf(TestReplChangeManager.class); - hiveConf.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + hiveConf.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); hiveConf.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() - + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); + + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); cmroot = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmroot"; cmrootFallBack = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmrootFallback"; cmrootEncrypted = "cmrootEncrypted"; - hiveConf.set(HiveConf.ConfVars.REPLCMDIR.varname, cmroot); - hiveConf.set(HiveConf.ConfVars.REPLCMENCRYPTEDDIR.varname, cmrootEncrypted); - hiveConf.set(HiveConf.ConfVars.REPLCMFALLBACKNONENCRYPTEDDIR.varname, cmrootFallBack); + hiveConf.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmroot); + hiveConf.set(HiveConf.ConfVars.REPL_CM_ENCRYPTED_DIR.varname, cmrootEncrypted); + hiveConf.set(HiveConf.ConfVars.REPL_CM_FALLBACK_NONENCRYPTED_DIR.varname, cmrootFallBack); initReplChangeManager(); try { @@ -1253,14 +1253,14 @@ public void recycleFailureWithDifferentEncryptionZonesForCm() throws Throwable { @Test public void testClearerEncrypted() throws Exception { HiveConf hiveConfCmClearer = new HiveConf(TestReplChangeManager.class); - hiveConfCmClearer.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + hiveConfCmClearer.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); hiveConfCmClearer.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); - hiveConfCmClearer.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + hiveConfCmClearer.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() - + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); + + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); String cmrootCmClearer = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmrootClearer"; - hiveConfCmClearer.set(HiveConf.ConfVars.REPLCMDIR.varname, cmrootCmClearer); + hiveConfCmClearer.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmrootCmClearer); Warehouse warehouseCmClearer = new Warehouse(hiveConfCmClearer); FileSystem cmfs = new Path(cmrootCmClearer).getFileSystem(hiveConfCmClearer); cmfs.mkdirs(warehouseCmClearer.getWhRoot()); @@ -1359,21 +1359,21 @@ public void testClearerEncrypted() throws Exception { @Test public void testCmRootAclPermissions() throws Exception { HiveConf hiveConfAclPermissions = new HiveConf(TestReplChangeManager.class); - hiveConfAclPermissions.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + hiveConfAclPermissions.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); hiveConfAclPermissions.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); - hiveConfAclPermissions.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + hiveConfAclPermissions.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() - + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); + + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); String cmRootAclPermissions = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmRootAclPermissions"; - hiveConfAclPermissions.set(HiveConf.ConfVars.REPLCMDIR.varname, cmRootAclPermissions); + hiveConfAclPermissions.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmRootAclPermissions); Warehouse warehouseCmPermissions = new Warehouse(hiveConfAclPermissions); FileSystem cmfs = new Path(cmRootAclPermissions).getFileSystem(hiveConfAclPermissions); cmfs.mkdirs(warehouseCmPermissions.getWhRoot()); FileSystem fsWarehouse = warehouseCmPermissions.getWhRoot().getFileSystem(hiveConfAclPermissions); //change the group of warehouse for testing - Path warehouse = new Path(hiveConfAclPermissions.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + Path warehouse = new Path(hiveConfAclPermissions.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname)); fsWarehouse.setOwner(warehouse, null, "testgroup"); long now = System.currentTimeMillis(); @@ -1434,7 +1434,7 @@ public void testCmRootAclPermissions() throws Exception { return null; }); - String cmEncrypted = hiveConf.get(HiveConf.ConfVars.REPLCMENCRYPTEDDIR.varname, cmrootEncrypted); + String cmEncrypted = hiveConf.get(HiveConf.ConfVars.REPL_CM_ENCRYPTED_DIR.varname, cmrootEncrypted); AclStatus aclStatus = fsWarehouse.getAclStatus(new Path(dirTbl1 + Path.SEPARATOR + cmEncrypted)); AclStatus aclStatus2 = fsWarehouse.getAclStatus(new Path(dirTbl2 + Path.SEPARATOR + cmEncrypted)); AclStatus aclStatus3 = fsWarehouse.getAclStatus(new Path(dirTbl3 + Path.SEPARATOR + cmEncrypted)); @@ -1501,17 +1501,17 @@ public void testCmRootAclPermissions() throws Exception { @Test public void testCmrootEncrypted() throws Exception { HiveConf encryptedHiveConf = new HiveConf(TestReplChangeManager.class); - encryptedHiveConf.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + encryptedHiveConf.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); encryptedHiveConf.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); - encryptedHiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + encryptedHiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() - + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); + + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); String cmrootdirEncrypted = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmrootDirEncrypted"; - encryptedHiveConf.set(HiveConf.ConfVars.REPLCMDIR.varname, cmrootdirEncrypted); + encryptedHiveConf.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmrootdirEncrypted); FileSystem cmrootdirEncryptedFs = new Path(cmrootdirEncrypted).getFileSystem(hiveConf); cmrootdirEncryptedFs.mkdirs(new Path(cmrootdirEncrypted)); - encryptedHiveConf.set(HiveConf.ConfVars.REPLCMFALLBACKNONENCRYPTEDDIR.varname, cmrootFallBack); + encryptedHiveConf.set(HiveConf.ConfVars.REPL_CM_FALLBACK_NONENCRYPTED_DIR.varname, cmrootFallBack); //Create cm in encrypted zone EncryptionZoneUtils.createEncryptionZone(new Path(cmrootdirEncrypted), "test_key_db", conf); @@ -1562,11 +1562,11 @@ public void testCmrootEncrypted() throws Exception { @Test public void testCmrootFallbackEncrypted() throws Exception { HiveConf encryptedHiveConf = new HiveConf(TestReplChangeManager.class); - encryptedHiveConf.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + encryptedHiveConf.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); encryptedHiveConf.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); - encryptedHiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + encryptedHiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() - + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); + + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); String cmrootdirEncrypted = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmrootIsEncrypted"; String cmRootFallbackEncrypted = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmrootFallbackEncrypted"; @@ -1574,8 +1574,8 @@ public void testCmrootFallbackEncrypted() throws Exception { try { cmrootdirEncryptedFs.mkdirs(new Path(cmrootdirEncrypted)); cmrootdirEncryptedFs.mkdirs(new Path(cmRootFallbackEncrypted)); - encryptedHiveConf.set(HiveConf.ConfVars.REPLCMDIR.varname, cmrootdirEncrypted); - encryptedHiveConf.set(HiveConf.ConfVars.REPLCMFALLBACKNONENCRYPTEDDIR.varname, cmRootFallbackEncrypted); + encryptedHiveConf.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmrootdirEncrypted); + encryptedHiveConf.set(HiveConf.ConfVars.REPL_CM_FALLBACK_NONENCRYPTED_DIR.varname, cmRootFallbackEncrypted); //Create cm in encrypted zone EncryptionZoneUtils.createEncryptionZone(new Path(cmrootdirEncrypted), "test_key_db", conf); @@ -1600,19 +1600,19 @@ public void testCmrootFallbackEncrypted() throws Exception { @Test public void testCmrootFallbackRelative() throws Exception { HiveConf encryptedHiveConf = new HiveConf(TestReplChangeManager.class); - encryptedHiveConf.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + encryptedHiveConf.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); encryptedHiveConf.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); - encryptedHiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, + encryptedHiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() - + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); + + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); String cmrootdirEncrypted = "hdfs://" + miniDFSCluster.getNameNode().getHostAndPort() + "/cmrootIsEncrypted"; String cmRootFallbackEncrypted = "cmrootFallbackEncrypted"; FileSystem cmrootdirEncryptedFs = new Path(cmrootdirEncrypted).getFileSystem(encryptedHiveConf); try { cmrootdirEncryptedFs.mkdirs(new Path(cmrootdirEncrypted)); cmrootdirEncryptedFs.mkdirs(new Path(cmRootFallbackEncrypted)); - encryptedHiveConf.set(HiveConf.ConfVars.REPLCMDIR.varname, cmrootdirEncrypted); - encryptedHiveConf.set(HiveConf.ConfVars.REPLCMFALLBACKNONENCRYPTEDDIR.varname, cmRootFallbackEncrypted); + encryptedHiveConf.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmrootdirEncrypted); + encryptedHiveConf.set(HiveConf.ConfVars.REPL_CM_FALLBACK_NONENCRYPTED_DIR.varname, cmRootFallbackEncrypted); //Create cm in encrypted zone EncryptionZoneUtils.createEncryptionZone(new Path(cmrootdirEncrypted), "test_key_db", conf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java index 981f5fb4c211..69db7180cfff 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java @@ -70,7 +70,7 @@ public void setUp() throws Exception { DummyPreListener.class.getName()); testMetastoreDB = System.getProperty("java.io.tmpdir") + File.separator + "test_metastore-" + System.currentTimeMillis(); - System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + System.setProperty(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, "jdbc:derby:" + testMetastoreDB + ";create=true"); metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf, System.getProperty("test.tmp.dir", "target/tmp"), "derby"); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java index ebac38d10944..78304634c4ed 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestReplChangeManager.java @@ -97,11 +97,11 @@ private static void internalSetUpProvidePerm() throws Exception { configuration.set("dfs.client.use.datanode.hostname", "true"); permDdfs = new MiniDFSCluster.Builder(configuration).numDataNodes(2).format(true).build(); permhiveConf = new HiveConf(TestReplChangeManager.class); - permhiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, - "hdfs://" + permDdfs.getNameNode().getHostAndPort() + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); - permhiveConf.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + permhiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, + "hdfs://" + permDdfs.getNameNode().getHostAndPort() + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); + permhiveConf.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); permCmroot = "hdfs://" + permDdfs.getNameNode().getHostAndPort() + "/cmroot"; - permhiveConf.set(HiveConf.ConfVars.REPLCMDIR.varname, permCmroot); + permhiveConf.set(HiveConf.ConfVars.REPL_CM_DIR.varname, permCmroot); permhiveConf.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); permWarehouse = new Warehouse(permhiveConf); } @@ -109,11 +109,11 @@ private static void internalSetUpProvidePerm() throws Exception { private static void internalSetUp() throws Exception { m_dfs = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(2).format(true).build(); hiveConf = new HiveConf(TestReplChangeManager.class); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, - "hdfs://" + m_dfs.getNameNode().getHostAndPort() + HiveConf.ConfVars.METASTOREWAREHOUSE.defaultStrVal); - hiveConf.setBoolean(HiveConf.ConfVars.REPLCMENABLED.varname, true); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, + "hdfs://" + m_dfs.getNameNode().getHostAndPort() + HiveConf.ConfVars.METASTORE_WAREHOUSE.defaultStrVal); + hiveConf.setBoolean(HiveConf.ConfVars.REPL_CM_ENABLED.varname, true); cmroot = "hdfs://" + m_dfs.getNameNode().getHostAndPort() + "/cmroot"; - hiveConf.set(HiveConf.ConfVars.REPLCMDIR.varname, cmroot); + hiveConf.set(HiveConf.ConfVars.REPL_CM_DIR.varname, cmroot); hiveConf.setInt(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, 60); warehouse = new Warehouse(hiveConf); fs = new Path(cmroot).getFileSystem(hiveConf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestServerSpecificConfig.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestServerSpecificConfig.java index 17542f177c1e..e2fbf0450270 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestServerSpecificConfig.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestServerSpecificConfig.java @@ -181,7 +181,7 @@ private void setHiveSiteWithRemoteMetastore() throws IOException { FileOutputStream out = new FileOutputStream(hiveSite); HiveConf.setHiveSiteLocation(oldDefaultHiveSite); HiveConf defaultHiveConf = new HiveConf(); - defaultHiveConf.setVar(ConfVars.METASTOREURIS, "dummyvalue"); + defaultHiveConf.setVar(ConfVars.METASTORE_URIS, "dummyvalue"); // reset to the hive-site.xml values for following param defaultHiveConf.set("hive.dummyparam.test.server.specific.config.override", "from.hive-site.xml"); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java index af4f4bb36196..2f3e83d9e153 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java @@ -61,8 +61,7 @@ import org.junit.Assert; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; -import com.google.gson.JsonParser; -import org.json.JSONObject; + import org.junit.Before; import org.junit.After; import org.junit.Test; @@ -110,10 +109,10 @@ public void setUp() throws Exception { + File.separator + "mapred" + File.separator + "staging"); hiveConf.set("mapred.temp.dir", workDir + File.separator + this.getClass().getSimpleName() + File.separator + "mapred" + File.separator + "temp"); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, getWarehouseDir()); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, getWarehouseDir()); + hiveConf.setVar(HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); hiveConf.setBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK, true); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java index 36ba35f2aea0..245fc156512c 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java @@ -113,12 +113,12 @@ public String toString() { @Before public void setUp() throws Exception { hiveConf = new HiveConf(this.getClass()); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, TEST_WAREHOUSE_DIR); hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false); - hiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + hiveConf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true); @@ -344,7 +344,7 @@ public void testNonStandardConversion01() throws Exception { * data files in directly. * * Actually Insert Into ... select ... union all ... with - * HIVE_OPTIMIZE_UNION_REMOVE (and HIVEFETCHTASKCONVERSION="none"?) will create subdirs + * HIVE_OPTIMIZE_UNION_REMOVE (and HIVE_FETCH_TASK_CONVERSION="none"?) will create subdirs * but if writing to non acid table there is a merge task on MR (but not on Tez) */ @Ignore("HIVE-17214")//this consistently works locally but never in ptest.... @@ -757,7 +757,7 @@ public void testGetSplitsLocks() throws Exception { HiveConf modConf = new HiveConf(hiveConf); setupTez(modConf); modConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez"); - modConf.setVar(ConfVars.HIVEFETCHTASKCONVERSION, "more"); + modConf.setVar(ConfVars.HIVE_FETCH_TASK_CONVERSION, "more"); modConf.setVar(HiveConf.ConfVars.LLAP_DAEMON_SERVICE_HOSTS, "localhost"); // SessionState/Driver needs to be restarted with the Tez conf settings. @@ -818,7 +818,7 @@ public void testGetSplitsLocksWithMaterializedView() throws Exception { HiveConf modConf = new HiveConf(hiveConf); setupTez(modConf); modConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez"); - modConf.setVar(ConfVars.HIVEFETCHTASKCONVERSION, "more"); + modConf.setVar(ConfVars.HIVE_FETCH_TASK_CONVERSION, "more"); modConf.setVar(HiveConf.ConfVars.LLAP_DAEMON_SERVICE_HOSTS, "localhost"); // SessionState/Driver needs to be restarted with the Tez conf settings. @@ -868,7 +868,7 @@ public void testCrudMajorCompactionSplitGrouper() throws Exception { // make a clone of existing hive conf HiveConf confForTez = new HiveConf(hiveConf); setupTez(confForTez); // one-time setup to make query able to run with Tez - HiveConf.setVar(confForTez, HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + HiveConf.setVar(confForTez, HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); runStatementOnDriver("create transactional table " + tblName + " (a int, b int) clustered by (a) into 2 buckets " + "stored as ORC TBLPROPERTIES('bucketing_version'='2', 'transactional'='true'," + " 'transactional_properties'='default')", confForTez); @@ -1020,9 +1020,9 @@ public static void setupTez(HiveConf conf) { } private void setupMapJoin(HiveConf conf) { - conf.setBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN, true); - conf.setBoolVar(HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK, true); - conf.setLongVar(HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD, 100000); + conf.setBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN, true); + conf.setBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONALTASK, true); + conf.setLongVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD, 100000); } private List runStatementOnDriver(String stmt) throws Exception { diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestConstraintsMerge.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestConstraintsMerge.java index 12b626d09d3d..1e1a8d74ae1a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestConstraintsMerge.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestConstraintsMerge.java @@ -28,9 +28,7 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.tez.mapreduce.hadoop.MRJobConfig; import org.junit.After; -import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; @@ -73,12 +71,12 @@ public String toString() { @Before public void setUp() throws Exception { hiveConf = new HiveConf(this.getClass()); - hiveConf.set(ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(ConfVars.POSTEXECHOOKS.varname, ""); - hiveConf.set(ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + hiveConf.set(ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(ConfVars.POST_EXEC_HOOKS.varname, ""); + hiveConf.set(ConfVars.METASTORE_WAREHOUSE.varname, TEST_WAREHOUSE_DIR); hiveConf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, false); - hiveConf.setVar(ConfVars.HIVEMAPREDMODE, "nonstrict"); - hiveConf.setVar(ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + hiveConf.setVar(ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + hiveConf.setVar(ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); hiveConf .setVar(ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java index 7e54dde6f926..b22a3c0f3fde 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java @@ -83,10 +83,10 @@ public void setUp() throws Exception { // Test with remote metastore service int port = MetaStoreTestUtils.startMetaStoreWithRetry(); - conf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port); - conf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + conf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + port); + conf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); conf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, new URI(tmppath + "/warehouse").getPath()); + conf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, new URI(tmppath + "/warehouse").getPath()); // Initialize second mocked filesystem (implement only necessary stuff) // Physical files are resides in local file system in the similar location diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDatabaseTableDefault.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDatabaseTableDefault.java index 041be2d063f7..f5cbd1636962 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDatabaseTableDefault.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDatabaseTableDefault.java @@ -23,7 +23,6 @@ import java.io.File; import java.util.ArrayList; import java.util.List; -import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.FileUtils; @@ -36,9 +35,7 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.metastore.api.Table; import org.junit.After; -import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,7 +90,7 @@ public void setUp() throws Exception { HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.CREATE_TABLES_AS_ACID, true); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_CREATE_TABLES_AS_INSERT_ONLY, true); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); - hiveConf.set(HiveConf.ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT.varname, "ORC"); + hiveConf.set(HiveConf.ConfVars.HIVE_DEFAULT_MANAGED_FILEFORMAT.varname, "ORC"); hiveConf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName()); hiveConf.set(HiveConf.ConfVars.METASTORE_CLIENT_CAPABILITIES.varname, "HIVEFULLACIDREAD,HIVEFULLACIDWRITE,HIVECACHEINVALIDATE,HIVEMANAGESTATS,HIVEMANAGEDINSERTWRITE,HIVEMANAGEDINSERTREAD"); TestTxnDbUtil.setConfValues(hiveConf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMetaStoreLimitPartitionRequest.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMetaStoreLimitPartitionRequest.java index 05a104f53ea5..470b5ccdfd0e 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMetaStoreLimitPartitionRequest.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMetaStoreLimitPartitionRequest.java @@ -33,7 +33,6 @@ import java.util.Set; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.HMSHandler; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hive.jdbc.miniHS2.MiniHS2; @@ -72,7 +71,7 @@ public static void beforeTest() throws Exception { conf.setIntVar(HiveConf.ConfVars.METASTORE_LIMIT_PARTITION_REQUEST, PARTITION_REQUEST_LIMIT); conf.setBoolVar(HiveConf.ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN, true); conf.setBoolVar(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL, true); - conf.setBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING, true); + conf.setBoolVar(HiveConf.ConfVars.DYNAMIC_PARTITIONING, true); conf.setBoolVar(HiveConf.ConfVars.HIVE_CBO_ENABLED, false); miniHS2 = new MiniHS2.Builder().withConf(conf).build(); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java index 127de2301b16..6ae8239c667b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java @@ -188,7 +188,7 @@ public void testQueryloglocParentDirNotExist() throws Exception { try { String actualDir = parentTmpDir + "/test"; HiveConf conf = new HiveConf(SessionState.class); - conf.set(HiveConf.ConfVars.HIVEHISTORYFILELOC.toString(), actualDir); + conf.set(HiveConf.ConfVars.HIVE_HISTORY_FILE_LOC.toString(), actualDir); SessionState ss = new CliSessionState(conf); HiveHistory hiveHistory = new HiveHistoryImpl(ss); Path actualPath = new Path(actualDir); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplWithReadOnlyHook.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplWithReadOnlyHook.java index 379c53bb2f7d..f81f99d2c8ea 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplWithReadOnlyHook.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplWithReadOnlyHook.java @@ -22,7 +22,6 @@ import static org.apache.hadoop.hive.common.repl.ReplConst.READ_ONLY_HOOK; import static org.junit.Assert.assertEquals; -import com.google.common.collect.ImmutableMap; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; @@ -66,7 +65,7 @@ public static void classLevelSetup() throws Exception { acidEnableConf.put(HiveConf.ConfVars.REPL_RUN_DATA_COPY_TASKS_ON_TARGET.varname, "false"); acidEnableConf.put(HiveConf.ConfVars.REPL_RETAIN_CUSTOM_LOCATIONS_FOR_DB_ON_TARGET.varname, "false"); - acidEnableConf.put(HiveConf.ConfVars.PREEXECHOOKS.varname, READ_ONLY_HOOK); + acidEnableConf.put(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, READ_ONLY_HOOK); acidEnableConf.putAll(overrides); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java index 731eb9c6bd73..92879d5ebba3 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOnHDFSEncryptedZones.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.security.UserGroupInformation; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; @@ -119,7 +118,7 @@ public void targetAndSourceHaveDifferentEncryptionZoneKeys() throws Throwable { put(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS.varname, "false"); put(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER.varname, UserGroupInformation.getCurrentUser().getUserName()); - put(HiveConf.ConfVars.REPLDIR.varname, primary.repldDir); + put(HiveConf.ConfVars.REPL_DIR.varname, primary.repldDir); }}, "test_key123"); //read should pass without raw-byte distcp @@ -162,7 +161,7 @@ public void targetAndSourceHaveSameEncryptionZoneKeys() throws Throwable { put(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS.varname, "false"); put(HiveConf.ConfVars.HIVE_DISTCP_DOAS_USER.varname, UserGroupInformation.getCurrentUser().getUserName()); - put(HiveConf.ConfVars.REPLDIR.varname, primary.repldDir); + put(HiveConf.ConfVars.REPL_DIR.varname, primary.repldDir); }}, "test_key"); List dumpWithClause = Arrays.asList( diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOptimisedBootstrap.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOptimisedBootstrap.java index 91f8f76e1ff5..0d3178e8619b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOptimisedBootstrap.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationOptimisedBootstrap.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.hive.ql.parse.repl.metric.event.Status; import org.apache.hadoop.hive.ql.parse.repl.metric.event.ReplicationMetric; import org.apache.hadoop.hive.ql.parse.repl.metric.event.Stage; import org.apache.hadoop.hive.ql.parse.repl.metric.event.Metric; @@ -54,7 +53,6 @@ import org.junit.BeforeClass; import org.junit.Test; -import javax.annotation.Nullable; import java.io.File; import java.util.ArrayList; import java.util.Arrays; @@ -68,11 +66,6 @@ import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_ENABLE_BACKGROUND_THREAD; import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_TARGET_DB_PROPERTY; import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_FAILOVER_ENDPOINT; -import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_METRICS_FAILBACK_COUNT; -import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_METRICS_FAILOVER_COUNT; -import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_METRICS_LAST_FAILBACK_ENDTIME; -import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_METRICS_LAST_FAILBACK_STARTTIME; -import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_METRICS_LAST_FAILOVER_TYPE; import static org.apache.hadoop.hive.common.repl.ReplConst.TARGET_OF_REPLICATION; import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION; import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.EVENT_ACK_FILE; @@ -137,7 +130,7 @@ public void tearDown() throws Throwable { @Test public void testBuildTableDiffGeneration() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Create two external & two managed tables and do a bootstrap dump & load. WarehouseInstance.Tuple tuple = primary.run("use " + primaryDbName) .run("create external table t1 (id int)") @@ -202,7 +195,7 @@ public void testBuildTableDiffGeneration() throws Throwable { Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump tuple = replica.dump(replicatedDbName, withClause); @@ -263,7 +256,7 @@ public void testEmptyDiffForControlFailover() throws Throwable { // In case of control failover both A & B will be in sync, so the table diff should be created empty, without any // error. List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle(A->B) primary.dump(primaryDbName, withClause); @@ -293,7 +286,7 @@ public void testEmptyDiffForControlFailover() throws Throwable { Path newReplDir = new Path(replica.repldDir + "rev"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump tuple = replica.dump(replicatedDbName, withClause); @@ -317,7 +310,7 @@ public void testEmptyDiffForControlFailover() throws Throwable { @Test public void testFirstIncrementalMandatory() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Create one external and one managed tables and do a bootstrap dump. WarehouseInstance.Tuple tuple = primary.run("use " + primaryDbName) .run("create external table t1 (id int)") @@ -343,7 +336,7 @@ public void testFirstIncrementalMandatory() throws Throwable { Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a dump on cluster B, it should throw an exception, since the first incremental isn't done yet. try { @@ -355,13 +348,13 @@ public void testFirstIncrementalMandatory() throws Throwable { // Do a incremental cycle and check we don't get this exception. withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); primary.dump(primaryDbName, withClause); replica.load(replicatedDbName, primaryDbName, withClause); // Retrigger reverse dump, this time it should be successful and event ack should get created. withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); tuple = replica.dump(replicatedDbName, withClause); @@ -373,7 +366,7 @@ public void testFirstIncrementalMandatory() throws Throwable { @Test public void testFailureCasesInTableDiffGeneration() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle(A->B) primary.dump(primaryDbName, withClause); @@ -418,7 +411,7 @@ public void testFailureCasesInTableDiffGeneration() throws Throwable { Path newReplDir = new Path(replica.repldDir + "reverse"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Trigger dump on target cluster. @@ -505,7 +498,7 @@ public void testFailureCasesInTableDiffGeneration() throws Throwable { @Test public void testReverseReplicationFailureWhenSourceDbIsDropped() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle. primary.dump(primaryDbName, withClause); @@ -542,7 +535,7 @@ public void testReverseReplicationFailureWhenSourceDbIsDropped() throws Throwabl Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump, this should create event_ack file tuple = replica.dump(replicatedDbName, withClause); @@ -800,7 +793,7 @@ public void testReverseBootstrapWithFailedIncremental() throws Throwable { @Test public void testOverwriteDuringBootstrap() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle. primary.dump(primaryDbName, withClause); @@ -862,7 +855,7 @@ public void testOverwriteDuringBootstrap() throws Throwable { Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump tuple = replica.dump(replicatedDbName, withClause); @@ -928,7 +921,7 @@ public void testOverwriteDuringBootstrap() throws Throwable { @Test public void testTblMetricRegisterDuringSecondCycleOfOptimizedBootstrap() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(false); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); WarehouseInstance.Tuple tuple = primary.run("use " + primaryDbName) .run("create table t1_managed (id int) clustered by(id) into 3 buckets stored as orc " + "tblproperties (\"transactional\"=\"true\")") @@ -968,7 +961,7 @@ public void testTblMetricRegisterDuringSecondCycleOfOptimizedBootstrap() throws Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(false); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump @@ -1004,7 +997,7 @@ public void testTblMetricRegisterDuringSecondCycleOfOptimizedBootstrap() throws @Test public void testTblMetricRegisterDuringSecondLoadCycleOfOptimizedBootstrap() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(false); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); WarehouseInstance.Tuple tuple = primary.run("use " + primaryDbName) .run("create table t1_managed (id int) clustered by(id) into 3 buckets stored as orc " + "tblproperties (\"transactional\"=\"true\")") @@ -1044,7 +1037,7 @@ public void testTblMetricRegisterDuringSecondLoadCycleOfOptimizedBootstrap() thr Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(false); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump @@ -1085,7 +1078,7 @@ public void testTblMetricRegisterDuringSecondLoadCycleOfOptimizedBootstrap() thr @NotNull private List setUpFirstIterForOptimisedBootstrap() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle. primary.dump(primaryDbName, withClause); @@ -1210,7 +1203,7 @@ private List setUpFirstIterForOptimisedBootstrap() throws Throwable { Path newReplDir = new Path(replica.repldDir + "1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); // Do a reverse dump tuple = replica.dump(replicatedDbName, withClause); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java index 9345d34bc096..60788ad75ca4 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.common.repl.ReplScope; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.repl.ReplAck; -import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.StringAppender; import org.apache.hadoop.hive.ql.parse.repl.metric.MetricCollector; import org.apache.hadoop.hive.ql.parse.repl.metric.event.Metadata; @@ -203,16 +202,16 @@ static void internalBeforeClassSetup(Map additionalProperties) MetastoreConf.setBoolVar(hconf, MetastoreConf.ConfVars.EVENT_DB_NOTIFICATION_API_AUTH, false); hconf.set(MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS.getHiveName(), DBNOTIF_LISTENER_CLASSNAME); // turn on db notification listener on metastore - hconf.setBoolVar(HiveConf.ConfVars.REPLCMENABLED, true); + hconf.setBoolVar(HiveConf.ConfVars.REPL_CM_ENABLED, true); hconf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true); - hconf.setVar(HiveConf.ConfVars.REPLCMDIR, TEST_PATH + "/cmroot/"); + hconf.setVar(HiveConf.ConfVars.REPL_CM_DIR, TEST_PATH + "/cmroot/"); proxySettingName = "hadoop.proxyuser." + Utils.getUGI().getShortUserName() + ".hosts"; hconf.set(proxySettingName, "*"); MetastoreConf.setBoolVar(hconf, MetastoreConf.ConfVars.EVENT_DB_NOTIFICATION_API_AUTH, false); - hconf.setVar(HiveConf.ConfVars.REPLDIR,TEST_PATH + "/hrepl/"); + hconf.setVar(HiveConf.ConfVars.REPL_DIR,TEST_PATH + "/hrepl/"); hconf.set(MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES.getHiveName(), "3"); - hconf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hconf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hconf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hconf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); hconf.set(HiveConf.ConfVars.HIVE_IN_TEST_REPL.varname, "true"); hconf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, true); hconf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); @@ -221,13 +220,13 @@ static void internalBeforeClassSetup(Map additionalProperties) hconf.set(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL.varname, "org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore"); hconf.set(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname, "/tmp/warehouse/external"); - hconf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, true); + hconf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, true); hconf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, true); hconf.setBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE, true); hconf.setBoolVar(HiveConf.ConfVars.REPL_RUN_DATA_COPY_TASKS_ON_TARGET, false); hconf.setBoolVar(HiveConf.ConfVars.REPL_BATCH_INCREMENTAL_EVENTS, false); - System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, " "); additionalProperties.forEach((key, value) -> { hconf.set(key, value); @@ -247,7 +246,7 @@ static void internalBeforeClassSetup(Map additionalProperties) FileUtils.deleteDirectory(new File("metastore_db2")); HiveConf hconfMirrorServer = new HiveConf(); - hconfMirrorServer.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, "jdbc:derby:;databaseName=metastore_db2;create=true"); + hconfMirrorServer.set(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY.varname, "jdbc:derby:;databaseName=metastore_db2;create=true"); MetaStoreTestUtils.startMetaStoreWithRetry(hconfMirrorServer, true); hconfMirror = new HiveConf(hconf); MetastoreConf.setBoolVar(hconfMirror, MetastoreConf.ConfVars.EVENT_DB_NOTIFICATION_API_AUTH, false); @@ -4050,7 +4049,7 @@ public void testDeleteStagingDir() throws IOException { @Override public boolean accept(Path path) { - return path.getName().startsWith(HiveConf.getVar(hconf, HiveConf.ConfVars.STAGINGDIR)); + return path.getName().startsWith(HiveConf.getVar(hconf, HiveConf.ConfVars.STAGING_DIR)); } }; FileStatus[] statuses = fs.listStatus(path, filter); @@ -4230,7 +4229,7 @@ public void testRecycleFileDropTempTable() throws IOException { run("INSERT INTO " + dbName + ".normal values (1)", driver); run("DROP TABLE " + dbName + ".normal", driver); - String cmDir = hconf.getVar(HiveConf.ConfVars.REPLCMDIR); + String cmDir = hconf.getVar(HiveConf.ConfVars.REPL_CM_DIR); Path path = new Path(cmDir); FileSystem fs = path.getFileSystem(hconf); ContentSummary cs = fs.getContentSummary(path); @@ -4285,7 +4284,7 @@ public void testLoadCmPathMissing() throws Exception { run("DROP TABLE " + dbName + ".normal", driver); - String cmDir = hconf.getVar(HiveConf.ConfVars.REPLCMDIR); + String cmDir = hconf.getVar(HiveConf.ConfVars.REPL_CM_DIR); Path path = new Path(cmDir); FileSystem fs = path.getFileSystem(hconf); ContentSummary cs = fs.getContentSummary(path); @@ -4370,7 +4369,7 @@ public void testDDLTasksInParallel() throws Throwable{ StringAppender appender = null; LoggerConfig loggerConfig = null; try { - driverMirror.getConf().set(HiveConf.ConfVars.EXECPARALLEL.varname, "true"); + driverMirror.getConf().set(HiveConf.ConfVars.EXEC_PARALLEL.varname, "true"); logger = LogManager.getLogger("hive.ql.metadata.Hive"); oldLevel = logger.getLevel(); ctx = (LoggerContext) LogManager.getContext(false); @@ -4403,7 +4402,7 @@ public void testDDLTasksInParallel() throws Throwable{ assertEquals(count, 2); appender.reset(); } finally { - driverMirror.getConf().set(HiveConf.ConfVars.EXECPARALLEL.varname, "false"); + driverMirror.getConf().set(HiveConf.ConfVars.EXEC_PARALLEL.varname, "false"); loggerConfig.setLevel(oldLevel); ctx.updateLoggers(); appender.removeFromLogger(logger.getName()); @@ -4414,7 +4413,7 @@ public void testDDLTasksInParallel() throws Throwable{ public void testRecycleFileNonReplDatabase() throws IOException { String dbName = createDBNonRepl(testName.getMethodName(), driver); - String cmDir = hconf.getVar(HiveConf.ConfVars.REPLCMDIR); + String cmDir = hconf.getVar(HiveConf.ConfVars.REPL_CM_DIR); Path path = new Path(cmDir); FileSystem fs = path.getFileSystem(hconf); ContentSummary cs = fs.getContentSummary(path); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java index 5546f41447e2..6d9fea15fd55 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore.BehaviourInjection; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.Assert; @@ -140,7 +139,7 @@ public Boolean apply(@Nullable CallerArguments args) { } finally { InjectableBehaviourObjectStore.resetAlterTableModifier(); } - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = TestReplicationScenarios.getNonRecoverablePath(baseDumpDir, primaryDbName, primary.hiveConf); if(nonRecoverablePath != null){ baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java index fb2ad07acb7a..893ccd48786a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; import org.apache.hadoop.hive.ql.parse.repl.PathBuilder; import org.apache.hadoop.hive.ql.parse.repl.dump.Utils; -import org.apache.hadoop.hive.ql.processors.CommandProcessorException; import org.apache.hadoop.hive.ql.util.DependencyResolver; import org.apache.hadoop.security.UserGroupInformation; import org.junit.Assert; @@ -77,7 +76,6 @@ import static org.apache.hadoop.hive.common.repl.ReplConst.SOURCE_OF_REPLICATION; import static org.apache.hadoop.hive.ql.exec.repl.ReplAck.LOAD_ACKNOWLEDGEMENT; import static org.apache.hadoop.hive.ql.exec.repl.ReplAck.NON_RECOVERABLE_MARKER; -import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -515,7 +513,7 @@ public void testParallelExecutionOfReplicationBootStrapLoad() throws Throwable { .run("create table t3 (rank int)") .dump(primaryDbName); - replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXECPARALLEL, true); + replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXEC_PARALLEL, true); replica.load(replicatedDbName, primaryDbName) .run("use " + replicatedDbName) .run("repl status " + replicatedDbName) @@ -525,7 +523,7 @@ public void testParallelExecutionOfReplicationBootStrapLoad() throws Throwable { .run("select country from t2") .verifyResults(Arrays.asList("india", "australia", "russia", "uk", "us", "france", "japan", "china")); - replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXECPARALLEL, false); + replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXEC_PARALLEL, false); } @Test @@ -710,8 +708,8 @@ public void testBootStrapDumpOfWarehouse() throws Throwable { public void testReplLoadFromSourceUsingWithClause() throws Throwable { HiveConf replicaConf = replica.getConf(); List withConfigs = Arrays.asList( - "'hive.metastore.warehouse.dir'='" + replicaConf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) + "'", - "'hive.metastore.uris'='" + replicaConf.getVar(HiveConf.ConfVars.METASTOREURIS) + "'", + "'hive.metastore.warehouse.dir'='" + replicaConf.getVar(HiveConf.ConfVars.METASTORE_WAREHOUSE) + "'", + "'hive.metastore.uris'='" + replicaConf.getVar(HiveConf.ConfVars.METASTORE_URIS) + "'", "'hive.repl.replica.functions.root.dir'='" + replicaConf.getVar(HiveConf.ConfVars.REPL_FUNCTIONS_ROOT_DIR) + "'"); //////////// Bootstrap //////////// @@ -1654,7 +1652,7 @@ public Boolean apply(@Nullable CallerArguments args) { // is loaded before t2. So that scope is set to table in first iteration for table t1. In the next iteration, it // loads only remaining partitions of t2, so that the table tracker has no tasks. - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); if(nonRecoverablePath != null){ baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -1994,7 +1992,7 @@ public void testRangerReplicationRetryExhausted() throws Throwable { ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode()); } //Delete non recoverable marker to fix this - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2041,7 +2039,7 @@ public void testFailureUnsupportedAuthorizerReplication() throws Throwable { ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode()); } //Delete non recoverable marker to fix this - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2160,7 +2158,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_ATLAS_ENDPOINT.varname, true); ensureFailedAdminRepl(getAtlasClause(confMap), true); //Delete non recoverable marker to fix this - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2170,7 +2168,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_ATLAS_REPLICATED_TO_DB.varname, true); ensureFailedAdminRepl(getAtlasClause(confMap), true); //Delete non recoverable marker to fix this - baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2178,7 +2176,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_SOURCE_CLUSTER_NAME.varname, true); ensureFailedAdminRepl(getAtlasClause(confMap), true); //Delete non recoverable marker to fix this - baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2186,7 +2184,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_TARGET_CLUSTER_NAME.varname, true); ensureFailedAdminRepl(getAtlasClause(confMap), true); //Delete non recoverable marker to fix this - baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2200,7 +2198,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_ATLAS_ENDPOINT.varname, false); ensureFailedAdminRepl(getAtlasClause(confMap), false); //Delete non recoverable marker to fix this - baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2210,7 +2208,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_SOURCE_CLUSTER_NAME.varname, false); ensureFailedAdminRepl(getAtlasClause(confMap), false); //Delete non recoverable marker to fix this - baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -2218,7 +2216,7 @@ public void testAtlasMissingConfigs() throws Throwable { ensureFailedReplOperation(getAtlasClause(confMap), HiveConf.ConfVars.REPL_TARGET_CLUSTER_NAME.varname, false); ensureFailedAdminRepl(getAtlasClause(confMap), false); //Delete non recoverable marker to fix this - baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); nonRecoverablePath = getNonRecoverablePath(baseDumpDir, primaryDbName); Assert.assertNotNull(nonRecoverablePath); baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExclusiveReplica.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExclusiveReplica.java index 8710e2c70a0d..9645f8d03fe3 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExclusiveReplica.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExclusiveReplica.java @@ -96,7 +96,7 @@ public void tearDown() throws Throwable { @Test public void testTargetEventIdGenerationAfterFirstIncrementalInOptFailover() throws Throwable { List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle(A->B) primary.dump(primaryDbName, withClause); @@ -156,7 +156,7 @@ public void testTargetEventIdGenerationAfterFirstIncrementalInOptFailover() thro Path newReplDir = new Path(replica.repldDir + "reverse1"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); tuple = replica.dump(replicatedDbName); @@ -177,7 +177,7 @@ public void testTargetEventIdGenerationAfterFirstIncrementalInOptFailover() thro public void testTargetEventIdGenerationInOptmisedFailover() throws Throwable { // Do a a cycle of bootstrap dump & load. List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle(A->B) primary.dump(primaryDbName, withClause); @@ -265,7 +265,7 @@ public void testTargetEventIdGenerationInOptmisedFailover() throws Throwable { Path newReplDir = new Path(replica.repldDir + "reverse01"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); tuple = replica.dump(replicatedDbName, withClause); @@ -285,7 +285,7 @@ public void testTargetEventIdGenerationInOptmisedFailover() throws Throwable { public void testTargetEventIdWithNotificationsExpiredInOptimisedFailover() throws Throwable { // Do a a cycle of bootstrap dump & load. List withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + primary.repldDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + primary.repldDir + "'"); // Do a bootstrap cycle(A->B) primary.dump(primaryDbName, withClause); @@ -347,7 +347,7 @@ public NotificationEventResponse apply(@Nullable NotificationEventResponse event Path newReplDir = new Path(replica.repldDir + "reverse01"); replicaFs.mkdirs(newReplDir); withClause = ReplicationTestUtils.includeExternalTableClause(true); - withClause.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + newReplDir + "'"); + withClause.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + newReplDir + "'"); try { replica.dump(replicatedDbName, withClause); @@ -1091,7 +1091,7 @@ private void verifyTableDataExists(WarehouseInstance warehouse, Path dbDataPath, private List getStagingLocationConfig(String stagingLoc, boolean addDistCpConfigs) throws IOException { List confList = new ArrayList<>(); - confList.add("'" + HiveConf.ConfVars.REPLDIR.varname + "'='" + stagingLoc + "'"); + confList.add("'" + HiveConf.ConfVars.REPL_DIR.varname + "'='" + stagingLoc + "'"); if (addDistCpConfigs) { confList.add("'" + HiveConf.ConfVars.HIVE_EXEC_COPYFILE_MAXSIZE.varname + "'='1'"); confList.add("'" + HiveConf.ConfVars.HIVE_EXEC_COPYFILE_MAXNUMFILES.varname + "'='0'"); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExternalTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExternalTables.java index 9eefd04e7f9a..8badc4c2895b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExternalTables.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosExternalTables.java @@ -1026,7 +1026,7 @@ public Boolean apply(@Nullable CallerArguments args) { InjectableBehaviourObjectStore.resetAlterTableModifier(); } - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = TestReplicationScenarios.getNonRecoverablePath(baseDumpDir, primaryDbName, primary.hiveConf); if(nonRecoverablePath != null){ baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -1302,7 +1302,7 @@ public void testExternalTableBaseDirMandatory() throws Throwable { ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode()); } //delete non recoverable marker - Path dumpPath = new Path(primary.hiveConf.get(HiveConf.ConfVars.REPLDIR.varname), + Path dumpPath = new Path(primary.hiveConf.get(HiveConf.ConfVars.REPL_DIR.varname), Base64.getEncoder().encodeToString(primaryDbName.toLowerCase() .getBytes(StandardCharsets.UTF_8.name()))); FileSystem fs = dumpPath.getFileSystem(conf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosUsingSnapshots.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosUsingSnapshots.java index cdc87733581f..f941d183b82d 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosUsingSnapshots.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosUsingSnapshots.java @@ -488,7 +488,7 @@ public void testFailureScenarios() throws Throwable { // Ignore } // Check if there is a non-recoverable error or not. - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = TestReplicationScenarios.getNonRecoverablePath(baseDumpDir, primaryDbName, primary.hiveConf); assertTrue(fs.exists(nonRecoverablePath)); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java index ff7733d2b9b6..2c0a70398d42 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java @@ -137,7 +137,7 @@ public void testAcidTablesReplLoadBootstrapIncr() throws Throwable { primary.run("create scheduled query s1_t1 every 5 seconds as repl dump " + primaryDbName); replica.run("create scheduled query s2_t1 every 5 seconds as repl load " + primaryDbName + " INTO " + replicatedDbName); - Path dumpRoot = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR), + Path dumpRoot = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR), Base64.getEncoder().encodeToString(primaryDbName.toLowerCase().getBytes(StandardCharsets.UTF_8.name()))); FileSystem fs = FileSystem.get(dumpRoot.toUri(), primary.hiveConf); @@ -208,7 +208,7 @@ public void testExternalTablesReplLoadBootstrapIncr() throws Throwable { primary.run("create scheduled query s1_t2 every 5 seconds as repl dump " + primaryDbName + withClause); replica.run("create scheduled query s2_t2 every 5 seconds as repl load " + primaryDbName + " INTO " + replicatedDbName + withClause); - Path dumpRoot = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR), + Path dumpRoot = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR), Base64.getEncoder().encodeToString(primaryDbName.toLowerCase().getBytes(StandardCharsets.UTF_8.name()))); FileSystem fs = FileSystem.get(dumpRoot.toUri(), primary.hiveConf); next = Integer.parseInt(ReplDumpWork.getTestInjectDumpDir()) + 1; diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java index e23c542d670b..d842385f48ed 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore.CallerArguments; import org.apache.hadoop.hive.ql.metadata.HiveMetaStoreClientWithLocalCache; import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.After; @@ -46,7 +45,6 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.junit.Ignore; import org.junit.Assert; import java.io.IOException; @@ -337,7 +335,7 @@ private String dumpLoadVerify(List tableNames, String lastReplicationId, // Load, if necessary changing configuration. if (parallelLoad) { - replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXECPARALLEL, true); + replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXEC_PARALLEL, true); } // Fail load if for testing failure and retry scenario. Fail the load while setting @@ -350,7 +348,7 @@ private String dumpLoadVerify(List tableNames, String lastReplicationId, } } - Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPLDIR)); + Path baseDumpDir = new Path(primary.hiveConf.getVar(HiveConf.ConfVars.REPL_DIR)); Path nonRecoverablePath = TestReplicationScenarios.getNonRecoverablePath(baseDumpDir, primaryDbName, primary.hiveConf); if(nonRecoverablePath != null){ baseDumpDir.getFileSystem(primary.hiveConf).delete(nonRecoverablePath, true); @@ -366,7 +364,7 @@ private String dumpLoadVerify(List tableNames, String lastReplicationId, } if (parallelLoad) { - replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXECPARALLEL, false); + replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXEC_PARALLEL, false); } // Test statistics diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java index e7701b54ca6e..ac656b45cff7 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java @@ -129,35 +129,35 @@ private void initialize(String cmRoot, String externalTableWarehouseRoot, String Map overridesForHiveConf) throws Exception { hiveConf = new HiveConf(miniDFSCluster.getConfiguration(0), TestReplicationScenarios.class); - String metaStoreUri = System.getProperty("test." + HiveConf.ConfVars.METASTOREURIS.varname); + String metaStoreUri = System.getProperty("test." + HiveConf.ConfVars.METASTORE_URIS.varname); if (metaStoreUri != null) { - hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreUri); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, metaStoreUri); return; } // hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, hiveInTest); // turn on db notification listener on meta store - hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, warehouseRoot); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, warehouseRoot); hiveConf.setVar(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL, externalTableWarehouseRoot); hiveConf.setVar(HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS, LISTENER_CLASS); - hiveConf.setBoolVar(HiveConf.ConfVars.REPLCMENABLED, true); + hiveConf.setBoolVar(HiveConf.ConfVars.REPL_CM_ENABLED, true); hiveConf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true); - hiveConf.setVar(HiveConf.ConfVars.REPLCMDIR, cmRoot); + hiveConf.setVar(HiveConf.ConfVars.REPL_CM_DIR, cmRoot); hiveConf.setVar(HiveConf.ConfVars.REPL_FUNCTIONS_ROOT_DIR, functionsRoot); hiveConf.setBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE, false); - hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, + hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY, "jdbc:derby:memory:${test.tmp.dir}/APP;create=true"); - hiveConf.setVar(HiveConf.ConfVars.REPLDIR, this.repldDir); - hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.setVar(HiveConf.ConfVars.REPL_DIR, this.repldDir); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); if (!hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER).equals("org.apache.hadoop.hive.ql.lockmgr.DbTxnManager")) { hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); } hiveConf.set(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL.varname, "org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore"); - System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); - System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, " "); for (Map.Entry entry : overridesForHiveConf.entrySet()) { hiveConf.set(entry.getKey(), entry.getValue()); @@ -181,14 +181,14 @@ private void initialize(String cmRoot, String externalTableWarehouseRoot, String */ - /*hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, "jdbc:mysql://localhost:3306/APP"); + /*hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY, "jdbc:mysql://localhost:3306/APP"); hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, "com.mysql.jdbc.Driver"); - hiveConf.setVar(HiveConf.ConfVars.METASTOREPWD, "hivepassword"); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_PWD, "hivepassword"); hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME, "hiveuser");*/ - /*hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY,"jdbc:postgresql://localhost/app"); + /*hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY,"jdbc:postgresql://localhost/app"); hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, "org.postgresql.Driver"); - hiveConf.setVar(HiveConf.ConfVars.METASTOREPWD, "password"); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_PWD, "password"); hiveConf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME, "postgres");*/ driver = DriverFactory.newDriver(hiveConf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java index b3383d923bec..b2e2678a8165 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.MetaStoreTestUtils; import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.ql.DriverFactory; import org.apache.hadoop.hive.ql.IDriver; import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener; @@ -77,11 +76,11 @@ public void setUp() throws Exception { // Turn off client-side authorization clientHiveConf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED,false); - clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); ugi = Utils.getUGI(); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java index b78c348c5203..0235aef1d54b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.ql.DriverFactory; import org.apache.hadoop.hive.ql.IDriver; import org.apache.hadoop.hive.ql.security.DummyHiveMetastoreAuthorizationProvider.AuthCallContext; @@ -68,12 +67,12 @@ public void setUp() throws Exception { clientHiveConf = new HiveConf(this.getClass()); - clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port); - clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + clientHiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + port); + clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); SessionState.start(new CliSessionState(clientHiveConf)); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java index 948ab4d10370..120d967a4754 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java @@ -80,13 +80,13 @@ public void setUp() throws Exception { clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER.varname, InjectableDummyAuthenticator.class.getName()); clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS.varname, ""); - clientHiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); - clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port); - clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + clientHiveConf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + clientHiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + port); + clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); ugi = Utils.getUGI(); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java index 3fa1d0d5b50d..72a953fea425 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java @@ -113,12 +113,12 @@ public void setUp() throws Exception { // Turn off client-side authorization clientHiveConf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED,false); - clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port); - clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + clientHiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + port); + clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); ugi = Utils.getUGI(); @@ -175,7 +175,7 @@ public void testSimplePrivileges() throws Exception { String tblName = getTestTableName(); String userName = setupUser(); String loc = clientHiveConf.get(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname) + "/" + dbName; - String mLoc = clientHiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname) + "/" + dbName; + String mLoc = clientHiveConf.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname) + "/" + dbName; allowCreateDatabase(userName); driver.run("create database " + dbName + " location '" + loc + "' managedlocation '" + mLoc + "'"); Database db = msc.getDatabase(dbName); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreClientSideAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreClientSideAuthorizationProvider.java index dbd71cb0ceb4..b166df33b8bd 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreClientSideAuthorizationProvider.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreClientSideAuthorizationProvider.java @@ -63,13 +63,13 @@ public void setUp() throws Exception { clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER.varname, InjectableDummyAuthenticator.class.getName()); clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS.varname, ""); - clientHiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); - clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port); - clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + clientHiveConf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + clientHiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + port); + clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, 3); clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); - clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + clientHiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); ugi = Utils.getUGI(); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java index d1e80698efa9..1a5a840c8680 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.MetaStoreTestUtils; import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.ql.DriverFactory; import org.apache.hadoop.hive.ql.IDriver; import org.apache.hadoop.hive.ql.security.DummyHiveMetastoreAuthorizationProvider.AuthCallContext; @@ -63,7 +62,7 @@ public static void setUp() throws Exception { clientHiveConf = new HiveConf(); - clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port); + clientHiveConf.setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + port); clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); SessionState.start(new CliSessionState(clientHiveConf)); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java index b7148fa98e1e..0d939af9de46 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java @@ -107,7 +107,7 @@ public static void beforeTest() throws Exception { conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, true); conf.setVar(ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName()); conf.setBoolVar(ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED, true); - conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + conf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); conf.setBoolVar(ConfVars.HIVE_TEST_AUTHORIZATION_SQLSTD_HS2_MODE, true); conf.setBoolVar(ConfVars.HIVE_ZOOKEEPER_KILLQUERY_ENABLE, false); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/session/TestClearDanglingScratchDir.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/session/TestClearDanglingScratchDir.java index 8645812ab95f..8e09d9697f4a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/session/TestClearDanglingScratchDir.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/session/TestClearDanglingScratchDir.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.util.Shell; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -56,12 +55,12 @@ static public void oneTimeSetup() throws Exception { conf.set(HiveConf.ConfVars.HIVE_SCRATCH_DIR_LOCK.toString(), "true"); conf.set(HiveConf.ConfVars.METASTORE_AUTO_CREATE_ALL.toString(), "true"); LoggerFactory.getLogger("SessionState"); - conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, + conf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, new Path(System.getProperty("test.tmp.dir"), "warehouse").toString()); conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, m_dfs.getFileSystem().getUri().toString()); - scratchDir = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR)); + scratchDir = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR)); m_dfs.getFileSystem().mkdirs(scratchDir); m_dfs.getFileSystem().setPermission(scratchDir, new FsPermission("777")); } @@ -163,7 +162,7 @@ public void testLocalDanglingFilesCleaning() throws Exception { // Simulating hdfs dangling dir and its inuse.lck file // Note: Give scratch dirs all the write permissions FsPermission allPermissions = new FsPermission((short)00777); - customScratchDir = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR)); + customScratchDir = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR)); Utilities.createDirsWithPermission(conf, customScratchDir, allPermissions, true); Path hdfsRootDir = new Path(customScratchDir + l + userName + l + hdfs); Path hdfsSessionDir = new Path(hdfsRootDir + l + userName + l + appId); @@ -171,7 +170,7 @@ public void testLocalDanglingFilesCleaning() throws Exception { fs.create(hdfsSessionLock); // Simulating local dangling files - customLocalTmpDir = new Path (HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR)); + customLocalTmpDir = new Path (HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); Path localSessionDir = new Path(customLocalTmpDir + l + appId); Path localPipeOutFileRemove = new Path(customLocalTmpDir + l + appId + "-started-with-session-name.pipeout"); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java index 5a3324831e0d..03c6f70e0fe1 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorOnTezTest.java @@ -45,7 +45,6 @@ import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; -import java.io.EOFException; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -102,11 +101,11 @@ protected void setupWithConf(HiveConf hiveConf) throws Exception { if (!(new File(TEST_WAREHOUSE_DIR).mkdirs())) { throw new RuntimeException("Could not create " + TEST_WAREHOUSE_DIR); } - hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, ""); - hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, ""); - hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, TEST_WAREHOUSE_DIR); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.POST_EXEC_HOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, TEST_WAREHOUSE_DIR); + hiveConf.setVar(HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.TXN_OPENTXN_TIMEOUT, 2, TimeUnit.SECONDS); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java index 4a7bb34bad1a..67af2443b259 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java @@ -52,9 +52,9 @@ public class TestCleanerWithReplication extends CompactorTest { public void setup() throws Exception { HiveConf conf = new HiveConf(); conf.set("fs.defaultFS", miniDFSCluster.getFileSystem().getUri().toString()); - conf.setBoolVar(HiveConf.ConfVars.REPLCMENABLED, true); + conf.setBoolVar(HiveConf.ConfVars.REPL_CM_ENABLED, true); setup(conf); - cmRootDirectory = new Path(conf.get(HiveConf.ConfVars.REPLCMDIR.varname)); + cmRootDirectory = new Path(conf.get(HiveConf.ConfVars.REPL_CM_DIR.varname)); if (!fs.exists(cmRootDirectory)) { fs.mkdirs(cmRootDirectory); } diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorBase.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorBase.java index a57a817e1612..3027028a0798 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorBase.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactorBase.java @@ -81,12 +81,12 @@ public void setup() throws Exception { } HiveConf hiveConf = new HiveConf(this.getClass()); - hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, ""); - hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, ""); - hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, TEST_WAREHOUSE_DIR); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + hiveConf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.POST_EXEC_HOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_WAREHOUSE, TEST_WAREHOUSE_DIR); + hiveConf.setVar(HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, false); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, true); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java index 1342b25916b4..a1885423a9d9 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java @@ -885,10 +885,10 @@ public void testMajorCompactionNotPartitionedWithoutBuckets() throws Exception { Assert.assertEquals("pre-compaction bucket 0", expectedRsBucket0, testDataProvider.getBucketData(tblName, "536870912")); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); // Run major compaction and cleaner CompactorTestUtil.runCompaction(conf, dbName, tblName, CompactionType.MAJOR, true); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); verifySuccessfulCompaction(1); @@ -2215,8 +2215,8 @@ public void testCompactionWithSchemaEvolutionAndBuckets() throws Exception { @Test public void testCompactionWithSchemaEvolutionNoBucketsMultipleReducers() throws Exception { HiveConf hiveConf = new HiveConf(conf); - hiveConf.setIntVar(HiveConf.ConfVars.MAXREDUCERS, 2); - hiveConf.setIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS, 2); + hiveConf.setIntVar(HiveConf.ConfVars.MAX_REDUCERS, 2); + hiveConf.setIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS, 2); driver = DriverFactory.newDriver(hiveConf); String dbName = "default"; String tblName = "testCompactionWithSchemaEvolutionNoBucketsMultipleReducers"; @@ -2915,10 +2915,10 @@ public void testCompactionWithCreateTableProps() throws Exception { CompactionRequest rqst = new CompactionRequest(dbName, tblName, CompactionType.MAJOR); CompactionResponse resp = txnHandler.compact(rqst); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); // Run major compaction and cleaner runWorker(conf); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); @@ -2981,10 +2981,10 @@ public void testCompactionWithAlterTableProps() throws Exception { // Get all data before compaction is run List expectedData = testDP.getAllData(tblName); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); // Run major compaction and cleaner runWorker(conf); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorOnTez.java index 14197a5326d2..90d3cb1cc55a 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorOnTez.java @@ -59,10 +59,10 @@ public void testCompactorGatherStats() throws Exception{ executeStatementOnDriver("analyze table " + dbName + "." + tableName + " compute statistics for columns", driver); executeStatementOnDriver("insert into " + dbName + "." + tableName + " values(2)", driver); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); // Run major compaction and cleaner CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MAJOR, false); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); verifySuccessfulCompaction(1); @@ -76,10 +76,10 @@ public void testCompactorGatherStats() throws Exception{ executeStatementOnDriver("alter table " + dbName + "." + tableName + " set tblproperties('compactor.mapred.job.queue.name'='" + CUSTOM_COMPACTION_QUEUE + "')", driver); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); // Run major compaction and cleaner CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MAJOR, false); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); verifySuccessfulCompaction(2); @@ -90,10 +90,10 @@ public void testCompactorGatherStats() throws Exception{ assertEquals("Value should contain new data", 1, colStats.get(0).getStatsData().getLongStats().getLowValue()); executeStatementOnDriver("insert into " + dbName + "." + tableName + " values(4)", driver); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MAJOR, false, Collections.singletonMap("compactor.mapred.job.queue.name", CUSTOM_COMPACTION_QUEUE)); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); verifySuccessfulCompaction(3); diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java index 0a57c4588337..55d36ed57bcc 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java @@ -85,11 +85,11 @@ public TestMmCompactorOnTez() { CompactorTestUtil.getBaseOrDeltaNames(fs, AcidUtils.deltaFileFilter, table, null)); if (isTez(conf)) { - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, HiveProtoLoggingHook.class.getName()); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, HiveProtoLoggingHook.class.getName()); } // Run a compaction CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MINOR, true); - conf.setVar(HiveConf.ConfVars.PREEXECHOOKS, StringUtils.EMPTY); + conf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, StringUtils.EMPTY); CompactorTestUtil.runCleaner(conf); verifySuccessulTxn(1); @@ -480,8 +480,8 @@ private void testMmMinorCompactionPartitionedWithBuckets(String fileFormat) thro @Test public void testMmMinorCompactionWithSchemaEvolutionNoBucketsMultipleReducers() throws Exception { HiveConf hiveConf = new HiveConf(conf); - hiveConf.setIntVar(HiveConf.ConfVars.MAXREDUCERS, 2); - hiveConf.setIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS, 2); + hiveConf.setIntVar(HiveConf.ConfVars.MAX_REDUCERS, 2); + hiveConf.setIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS, 2); driver = DriverFactory.newDriver(hiveConf); String dbName = "default"; String tblName = "testMmMinorCompactionWithSchemaEvolutionNoBucketsMultipleReducers"; @@ -638,7 +638,7 @@ private void verifyAllContents(String tblName, TestDataProvider dataProvider, * Set to true to cause all transactions to be rolled back, until set back to false. */ private static void rollbackAllTxns(boolean val, IDriver driver) { - driver.getConf().setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, val); + driver.getConf().setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, val); } private boolean isTez(HiveConf conf){ diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java index e8ebf251297d..08626809fb8b 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java @@ -92,7 +92,7 @@ public static void preTests() throws Exception { HiveConf hiveConf = UtilsForTest.getHiveOnTezConfFromDir("../../data/conf/tez/"); hiveConf.setVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, false); hiveConf.set(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose"); miniHS2 = new MiniHS2(hiveConf, MiniClusterType.TEZ); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java index 31c29baccf02..2e3a3c945a80 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java @@ -54,7 +54,7 @@ public static void preTests() throws Exception { hiveConf.setVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager"); hiveConf.setIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_DEFAULT_FETCH_SIZE, 10); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, false); hiveConf.set(HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose"); miniHS2 = new MiniHS2(hiveConf, MiniHS2.MiniClusterType.TEZ); Map confOverlay = new HashMap<>(); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java index d6f285498a1e..821f504bac22 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java @@ -205,12 +205,12 @@ public static void setUpBeforeClass() throws Exception { // Create test database and base tables once for all the test Class.forName(driverName); System.setProperty(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose"); - System.setProperty(ConfVars.HIVEMAPREDMODE.varname, "nonstrict"); + System.setProperty(ConfVars.HIVE_MAPRED_MODE.varname, "nonstrict"); System.setProperty(ConfVars.HIVE_AUTHORIZATION_MANAGER.varname, "org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"); System.setProperty(ConfVars.HIVE_SERVER2_PARALLEL_OPS_IN_SESSION.varname, "false"); - System.setProperty(ConfVars.REPLCMENABLED.varname, "true"); - System.setProperty(ConfVars.REPLCMDIR.varname, "cmroot"); + System.setProperty(ConfVars.REPL_CM_ENABLED.varname, "true"); + System.setProperty(ConfVars.REPL_CM_DIR.varname, "cmroot"); con = getConnection(defaultDbName + ";create=true"); Statement stmt = con.createStatement(); assertNotNull("Statement is null", stmt); @@ -2100,7 +2100,7 @@ public void testSetCommand() throws SQLException { String rline = res.getString(1); assertFalse( "set output must not contain hidden variables such as the metastore password:" + rline, - rline.contains(HiveConf.ConfVars.METASTOREPWD.varname) + rline.contains(HiveConf.ConfVars.METASTORE_PWD.varname) && !(rline.contains(HiveConf.ConfVars.HIVE_CONF_HIDDEN_LIST.varname))); // the only conf allowed to have the metastore pwd keyname is the hidden list configuration // value @@ -2347,7 +2347,7 @@ public void testFetchFirstSetCmds() throws Exception { */ @Test public void testFetchFirstDfsCmds() throws Exception { - String wareHouseDir = conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname); + String wareHouseDir = conf.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname); execFetchFirst("dfs -ls " + wareHouseDir, DfsProcessor.DFS_RESULT_HEADER, false); } diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java index 2436f5f9d8f1..fcf666da13df 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java @@ -67,7 +67,6 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.ObjectStore; import org.apache.hadoop.hive.metastore.PersistenceManagerProvider; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; @@ -85,7 +84,6 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import static org.apache.hadoop.hive.common.repl.ReplConst.SOURCE_OF_REPLICATION; @@ -219,7 +217,7 @@ private static void startMiniHS2(HiveConf conf, boolean httpMode) throws Excepti conf.setBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED, false); conf.setBoolVar(ConfVars.HIVESTATSCOLAUTOGATHER, false); // store post-exec hooks calls so we can look at them later - conf.setVar(ConfVars.POSTEXECHOOKS, ReadableHook.class.getName() + "," + + conf.setVar(ConfVars.POST_EXEC_HOOKS, ReadableHook.class.getName() + "," + LineageLogger.class.getName()); MiniHS2.Builder builder = new MiniHS2.Builder().withConf(conf).cleanupLocalDirOnStartup(false); if (httpMode) { @@ -807,15 +805,15 @@ public void testSessionScratchDirs() throws Exception { // FS FileSystem fs = miniHS2.getLocalFS(); FsPermission expectedFSPermission = new FsPermission(HiveConf.getVar(conf, - HiveConf.ConfVars.SCRATCHDIRPERMISSION)); + HiveConf.ConfVars.SCRATCH_DIR_PERMISSION)); // Verify scratch dir paths and permission // HDFS scratch dir - scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR) + "/" + userName); + scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR) + "/" + userName); verifyScratchDir(conf, fs, scratchDirPath, expectedFSPermission, userName, false); // Local scratch dir - scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR)); + scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); verifyScratchDir(conf, fs, scratchDirPath, expectedFSPermission, userName, true); // Downloaded resources dir @@ -831,11 +829,11 @@ public void testSessionScratchDirs() throws Exception { // Verify scratch dir paths and permission // HDFS scratch dir - scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR) + "/" + userName); + scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR) + "/" + userName); verifyScratchDir(conf, fs, scratchDirPath, expectedFSPermission, userName, false); // Local scratch dir - scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR)); + scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); verifyScratchDir(conf, fs, scratchDirPath, expectedFSPermission, userName, true); // Downloaded resources dir @@ -982,7 +980,7 @@ public void testRootScratchDir() throws Exception { FsPermission expectedFSPermission = new FsPermission((short)00733); // Verify scratch dir paths and permission // HDFS scratch dir - scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR)); + scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR)); verifyScratchDir(conf, fs, scratchDirPath, expectedFSPermission, userName, false); conn.close(); @@ -992,7 +990,7 @@ public void testRootScratchDir() throws Exception { conf.set("hive.exec.scratchdir", tmpDir + "/level1/level2/level3"); startMiniHS2(conf); conn = getConnection(miniHS2.getJdbcURL(testDbName), userName, "password"); - scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR)); + scratchDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR)); verifyScratchDir(conf, fs, scratchDirPath, expectedFSPermission, userName, false); conn.close(); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/cbo_rp_TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/cbo_rp_TestJdbcDriver2.java index b43a1b7586de..1942dc194f13 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/cbo_rp_TestJdbcDriver2.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/cbo_rp_TestJdbcDriver2.java @@ -58,7 +58,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.ql.exec.UDF; -import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer; import org.apache.hadoop.hive.ql.processors.DfsProcessor; import org.apache.hive.common.util.HiveVersionInfo; import org.apache.hive.jdbc.Utils.JdbcConnectionParams; @@ -2059,7 +2058,7 @@ public void testFetchFirstSetCmds() throws Exception { */ @Test public void testFetchFirstDfsCmds() throws Exception { - String wareHouseDir = conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname); + String wareHouseDir = conf.get(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname); execFetchFirst("dfs -ls " + wareHouseDir, DfsProcessor.DFS_RESULT_HEADER, false); } diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/TestDFSErrorHandling.java b/itests/hive-unit/src/test/java/org/apache/hive/service/TestDFSErrorHandling.java index f088bc651c9f..6321e049f5cb 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/TestDFSErrorHandling.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/TestDFSErrorHandling.java @@ -29,10 +29,8 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim; import org.apache.hive.jdbc.miniHS2.MiniHS2; -import org.apache.hive.service.cli.HiveSQLException; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -98,7 +96,7 @@ public void testAccessDenied() throws Exception { miniHS2 != null && miniHS2.isStarted()); Class.forName(MiniHS2.getJdbcDriverName()); - Path scratchDir = new Path(HiveConf.getVar(hiveConf, HiveConf.ConfVars.SCRATCHDIR)); + Path scratchDir = new Path(HiveConf.getVar(hiveConf, HiveConf.ConfVars.SCRATCH_DIR)); MiniDFSShim dfs = miniHS2.getDfs(); FileSystem fs = dfs.getFileSystem(); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java index 819838d09120..850fa243d9da 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java @@ -39,7 +39,7 @@ public static void setUpBeforeClass() throws Exception { service = new EmbeddedThriftBinaryCLIService(); HiveConf conf = new HiveConf(); conf.setBoolean("datanucleus.schema.autoCreateTables", true); - conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + conf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); UtilsForTest.expandHiveConfParams(conf); service.init(conf); client = new ThriftCLIServiceClient(service); diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2ClearDanglingScratchDir.java b/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2ClearDanglingScratchDir.java index 1adfdebdf2be..ff36e24b2d13 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2ClearDanglingScratchDir.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2ClearDanglingScratchDir.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.util.Shell; import org.junit.Assert; import org.junit.Test; @@ -40,13 +39,13 @@ public void testScratchDirCleared() throws Exception { conf.set(HiveConf.ConfVars.HIVE_SCRATCH_DIR_LOCK.toString(), "true"); conf.set(HiveConf.ConfVars.HIVE_SERVER2_CLEAR_DANGLING_SCRATCH_DIR.toString(), "true"); - Path scratchDir = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR)); + Path scratchDir = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR)); m_dfs.getFileSystem().mkdirs(scratchDir); m_dfs.getFileSystem().setPermission(scratchDir, new FsPermission("777")); // Fake two live session SessionState.start(conf); - conf.setVar(HiveConf.ConfVars.HIVESESSIONID, UUID.randomUUID().toString()); + conf.setVar(HiveConf.ConfVars.HIVE_SESSION_ID, UUID.randomUUID().toString()); SessionState.start(conf); // Fake dead session diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2SessionHive.java b/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2SessionHive.java index 15cfb190d5e0..79ddd77556cd 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2SessionHive.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/service/server/TestHS2SessionHive.java @@ -186,7 +186,7 @@ public static void setupBeforeClass() throws Exception { throw e; } - miniHS2.getHiveConf().setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + miniHS2.getHmsPort()); + miniHS2.getHiveConf().setVar(HiveConf.ConfVars.METASTORE_URIS, "thrift://localhost:" + miniHS2.getHmsPort()); try (Connection conn = DriverManager. getConnection(miniHS2.getJdbcURL(), System.getProperty("user.name"), ""); diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMetaStoreHandler.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMetaStoreHandler.java index 184e56f158c1..e8827bda9007 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMetaStoreHandler.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMetaStoreHandler.java @@ -60,7 +60,7 @@ public boolean isDerby() { } public QTestMetaStoreHandler setMetaStoreConfiguration(HiveConf conf) { - conf.setVar(ConfVars.METASTOREDBTYPE, getDbTypeConfString()); + conf.setVar(ConfVars.METASTORE_DB_TYPE, getDbTypeConfString()); MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY, rule.getJdbcUrl()); MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECTION_DRIVER, rule.getJdbcDriver()); diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java index 38530f047e3f..e94a842d7636 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestMiniClusters.java @@ -420,7 +420,7 @@ private void createRemoteDirs(HiveConf conf) { // Create remote dirs once. if (getMr() != null) { assert fs != null; - Path warehousePath = fs.makeQualified(new Path(conf.getVar(ConfVars.METASTOREWAREHOUSE))); + Path warehousePath = fs.makeQualified(new Path(conf.getVar(ConfVars.METASTORE_WAREHOUSE))); assert warehousePath != null; Path hiveJarPath = fs.makeQualified(new Path(conf.getVar(ConfVars.HIVE_JAR_DIRECTORY))); assert hiveJarPath != null; @@ -570,13 +570,13 @@ private void setFsRelatedProperties(HiveConf conf, boolean isLocalFs, FileSystem conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsUriString); // Remote dirs - conf.setVar(ConfVars.METASTOREWAREHOUSE, warehousePath.toString()); + conf.setVar(ConfVars.METASTORE_WAREHOUSE, warehousePath.toString()); conf.setVar(ConfVars.HIVE_JAR_DIRECTORY, jarPath.toString()); conf.setVar(ConfVars.HIVE_USER_INSTALL_DIR, userInstallPath.toString()); - // ConfVars.SCRATCHDIR - {test.tmp.dir}/scratchdir + // ConfVars.SCRATCH_DIR - {test.tmp.dir}/scratchdir // Local dirs - // ConfVars.LOCALSCRATCHDIR - {test.tmp.dir}/localscratchdir + // ConfVars.LOCAL_SCRATCH_DIR - {test.tmp.dir}/localscratchdir // TODO Make sure to cleanup created dirs. } diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java index e9c86372bc98..180c6e70d6b6 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java @@ -22,7 +22,6 @@ import java.io.BufferedOutputStream; import java.io.File; -import java.util.LinkedHashSet; import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.FileWriter; @@ -570,7 +569,7 @@ public void postInit() throws Exception { sem = new SemanticAnalyzer(new QueryState.Builder().withHiveConf(conf).build()); - testWarehouse = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE); + testWarehouse = conf.getVar(HiveConf.ConfVars.METASTORE_WAREHOUSE); db = Hive.get(conf); pd = new ParseDriver(); @@ -750,7 +749,7 @@ private CommandProcessorResponse executeClientInternal(String commands) throws C * if you want to use another hive cmd after the failure to sanity check the state of the system. */ private boolean ignoreErrors() { - return conf.getBoolVar(HiveConf.ConfVars.CLIIGNOREERRORS); + return conf.getBoolVar(HiveConf.ConfVars.CLI_IGNORE_ERRORS); } boolean isHiveCommand(String command) { @@ -775,7 +774,7 @@ private CommandProcessorResponse executeTestCommand(String command) throws Comma //replace ${hiveconf:hive.metastore.warehouse.dir} with actual dir if existed. //we only want the absolute path, so remove the header, such as hdfs://localhost:57145 String wareHouseDir = - SessionState.get().getConf().getVar(ConfVars.METASTOREWAREHOUSE).replaceAll("^[a-zA-Z]+://.*?:\\d+", ""); + SessionState.get().getConf().getVar(ConfVars.METASTORE_WAREHOUSE).replaceAll("^[a-zA-Z]+://.*?:\\d+", ""); commandArgs = commandArgs.replaceAll("\\$\\{hiveconf:hive\\.metastore\\.warehouse\\.dir\\}", wareHouseDir); if (SessionState.get() != null) { diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java index 43cb8c9615cf..6e5262213444 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/MapJoinCounterHook.java @@ -30,7 +30,7 @@ public class MapJoinCounterHook implements ExecuteWithHookContext { public void run(HookContext hookContext) { HiveConf conf = hookContext.getConf(); - boolean enableConvert = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVECONVERTJOIN); + boolean enableConvert = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CONVERT_JOIN); if (!enableConvert) { return; } diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java index a4ac92b62570..e705c12f8db7 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java @@ -34,7 +34,7 @@ public class DummyStatsAggregator implements StatsAggregator { // denotes the method which needs to throw an error. @Override public boolean connect(StatsCollectionContext scc) { - errorMethod = HiveConf.getVar(scc.getHiveConf(), HiveConf.ConfVars.HIVETESTMODEDUMMYSTATAGGR); + errorMethod = HiveConf.getVar(scc.getHiveConf(), HiveConf.ConfVars.HIVE_TEST_MODE_DUMMY_STAT_AGGR); if (errorMethod.equalsIgnoreCase("connect")) { return false; } diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java index cc80fc2b3e6a..41e475fd7a17 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java @@ -37,7 +37,7 @@ public class DummyStatsPublisher implements StatsPublisher { // denotes the method which needs to throw an error. @Override public boolean init(StatsCollectionContext context) { - errorMethod = HiveConf.getVar(context.getHiveConf(), HiveConf.ConfVars.HIVETESTMODEDUMMYSTATPUB); + errorMethod = HiveConf.getVar(context.getHiveConf(), HiveConf.ConfVars.HIVE_TEST_MODE_DUMMY_STAT_PUB); if (errorMethod.equalsIgnoreCase("init")) { return false; } @@ -47,7 +47,7 @@ public boolean init(StatsCollectionContext context) { @Override public boolean connect(StatsCollectionContext context) { - errorMethod = HiveConf.getVar(context.getHiveConf(), HiveConf.ConfVars.HIVETESTMODEDUMMYSTATPUB); + errorMethod = HiveConf.getVar(context.getHiveConf(), HiveConf.ConfVars.HIVE_TEST_MODE_DUMMY_STAT_PUB); if (errorMethod.equalsIgnoreCase("connect")) { return false; } diff --git a/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java index d21b76418607..069d58c6a0b0 100644 --- a/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java +++ b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/AbstractHiveService.java @@ -90,13 +90,13 @@ public void clearProperties() { * @return */ public Path getWareHouseDir() { - return new Path(hiveConf.getVar(ConfVars.METASTOREWAREHOUSE)); + return new Path(hiveConf.getVar(ConfVars.METASTORE_WAREHOUSE)); } public void setWareHouseDir(String wareHouseURI) { verifyNotStarted(); - System.setProperty(ConfVars.METASTOREWAREHOUSE.varname, wareHouseURI); - hiveConf.setVar(ConfVars.METASTOREWAREHOUSE, wareHouseURI); + System.setProperty(ConfVars.METASTORE_WAREHOUSE.varname, wareHouseURI); + hiveConf.setVar(ConfVars.METASTORE_WAREHOUSE, wareHouseURI); } /** diff --git a/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java index 9e95d3b2db92..8076a0718ed7 100644 --- a/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java +++ b/itests/util/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java @@ -348,12 +348,12 @@ private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useM Path scratchDir = new Path(baseFsDir, "scratch"); // Create root scratchdir with write all, so that user impersonation has no issues. Utilities.createDirsWithPermission(hiveConf, scratchDir, WRITE_ALL_PERM, true); - System.setProperty(HiveConf.ConfVars.SCRATCHDIR.varname, scratchDir.toString()); - hiveConf.setVar(ConfVars.SCRATCHDIR, scratchDir.toString()); + System.setProperty(HiveConf.ConfVars.SCRATCH_DIR.varname, scratchDir.toString()); + hiveConf.setVar(ConfVars.SCRATCH_DIR, scratchDir.toString()); String localScratchDir = baseDir.getPath() + File.separator + "scratch"; - System.setProperty(HiveConf.ConfVars.LOCALSCRATCHDIR.varname, localScratchDir); - hiveConf.setVar(ConfVars.LOCALSCRATCHDIR, localScratchDir); + System.setProperty(HiveConf.ConfVars.LOCAL_SCRATCH_DIR.varname, localScratchDir); + hiveConf.setVar(ConfVars.LOCAL_SCRATCH_DIR, localScratchDir); } public MiniHS2(HiveConf hiveConf) throws Exception { diff --git a/jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java b/jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java index f7f3dd65ddf4..94cd398a9d41 100644 --- a/jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java +++ b/jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java @@ -192,11 +192,11 @@ private static void resolveMetadata(Properties props) throws HiveException, IOEx } private static String getMetastoreDatabaseType(HiveConf conf) { - return conf.getVar(HiveConf.ConfVars.METASTOREDBTYPE); + return conf.getVar(HiveConf.ConfVars.METASTORE_DB_TYPE); } private static String getMetastoreConnectionURL(HiveConf conf) { - return conf.getVar(HiveConf.ConfVars.METASTORECONNECTURLKEY); + return conf.getVar(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY); } private static String getMetastoreDriver(HiveConf conf) { @@ -209,6 +209,6 @@ private static String getMetastoreJdbcUser(HiveConf conf) { private static String getMetastoreJdbcPasswd(HiveConf conf) throws IOException { return ShimLoader.getHadoopShims().getPassword(conf, - HiveConf.ConfVars.METASTOREPWD.varname); + HiveConf.ConfVars.METASTORE_PWD.varname); } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java index 1ddda8e6992c..19048c1da426 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java @@ -67,7 +67,7 @@ public class KafkaOutputFormat implements HiveOutputFormat(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java index 93b153886c99..89bff9678f1c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java @@ -385,7 +385,7 @@ private ValidTxnWriteIdList getTxnWriteIds(String txnString) throws LockExceptio private void setValidWriteIds(ValidTxnWriteIdList txnWriteIds) { driverContext.getConf().set(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY, txnWriteIds.toString()); if (driverContext.getPlan().getFetchTask() != null) { - // This is needed for {@link HiveConf.ConfVars.HIVEFETCHTASKCONVERSION} optimization which initializes JobConf + // This is needed for {@link HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION} optimization which initializes JobConf // in FetchOperator before recordValidTxns() but this has to be done after locks are acquired to avoid race // conditions in ACID. This case is supported only for single source query. Operator source = driverContext.getPlan().getFetchTask().getWork().getSource(); @@ -602,7 +602,7 @@ synchronized void endTransactionAndCleanup(boolean commit, HiveTxnManager txnMan private void commitOrRollback(boolean commit, HiveTxnManager txnManager) throws LockException { if (commit) { if (driverContext.getConf().getBoolVar(ConfVars.HIVE_IN_TEST) && - driverContext.getConf().getBoolVar(ConfVars.HIVETESTMODEROLLBACKTXN)) { + driverContext.getConf().getBoolVar(ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN)) { txnManager.rollbackTxn(); } else { txnManager.commitTxn(); //both commit & rollback clear ALL locks for this transaction diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Executor.java b/ql/src/java/org/apache/hadoop/hive/ql/Executor.java index 1e2140ed8025..708e3870efa4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Executor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Executor.java @@ -297,7 +297,7 @@ private void handleFinished() throws Exception { } private String getJobName() { - int maxlen = driverContext.getConf().getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); + int maxlen = driverContext.getConf().getIntVar(HiveConf.ConfVars.HIVE_JOBNAME_LENGTH); return Utilities.abbreviate(driverContext.getQueryString(), maxlen - 6); } @@ -322,7 +322,7 @@ private int getJobCount() { private void launchTasks(boolean noName, int jobCount, String jobName) throws HiveException { // Launch upto maxthreads tasks Task task; - int maxthreads = HiveConf.getIntVar(driverContext.getConf(), HiveConf.ConfVars.EXECPARALLETHREADNUMBER); + int maxthreads = HiveConf.getIntVar(driverContext.getConf(), HiveConf.ConfVars.EXEC_PARALLEL_THREAD_NUMBER); while ((task = taskQueue.getRunnable(maxthreads)) != null) { TaskRunner runner = launchTask(task, noName, jobName, jobCount); if (!runner.isRunning()) { @@ -346,7 +346,7 @@ private TaskRunner launchTask(Task task, boolean noName, String jobName, int TaskRunner taskRun = new TaskRunner(task, taskQueue); taskQueue.launching(taskRun); - if (HiveConf.getBoolVar(task.getConf(), HiveConf.ConfVars.EXECPARALLEL) && task.canExecuteInParallel()) { + if (HiveConf.getBoolVar(task.getConf(), HiveConf.ConfVars.EXEC_PARALLEL) && task.canExecuteInParallel()) { LOG.info("Starting task [" + task + "] in parallel"); taskRun.start(); } else { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java index b39037cd65e1..10025dbd9026 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java @@ -98,7 +98,7 @@ private QueryState(HiveConf conf) { // Get the query id stored in query specific config. public String getQueryId() { - return queryConf.getVar(HiveConf.ConfVars.HIVEQUERYID); + return queryConf.getVar(HiveConf.ConfVars.HIVE_QUERY_ID); } public String getQueryString() { @@ -172,15 +172,15 @@ public void setNumModifiedRows(long numModifiedRows) { } public String getQueryTag() { - return HiveConf.getVar(this.queryConf, HiveConf.ConfVars.HIVEQUERYTAG); + return HiveConf.getVar(this.queryConf, HiveConf.ConfVars.HIVE_QUERY_TAG); } public void setQueryTag(String queryTag) { - HiveConf.setVar(this.queryConf, HiveConf.ConfVars.HIVEQUERYTAG, queryTag); + HiveConf.setVar(this.queryConf, HiveConf.ConfVars.HIVE_QUERY_TAG, queryTag); } public static void setApplicationTag(HiveConf queryConf, String queryTag) { - String jobTag = HiveConf.getVar(queryConf, HiveConf.ConfVars.HIVEQUERYTAG); + String jobTag = HiveConf.getVar(queryConf, HiveConf.ConfVars.HIVE_QUERY_TAG); if (jobTag == null || jobTag.isEmpty()) { jobTag = queryTag; } else { @@ -327,13 +327,13 @@ public QueryState build() { // Generate the new queryId if needed if (generateNewQueryId) { String queryId = QueryPlan.makeQueryId(); - queryConf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId); + queryConf.setVar(HiveConf.ConfVars.HIVE_QUERY_ID, queryId); setApplicationTag(queryConf, queryId); // FIXME: druid storage handler relies on query.id to maintain some staging directories // expose queryid to session level if (hiveConf != null) { - hiveConf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId); + hiveConf.setVar(HiveConf.ConfVars.HIVE_QUERY_ID, queryId); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseAnalyzer.java index fda2282cbfa5..cd0392dd066c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/lock/LockDatabaseAnalyzer.java @@ -52,7 +52,7 @@ public void analyzeInternal(ASTNode root) throws SemanticException { outputs.add(new WriteEntity(getDatabase(databaseName), WriteType.DDL_NO_LOCK)); LockDatabaseDesc desc = - new LockDatabaseDesc(databaseName, mode, HiveConf.getVar(conf, ConfVars.HIVEQUERYID), ctx.getCmd()); + new LockDatabaseDesc(databaseName, mode, HiveConf.getVar(conf, ConfVars.HIVE_QUERY_ID), ctx.getCmd()); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc))); ctx.setNeedLockMgr(true); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractBaseAlterTableAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractBaseAlterTableAnalyzer.java index 17f9fec4d177..9e7688a5b29a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractBaseAlterTableAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractBaseAlterTableAnalyzer.java @@ -105,7 +105,7 @@ protected void addInputsOutputsAlterTable(TableName tableName, Map partitionSpec) throws SemanticException { Set reservedPartitionValues = new HashSet<>(); // Partition can't have this name - reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.DEFAULTPARTITIONNAME)); + reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.DEFAULT_PARTITION_NAME)); reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.DEFAULT_ZOOKEEPER_PARTITION_NAME)); // Partition value can't end in this suffix reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.METASTORE_INT_ORIGINAL)); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java index 0fd8785d1bc7..24deedf63b0c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java @@ -97,7 +97,7 @@ private void checkPartitionValues(Table tbl, int colIndex) throws HiveException try { List values = Warehouse.getPartValuesFromPartName(partName); String value = values.get(colIndex); - if (value.equals(context.getConf().getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME))) { + if (value.equals(context.getConf().getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME))) { continue; } Object convertedValue = converter.convert(value); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionAnalyzer.java index 40500f13ff4b..c0bffcebdb23 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionAnalyzer.java @@ -119,7 +119,7 @@ ExprNodeDesc getShowPartitionsFilter(Table table, ASTNode command) throws Semant } showFilter = replaceDefaultPartNameAndCastType(target, colTypes, - HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME)); + HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULT_PARTITION_NAME)); } } return showFilter; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AbstractAlterTableArchiveAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AbstractAlterTableArchiveAnalyzer.java index 4b793eb545f0..b419c8332354 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AbstractAlterTableArchiveAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AbstractAlterTableArchiveAnalyzer.java @@ -51,7 +51,7 @@ public AbstractAlterTableArchiveAnalyzer(QueryState queryState) throws SemanticE // the AST tree protected void analyzeCommand(TableName tableName, Map partSpec, ASTNode command) throws SemanticException { - if (!conf.getBoolVar(HiveConf.ConfVars.HIVEARCHIVEENABLED)) { + if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_ARCHIVE_ENABLED)) { throw new SemanticException(ErrorMsg.ARCHIVE_METHODS_DISABLED.getMsg()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AlterTableArchiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AlterTableArchiveOperation.java index f54ea6a6a727..e218e590a24e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AlterTableArchiveOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/archive/AlterTableArchiveOperation.java @@ -182,7 +182,7 @@ private Path createArchiveInTmpDir(Table table, PartSpecInfo partitionSpecInfo, context.getConsole().printInfo("Creating " + ARCHIVE_NAME + " for " + originalDir.toString() + " in " + tmpPath); context.getConsole().printInfo("Please wait... (this may take a while)"); try { - int maxJobNameLength = context.getConf().getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); + int maxJobNameLength = context.getConf().getIntVar(HiveConf.ConfVars.HIVE_JOBNAME_LENGTH); String jobName = String.format("Archiving %s@%s", table.getTableName(), partitionSpecInfo.getName()); jobName = Utilities.abbreviate(jobName, maxJobNameLength - 6); context.getConf().set(MRJobConfig.JOB_NAME, jobName); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java index f9db0175edae..c661d2de4f88 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/BinaryRecordReader.java @@ -40,7 +40,7 @@ public class BinaryRecordReader implements RecordReader { public void initialize(InputStream in, Configuration conf, Properties tbl) throws IOException { this.in = in; - maxRecordLength = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVEBINARYRECORDMAX); + maxRecordLength = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_BINARY_RECORD_MAX); } public Writable createRow() throws IOException { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java index e36aaa86cf78..b0b860e16d3f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java @@ -239,7 +239,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { this.hconf = hconf; heartbeatInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVESENDHEARTBEAT); + HiveConf.ConfVars.HIVE_SEND_HEARTBEAT); countAfterReport = 0; totalSz = 0; @@ -296,9 +296,9 @@ protected void initializeOp(Configuration hconf) throws HiveException { dummyObjVectors = new RowContainer[numAliases]; joinEmitInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVEJOINEMITINTERVAL); + HiveConf.ConfVars.HIVE_JOIN_EMIT_INTERVAL); joinCacheSize = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVEJOINCACHESIZE); + HiveConf.ConfVars.HIVE_JOIN_CACHE_SIZE); logEveryNRows = HiveConf.getLongVar(hconf, HiveConf.ConfVars.HIVE_LOG_N_RECORDS); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java index 0044a042bd7b..c677796de73a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java @@ -136,13 +136,13 @@ public void initializeOp(Configuration hconf) throws HiveException { int bucketSize; - int oldVar = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEMAPJOINBUCKETCACHESIZE); + int oldVar = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_MAPJOIN_BUCKET_CACHE_SIZE); shortcutUnmatchedRows = HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVE_JOIN_SHORTCUT_UNMATCHED_ROWS); if (oldVar != 100) { bucketSize = oldVar; } else { - bucketSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVESMBJOINCACHEROWS); + bucketSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_SMBJOIN_CACHE_ROWS); } for (byte pos = 0; pos < order.length; pos++) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index 0cd931391a3c..ea6dff1becfc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -275,7 +275,7 @@ public JSONObject getJSONPlan(PrintStream out, List> tasks, Task fetc return getJSONPlan( out, tasks, fetchTask, jsonOutput, isExtended, appendTaskType, cboInfo, cboPlan, optimizedSQL, - conf.getVar(ConfVars.HIVESTAGEIDREARRANGE)); + conf.getVar(ConfVars.HIVE_STAGE_ID_REARRANGE)); } public JSONObject getJSONPlan(PrintStream out, List> tasks, Task fetchTask, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index f47cbffca663..d7903747ae9c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -655,7 +655,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { outputClass = serializer.getSerializedClass(); destTablePath = conf.getDestPath(); isInsertOverwrite = conf.getInsertOverwrite(); - counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP); + counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); LOG.info("Using serializer : " + serializer + " and formatter : " + hiveOutputFormat + (isCompressed ? " with compression" : "")); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java index a3af21a39a99..2fb17e068af5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java @@ -60,9 +60,9 @@ protected void initializeOp(Configuration hconf) throws HiveException { super.initializeOp(hconf); try { heartbeatInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVESENDHEARTBEAT); + HiveConf.ConfVars.HIVE_SEND_HEARTBEAT); conditionEvaluator = ExprNodeEvaluatorFactory.get(conf.getPredicate(), hconf); - if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVEEXPREVALUATIONCACHE)) { + if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVE_EXPR_EVALUATION_CACHE)) { conditionEvaluator = ExprNodeEvaluatorFactory.toCachedEval(conditionEvaluator); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java index f548afd52401..326c351c7382 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java @@ -206,7 +206,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { numRowsHashTbl = 0; heartbeatInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVESENDHEARTBEAT); + HiveConf.ConfVars.HIVE_SEND_HEARTBEAT); countAfterReport = 0; ObjectInspector rowInspector = inputObjInspectors[0]; @@ -367,7 +367,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { keyPositionsSize = new ArrayList(); aggrPositions = new List[aggregations.length]; groupbyMapAggrInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL); + HiveConf.ConfVars.HIVE_GROUPBY_MAP_INTERVAL); // compare every groupbyMapAggrInterval rows numRowsCompareHashAggr = groupbyMapAggrInterval; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java index ce7279c78ab6..fff5446daef8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java @@ -122,7 +122,7 @@ public HashTableSinkOperator(CompilationOpContext ctx, MapJoinOperator mjop) { @SuppressWarnings("unchecked") protected void initializeOp(Configuration hconf) throws HiveException { super.initializeOp(hconf); - boolean isSilent = HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVESESSIONSILENT); + boolean isSilent = HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVE_SESSION_SILENT); console = new LogHelper(LOG, isSilent); memoryExhaustionChecker = MemoryExhaustionCheckerFactory.getChecker(console, hconf, conf); emptyRowContainer.addRow(emptyObjectArray); @@ -172,7 +172,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { } mapJoinTables = new MapJoinPersistableTableContainer[tagLen]; mapJoinTableSerdes = new MapJoinTableContainerSerDe[tagLen]; - hashTableScale = HiveConf.getLongVar(hconf, HiveConf.ConfVars.HIVEHASHTABLESCALE); + hashTableScale = HiveConf.getLongVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_SCALE); if (hashTableScale <= 0) { hashTableScale = 1; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java index 41bee8d60f37..e5b60e7781ff 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java @@ -68,7 +68,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { currCount = 0; isMap = hconf.getBoolean("mapred.task.is.map", true); - String queryId = HiveConf.getVar(getConfiguration(), HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(getConfiguration(), HiveConf.ConfVars.HIVE_QUERY_ID); this.runtimeCache = ObjectCacheFactory.getCache(getConfiguration(), queryId, false, true); // this can happen in HS2 while doing local fetch optimization, where LimitOperator is used @@ -168,14 +168,14 @@ public static String getLimitReachedKey(Configuration conf) { } public static boolean checkLimitReached(JobConf jobConf) { - String queryId = HiveConf.getVar(jobConf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(jobConf, HiveConf.ConfVars.HIVE_QUERY_ID); String limitReachedKey = getLimitReachedKey(jobConf); return checkLimitReached(jobConf, queryId, limitReachedKey); } public static boolean checkLimitReachedForVertex(JobConf jobConf, String vertexName) { - String queryId = HiveConf.getVar(jobConf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(jobConf, HiveConf.ConfVars.HIVE_QUERY_ID); return checkLimitReached(jobConf, queryId, vertexName + LIMIT_REACHED_KEY_SUFFIX); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java index 02352c400842..57b1786483ca 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java @@ -177,7 +177,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { // On Tez only: The hash map might already be cached in the container we run // the task in. On MR: The cache is a no-op. - String queryId = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_QUERY_ID); // The cacheKey may have already been defined in the MapJoin conf spec // as part of the Shared Work Optimization if it can be reused among // multiple mapjoin operators. In that case, we take that key from conf diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index 9bc6aa1b41f7..2721977d6f9b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.ddl.DDLUtils; @@ -66,7 +65,6 @@ import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol; import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol; import org.apache.hadoop.hive.ql.parse.ExplainConfiguration.AnalyzeState; -import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; import org.apache.hadoop.hive.ql.plan.LoadFileDesc; import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc; @@ -834,7 +832,7 @@ private void checkFileFormats(Hive db, LoadTableDesc tbd, Table table) } // handle file format check for table level - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVECHECKFILEFORMAT)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CHECK_FILEFORMAT)) { boolean flag = true; // work.checkFileFormat is set to true only for Load Task, so assumption here is // dynamic partition context is null diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java index c52ca1877363..5b6248108257 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFPartition.java @@ -75,7 +75,7 @@ protected PTFPartition(Configuration cfg, this.inputOI = inputOI; this.outputOI = outputOI; if ( createElemContainer ) { - int containerNumRows = HiveConf.getIntVar(cfg, ConfVars.HIVEJOINCACHESIZE); + int containerNumRows = HiveConf.getIntVar(cfg, ConfVars.HIVE_JOIN_CACHE_SIZE); elems = new PTFRowContainer>(containerNumRows, cfg, null); elems.setSerDe(serDe, outputOI); elems.setTableDesc(PTFRowContainer.createTableDesc(inputOI)); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java index 24bec956a69b..bad296e38505 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/PartitionKeySampler.java @@ -135,9 +135,9 @@ public void writePartitionKeys(Path path, JobConf job) throws IOException { // random sampling public static FetchOperator createSampler(FetchWork work, JobConf job, Operator operator) throws HiveException { - int sampleNum = HiveConf.getIntVar(job, HiveConf.ConfVars.HIVESAMPLINGNUMBERFORORDERBY); + int sampleNum = HiveConf.getIntVar(job, HiveConf.ConfVars.HIVE_SAMPLING_NUMBER_FOR_ORDERBY); float samplePercent = - HiveConf.getFloatVar(job, HiveConf.ConfVars.HIVESAMPLINGPERCENTFORORDERBY); + HiveConf.getFloatVar(job, HiveConf.ConfVars.HIVE_SAMPLING_PERCENT_FOR_ORDERBY); if (samplePercent < 0.0 || samplePercent > 1.0) { throw new IllegalArgumentException("Percentile value must be within the range of 0 to 1."); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java index b3f167c3e98d..d119e688b710 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java @@ -132,13 +132,13 @@ protected void initializeOp(Configuration hconf) throws HiveException { int bucketSize; // For backwards compatibility reasons we honor the older - // HIVEMAPJOINBUCKETCACHESIZE if set different from default. + // HIVE_MAPJOIN_BUCKET_CACHE_SIZE if set different from default. // By hive 0.13 we should remove this code. - int oldVar = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEMAPJOINBUCKETCACHESIZE); + int oldVar = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_MAPJOIN_BUCKET_CACHE_SIZE); if (oldVar != 100) { bucketSize = oldVar; } else { - bucketSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVESMBJOINCACHEROWS); + bucketSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_SMBJOIN_CACHE_ROWS); } for (byte pos = 0; pos < order.length; pos++) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java index af22e5e95e88..2dd3e0f59522 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java @@ -149,8 +149,8 @@ boolean blackListed(Configuration conf, String name) { if (blackListedConfEntries == null) { blackListedConfEntries = new HashSet(); if (conf != null) { - String bl = conf.get(HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST.toString(), - HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST.getDefaultValue()); + String bl = conf.get(HiveConf.ConfVars.HIVE_SCRIPT_ENV_BLACKLIST.toString(), + HiveConf.ConfVars.HIVE_SCRIPT_ENV_BLACKLIST.getDefaultValue()); if (bl != null && !bl.isEmpty()) { String[] bls = bl.split(","); Collections.addAll(blackListedConfEntries, bls); @@ -175,7 +175,7 @@ void addJobConfToEnvironment(Configuration conf, Map env) { String value = conf.get(name); // does variable expansion name = safeEnvVarName(name); boolean truncate = conf - .getBoolean(HiveConf.ConfVars.HIVESCRIPTTRUNCATEENV.toString(), false); + .getBoolean(HiveConf.ConfVars.HIVE_SCRIPT_TRUNCATE_ENV.toString(), false); value = safeEnvVarValue(value, name, truncate); env.put(name, value); } @@ -290,12 +290,12 @@ boolean isBrokenPipeException(IOException e) { } boolean allowPartialConsumption() { - return HiveConf.getBoolVar(hconf, HiveConf.ConfVars.ALLOWPARTIALCONSUMP); + return HiveConf.getBoolVar(hconf, HiveConf.ConfVars.ALLOW_PARTIAL_CONSUMP); } void displayBrokenPipeInfo() { LOG.info("The script did not consume all input data. This is considered as an error."); - LOG.info("set " + HiveConf.ConfVars.ALLOWPARTIALCONSUMP.toString() + "=true; to ignore it."); + LOG.info("set " + HiveConf.ConfVars.ALLOW_PARTIAL_CONSUMP.toString() + "=true; to ignore it."); return; } @@ -339,13 +339,13 @@ public void process(Object row, int tag) throws HiveException { ProcessBuilder pb = new ProcessBuilder(wrappedCmdArgs); Map env = pb.environment(); addJobConfToEnvironment(hconf, env); - env.put(safeEnvVarName(HiveConf.ConfVars.HIVEALIAS.varname), String + env.put(safeEnvVarName(HiveConf.ConfVars.HIVE_ALIAS.varname), String .valueOf(alias)); // Create an environment variable that uniquely identifies this script // operator String idEnvVarName = HiveConf.getVar(hconf, - HiveConf.ConfVars.HIVESCRIPTIDENVVAR); + HiveConf.ConfVars.HIVE_SCRIPT_ID_ENV_VAR); String idEnvVarVal = getOperatorId(); env.put(safeEnvVarName(idEnvVarName), idEnvVarVal); @@ -376,11 +376,11 @@ public void process(Object row, int tag) throws HiveException { .getProperties()); errThread = new StreamThread(scriptErrReader, new ErrorStreamProcessor( - HiveConf.getIntVar(hconf, HiveConf.ConfVars.SCRIPTERRORLIMIT)), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.SCRIPT_ERROR_LIMIT)), "ErrorProcessor"); if (HiveConf - .getBoolVar(hconf, HiveConf.ConfVars.HIVESCRIPTAUTOPROGRESS)) { + .getBoolVar(hconf, HiveConf.ConfVars.HIVE_SCRIPT_AUTO_PROGRESS)) { autoProgressor = new AutoProgressor(this.getClass().getName(), reporter, Utilities.getDefaultNotificationInterval(hconf), HiveConf.getTimeVar( @@ -574,7 +574,7 @@ class CounterStatusProcessor { private final Reporter reporter; CounterStatusProcessor(Configuration hconf, Reporter reporter){ - this.reporterPrefix = HiveConf.getVar(hconf, HiveConf.ConfVars.STREAMREPORTERPERFIX); + this.reporterPrefix = HiveConf.getVar(hconf, HiveConf.ConfVars.STREAM_REPORTER_PREFIX); this.counterPrefix = reporterPrefix + "counter:"; this.statusPrefix = reporterPrefix + "status:"; this.reporter = reporter; @@ -625,7 +625,7 @@ class ErrorStreamProcessor implements StreamProcessor { public ErrorStreamProcessor(int maxBytes) { this.maxBytes = maxBytes; lastReportTime = 0; - if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.STREAMREPORTERENABLED)){ + if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.STREAM_REPORTER_ENABLED)){ counterStatus = new CounterStatusProcessor(hconf, reporter); } } @@ -732,7 +732,7 @@ public void run() { * Wrap the script in a wrapper that allows admins to control. */ protected String[] addWrapper(String[] inArgs) { - String wrapper = HiveConf.getVar(hconf, HiveConf.ConfVars.SCRIPTWRAPPER); + String wrapper = HiveConf.getVar(hconf, HiveConf.ConfVars.SCRIPT_WRAPPER); if (wrapper == null) { return inArgs; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java index ffaa252a25dc..10adb81fe050 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java @@ -66,7 +66,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { assert (colList.get(i) != null); eval[i] = ExprNodeEvaluatorFactory.get(colList.get(i), hconf); } - if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVEEXPREVALUATIONCACHE)) { + if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVE_EXPR_EVALUATION_CACHE)) { eval = ExprNodeEvaluatorFactory.toCachedEvals(eval); } output = new Object[eval.length]; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java index 843686bb30ad..5b1a7a7c2a1b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java @@ -320,7 +320,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { jc = new JobConf(hconf); } - defaultPartitionName = HiveConf.getVar(hconf, HiveConf.ConfVars.DEFAULTPARTITIONNAME); + defaultPartitionName = HiveConf.getVar(hconf, HiveConf.ConfVars.DEFAULT_PARTITION_NAME); currentStat = null; stats = new HashMap(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordReader.java index b227a70a147c..df43e9608900 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordReader.java @@ -60,7 +60,7 @@ public int next(Writable row) throws IOException { int bytesConsumed = lineReader.readLine((Text) row); - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESCRIPTESCAPE)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SCRIPT_ESCAPE)) { return HiveUtils.unescapeText((Text) row); } return bytesConsumed; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordWriter.java index acbf46678f6a..99c03fa2bd58 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordWriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TextRecordWriter.java @@ -46,7 +46,7 @@ public void write(Writable row) throws IOException { Text text = (Text) row; Text escapeText = text; - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESCRIPTESCAPE)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SCRIPT_ESCAPE)) { escapeText = HiveUtils.escapeText(text); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java index c58ed4e564d8..506266c829a7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java @@ -94,7 +94,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { // Set up periodic progress reporting in case the UDTF doesn't output rows // for a while - if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVEUDTFAUTOPROGRESS)) { + if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVE_UDTF_AUTO_PROGRESS)) { autoProgressor = new AutoProgressor(this.getClass().getName(), reporter, Utilities.getDefaultNotificationInterval(hconf), HiveConf.getTimeVar( diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 470d052d8982..6d38f1d5afca 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -2939,12 +2939,12 @@ public static Map getFullDPSpecs(Configuration conf, Dyn private static void validateDynPartitionCount(Configuration conf, Collection partitions) throws HiveException { int partsToLoad = partitions.size(); - int maxPartition = HiveConf.getIntVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS); + int maxPartition = HiveConf.getIntVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS); if (partsToLoad > maxPartition) { throw new HiveException("Number of dynamic partitions created is " + partsToLoad + ", which is more than " + maxPartition - +". To solve this try to set " + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname + +". To solve this try to set " + HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.varname + " to at least " + partsToLoad + '.'); } } @@ -3356,8 +3356,8 @@ public static String formatMsecToStr(long msec) { */ public static int estimateNumberOfReducers(HiveConf conf, ContentSummary inputSummary, MapWork work, boolean finalMapRed) throws IOException { - long bytesPerReducer = conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER); - int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); + long bytesPerReducer = conf.getLongVar(HiveConf.ConfVars.BYTES_PER_REDUCER); + int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAX_REDUCERS); double samplePercentage = getHighestSamplePercentage(work); long totalInputFileSize = getTotalInputFileSize(inputSummary, work, samplePercentage); @@ -3809,7 +3809,7 @@ public static void setInputPaths(JobConf job, List pathsToAdd) { */ public static void setInputAttributes(Configuration conf, MapWork mWork) { HiveConf.ConfVars var = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") ? - HiveConf.ConfVars.HIVETEZINPUTFORMAT : HiveConf.ConfVars.HIVEINPUTFORMAT; + HiveConf.ConfVars.HIVE_TEZ_INPUT_FORMAT : HiveConf.ConfVars.HIVE_INPUT_FORMAT; if (mWork.getInputformat() != null) { HiveConf.setVar(conf, var, mWork.getInputformat()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/errors/MapAggrMemErrorHeuristic.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/errors/MapAggrMemErrorHeuristic.java index aa6452e49103..da5bd837e0da 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/errors/MapAggrMemErrorHeuristic.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/errors/MapAggrMemErrorHeuristic.java @@ -46,7 +46,7 @@ public MapAggrMemErrorHeuristic() { @Override public void init(String query, JobConf conf) { super.init(query, conf); - configMatches = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE); + configMatches = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MAPSIDE_AGGREGATE); } @Override @@ -56,9 +56,9 @@ public ErrorAndSolution getErrorAndSolution() { List matchingLines = getRegexToLogLines().get(OUT_OF_MEMORY_REGEX); if (matchingLines.size() > 0) { - String confName = HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY.toString(); + String confName = HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY.toString(); float confValue = HiveConf.getFloatVar(getConf(), - HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); es = new ErrorAndSolution( "Out of memory due to hash maps used in map-side aggregation.", diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java index ce12cea66eda..e19b73dabcdf 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java @@ -208,7 +208,7 @@ public ExecDriver(MapredWork plan, JobConf job, boolean isSilent) throws HiveExc @Override public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { Counters.Counter cntr = ctrs.findCounter( - HiveConf.getVar(job, HiveConf.ConfVars.HIVECOUNTERGROUP), + HiveConf.getVar(job, HiveConf.ConfVars.HIVE_COUNTER_GROUP), Operator.HIVE_COUNTER_FATAL); return cntr != null && cntr.getValue() > 0; } @@ -264,7 +264,7 @@ public int execute() { job.setMapOutputValueClass(BytesWritable.class); try { - String partitioner = HiveConf.getVar(job, ConfVars.HIVEPARTITIONER); + String partitioner = HiveConf.getVar(job, ConfVars.HIVE_PARTITIONER); job.setPartitionerClass(JavaUtils.loadClass(partitioner)); } catch (ClassNotFoundException e) { throw new RuntimeException(e.getMessage(), e); @@ -282,7 +282,7 @@ public int execute() { job.setBoolean(MRJobConfig.REDUCE_SPECULATIVE, false); job.setBoolean(MRJobConfig.MAP_SPECULATIVE, false); - String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT); + String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVE_INPUT_FORMAT); if (mWork.isUseBucketizedHiveInputFormat()) { inpFormat = BucketizedHiveInputFormat.class.getName(); @@ -495,19 +495,19 @@ public static void propagateSplitSettings(JobConf job, MapWork work) { } if (work.getMaxSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, work.getMaxSplitSize()); + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, work.getMaxSplitSize()); } if (work.getMinSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, work.getMinSplitSize()); + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, work.getMinSplitSize()); } if (work.getMinSplitSizePerNode() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, work.getMinSplitSizePerNode()); + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE_PER_NODE, work.getMinSplitSizePerNode()); } if (work.getMinSplitSizePerRack() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, work.getMinSplitSizePerRack()); + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE_PER_RACK, work.getMinSplitSizePerRack()); } } @@ -572,7 +572,7 @@ private void handleSampling(Context context, MapWork mWork, JobConf job) protected void setInputAttributes(Configuration conf) { MapWork mWork = work.getMapWork(); if (mWork.getInputformat() != null) { - HiveConf.setVar(conf, ConfVars.HIVEINPUTFORMAT, mWork.getInputformat()); + HiveConf.setVar(conf, ConfVars.HIVE_INPUT_FORMAT, mWork.getInputformat()); } // Intentionally overwrites anything the user may have put here conf.setBoolean("hive.input.format.sorted", mWork.isInputFormatSorted()); @@ -692,14 +692,14 @@ public static void main(String[] args) throws IOException, HiveException { } } - boolean isSilent = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESESSIONSILENT); + boolean isSilent = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SESSION_SILENT); - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID, "").trim(); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID, "").trim(); if(queryId.isEmpty()) { queryId = "unknown-" + System.currentTimeMillis(); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYID, queryId); + HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID, queryId); } - System.setProperty(HiveConf.ConfVars.HIVEQUERYID.toString(), queryId); + System.setProperty(HiveConf.ConfVars.HIVE_QUERY_ID.toString(), queryId); LogUtils.registerLoggingContext(conf); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java index a5beb633bcb9..f7a658ea9242 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java @@ -236,7 +236,7 @@ public static class ReportStats implements Operator.OperatorFunc { public ReportStats(Reporter rp, Configuration conf) { this.rp = rp; - this.groupName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP); + this.groupName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); } @Override diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java index c365d41a8204..62d6e40d02c1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; -import java.io.Serializable; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Calendar; @@ -143,7 +142,7 @@ public void setJobId(JobID jobId) { public HadoopJobExecHelper(JobConf job, LogHelper console, Task task, HadoopJobExecHook hookCallBack) { - this.queryId = HiveConf.getVar(job, HiveConf.ConfVars.HIVEQUERYID, "unknown-" + System.currentTimeMillis()); + this.queryId = HiveConf.getVar(job, HiveConf.ConfVars.HIVE_QUERY_ID, "unknown-" + System.currentTimeMillis()); this.job = job; this.console = console; this.task = task; @@ -205,10 +204,10 @@ public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { return false; } // check for number of created files - Counters.Counter cntr = ctrs.findCounter(HiveConf.getVar(job, ConfVars.HIVECOUNTERGROUP), + Counters.Counter cntr = ctrs.findCounter(HiveConf.getVar(job, ConfVars.HIVE_COUNTER_GROUP), Operator.HIVE_COUNTER_CREATED_FILES); long numFiles = cntr != null ? cntr.getValue() : 0; - long upperLimit = HiveConf.getLongVar(job, HiveConf.ConfVars.MAXCREATEDFILES); + long upperLimit = HiveConf.getLongVar(job, HiveConf.ConfVars.MAX_CREATED_FILES); if (numFiles > upperLimit) { errMsg.append("total number of created files now is " + numFiles + ", which exceeds ").append(upperLimit); return true; @@ -226,7 +225,7 @@ private MapRedStats progress(ExecDriverTaskHandle th) throws IOException, LockEx job, HiveConf.ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL, TimeUnit.MILLISECONDS); boolean fatal = false; StringBuilder errMsg = new StringBuilder(); - long pullInterval = HiveConf.getLongVar(job, HiveConf.ConfVars.HIVECOUNTERSPULLINTERVAL); + long pullInterval = HiveConf.getLongVar(job, HiveConf.ConfVars.HIVE_COUNTERS_PULL_INTERVAL); boolean initializing = true; boolean initOutputPrinted = false; long cpuMsec = -1; @@ -437,7 +436,7 @@ private MapRedStats progress(ExecDriverTaskHandle th) throws IOException, LockEx //Set the number of table rows affected in mapRedStats to display number of rows inserted. if (ctrs != null) { Counter counter = ctrs.findCounter( - ss.getConf().getVar(HiveConf.ConfVars.HIVECOUNTERGROUP), + ss.getConf().getVar(HiveConf.ConfVars.HIVE_COUNTER_GROUP), FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN); if (counter != null) { mapRedStats.setNumModifiedRows(counter.getValue()); @@ -474,7 +473,7 @@ public void jobInfo(RunningJob rj) { } console.printInfo(getJobStartMsg(rj.getID()) + ", Tracking URL = " + rj.getTrackingURL()); - console.printInfo("Kill Command = " + HiveConf.getVar(job, ConfVars.MAPREDBIN) + console.printInfo("Kill Command = " + HiveConf.getVar(job, ConfVars.MAPRED_BIN) + " job -kill " + rj.getID()); } } @@ -557,14 +556,14 @@ public int progress(RunningJob rj, JobClient jc, Context ctx) throws IOException // remove the pwd from conf file so that job tracker doesn't show this // logs - String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); + String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTORE_PWD); if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); + HiveConf.setVar(job, HiveConf.ConfVars.METASTORE_PWD, "HIVE"); } // replace it back if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, pwd); + HiveConf.setVar(job, HiveConf.ConfVars.METASTORE_PWD, pwd); } // add to list of running jobs to kill in case of abnormal shutdown @@ -654,7 +653,7 @@ private Map extractAllCounterValues(Counters counters) { private List getClientStatPublishers() { List clientStatsPublishers = new ArrayList(); - String confString = HiveConf.getVar(job, HiveConf.ConfVars.CLIENTSTATSPUBLISHERS); + String confString = HiveConf.getVar(job, HiveConf.ConfVars.CLIENT_STATS_PUBLISHERS); confString = confString.trim(); if (confString.equals("")) { return clientStatsPublishers; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java index 386358493650..6c8b9ff00d7c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java @@ -107,7 +107,7 @@ public int execute() { // auto-determine local mode if allowed if (!ctx.isLocalOnlyExecutionMode() && - conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) { + conf.getBoolVar(HiveConf.ConfVars.LOCAL_MODE_AUTO)) { if (inputSummary == null) { inputSummary = Utilities.getInputSummary(ctx, work.getMapWork(), null); @@ -142,7 +142,7 @@ public int execute() { } } - runningViaChild = conf.getBoolVar(HiveConf.ConfVars.SUBMITVIACHILD); + runningViaChild = conf.getBoolVar(HiveConf.ConfVars.SUBMIT_VIA_CHILD); if (!runningViaChild) { // since we are running the mapred task in the same jvm, we should update the job conf @@ -172,7 +172,7 @@ public int execute() { super.setInputAttributes(conf); // enable assertion - String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOPBIN); + String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOP_BIN); String hiveJar = conf.getJar(); String libJars = super.getResource(conf, ResourceType.JAR); @@ -247,7 +247,7 @@ public int execute() { // if we are running in local mode - then the amount of memory used // by the child jvm can no longer default to the memory used by the // parent jvm - int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVEHADOOPMAXMEM); + int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVE_HADOOP_MAX_MEM); if (hadoopMem == 0) { // remove env var that would default child jvm to use parent's memory // as default. child jvm would use default memory for a hadoop client @@ -453,13 +453,13 @@ private void setNumberOfReducers() throws IOException { } console .printInfo("In order to change the average load for a reducer (in bytes):"); - console.printInfo(" set " + HiveConf.ConfVars.BYTESPERREDUCER.varname + console.printInfo(" set " + HiveConf.ConfVars.BYTES_PER_REDUCER.varname + "="); console.printInfo("In order to limit the maximum number of reducers:"); - console.printInfo(" set " + HiveConf.ConfVars.MAXREDUCERS.varname + console.printInfo(" set " + HiveConf.ConfVars.MAX_REDUCERS.varname + "="); console.printInfo("In order to set a constant number of reducers:"); - console.printInfo(" set " + HiveConf.ConfVars.HADOOPNUMREDUCERS + console.printInfo(" set " + HiveConf.ConfVars.HADOOP_NUM_REDUCERS + "="); } } @@ -478,13 +478,13 @@ public static String isEligibleForLocalMode(HiveConf conf, long inputLength, long inputFileCount) { - long maxBytes = conf.getLongVar(HiveConf.ConfVars.LOCALMODEMAXBYTES); - long maxInputFiles = conf.getIntVar(HiveConf.ConfVars.LOCALMODEMAXINPUTFILES); + long maxBytes = conf.getLongVar(HiveConf.ConfVars.LOCAL_MODE_MAX_BYTES); + long maxInputFiles = conf.getIntVar(HiveConf.ConfVars.LOCAL_MODE_MAX_INPUT_FILES); // check for max input size if (inputLength > maxBytes) { return "Input Size (= " + inputLength + ") is larger than " + - HiveConf.ConfVars.LOCALMODEMAXBYTES.varname + " (= " + maxBytes + ")"; + HiveConf.ConfVars.LOCAL_MODE_MAX_BYTES.varname + " (= " + maxBytes + ")"; } // ideally we would like to do this check based on the number of splits @@ -494,7 +494,7 @@ public static String isEligibleForLocalMode(HiveConf conf, if (inputFileCount > maxInputFiles) { return "Number of Input Files (= " + inputFileCount + ") is larger than " + - HiveConf.ConfVars.LOCALMODEMAXINPUTFILES.varname + "(= " + maxInputFiles + ")"; + HiveConf.ConfVars.LOCAL_MODE_MAX_INPUT_FILES.varname + "(= " + maxInputFiles + ")"; } // since local mode only runs with 1 reducers - make sure that the diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java index 62b74dc8425d..0a781a825809 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java @@ -152,7 +152,7 @@ public boolean requireLock() { @Override public int execute() { - if (conf.getBoolVar(HiveConf.ConfVars.SUBMITLOCALTASKVIACHILD)) { + if (conf.getBoolVar(HiveConf.ConfVars.SUBMIT_LOCAL_TASK_VIA_CHILD)) { // send task off to another jvm return executeInChildVM(); } else { @@ -167,8 +167,8 @@ private int executeInChildVM() { // generate the cmd line to run in the child jvm String hiveJar = conf.getJar(); - String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOPBIN); - conf.setVar(ConfVars.HIVEADDEDJARS, Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR)); + String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOP_BIN); + conf.setVar(ConfVars.HIVE_ADDED_JARS, Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR)); // write out the plan to a local file Path planPath = new Path(context.getLocalTmpPath(), "plan.xml"); MapredLocalWork plan = getWork(); @@ -235,8 +235,8 @@ private int executeInChildVM() { // if we are running in local mode - then the amount of memory used // by the child jvm can no longer default to the memory used by the // parent jvm - // int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVEHADOOPMAXMEM); - int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVEHADOOPMAXMEM); + // int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVE_HADOOP_MAX_MEM); + int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVE_HADOOP_MAX_MEM); if (hadoopMem == 0) { // remove env var that would default child jvm to use parent's memory // as default. child jvm would use default memory for a hadoop client diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java index 92a3df4fc8a7..9e116e0e243f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java @@ -624,8 +624,8 @@ private static void validateCapacity(long capacity) { if (capacity > Integer.MAX_VALUE) { throw new RuntimeException("Attempting to expand the hash table to " + capacity + " that overflows maximum array size. For this query, you may want to disable " - + ConfVars.HIVEDYNAMICPARTITIONHASHJOIN.varname + " or reduce " - + ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD.varname); + + ConfVars.HIVE_DYNAMIC_PARTITION_HASHJOIN.varname + " or reduce " + + ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD.varname); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java index ae84d2d6f06d..452ef5991e86 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; import org.apache.hadoop.hive.ql.exec.JoinUtil; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer.NonMatchedSmallTableIterator; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter; import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase; import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch; @@ -72,15 +71,15 @@ public HashMapWrapper(Map metaData) { } public HashMapWrapper() { - this(HiveConf.ConfVars.HIVEHASHTABLEKEYCOUNTADJUSTMENT.defaultFloatVal, - HiveConf.ConfVars.HIVEHASHTABLETHRESHOLD.defaultIntVal, - HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR.defaultFloatVal, -1); + this(HiveConf.ConfVars.HIVE_HASHTABLE_KEY_COUNT_ADJUSTMENT.defaultFloatVal, + HiveConf.ConfVars.HIVE_HASHTABLE_THRESHOLD.defaultIntVal, + HiveConf.ConfVars.HIVE_HASHTABLE_LOAD_FACTOR.defaultFloatVal, -1); } public HashMapWrapper(Configuration hconf, long keyCount) { - this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEKEYCOUNTADJUSTMENT), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLETHRESHOLD), - HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR), keyCount); + this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_KEY_COUNT_ADJUSTMENT), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_THRESHOLD), + HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_LOAD_FACTOR), keyCount); } private HashMapWrapper(float keyCountAdj, int threshold, float loadFactor, long keyCount) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java index e66977f758a8..d82c43df84c7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hive.ql.exec.JoinUtil.JoinResult; import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer.KeyValueHelper; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer.NonMatchedSmallTableIterator; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter; import org.apache.hadoop.hive.ql.exec.vector.rowbytescontainer.VectorRowBytesContainer; import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase; @@ -279,15 +278,15 @@ public int size() { public HybridHashTableContainer(Configuration hconf, long keyCount, long memoryAvailable, long estimatedTableSize, HybridHashTableConf nwayConf) throws SerDeException, IOException { - this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEKEYCOUNTADJUSTMENT), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLETHRESHOLD), - HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMEMCHECKFREQ), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINWBSIZE), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEWBSIZE), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINNUMPARTITIONS), - HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEMAPJOINOPTIMIZEDTABLEPROBEPERCENT), - HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINBLOOMFILTER), + this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_KEY_COUNT_ADJUSTMENT), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_THRESHOLD), + HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_LOAD_FACTOR), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MEMCHECK_FREQ), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MIN_WB_SIZE), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_WB_SIZE), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MIN_NUM_PARTITIONS), + HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_MAPJOIN_OPTIMIZED_TABLE_PROBE_PERCENT), + HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_BLOOMFILTER), estimatedTableSize, keyCount, memoryAvailable, nwayConf, HiveUtils.getLocalDirList(hconf)); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java index 79695975ef26..ac3570900775 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; import org.apache.hadoop.hive.ql.exec.JoinUtil; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer.NonMatchedSmallTableIterator; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter; import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase; import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch; @@ -104,10 +103,10 @@ public class MapJoinBytesTableContainer public MapJoinBytesTableContainer(Configuration hconf, MapJoinObjectSerDeContext valCtx, long keyCount, long memUsage) throws SerDeException { - this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEKEYCOUNTADJUSTMENT), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLETHRESHOLD), - HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEWBSIZE), + this(HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_KEY_COUNT_ADJUSTMENT), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_THRESHOLD), + HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_LOAD_FACTOR), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_WB_SIZE), valCtx, keyCount, memUsage); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java index 8ee54fe8a1d8..4100bbccab5d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java @@ -252,7 +252,7 @@ public int execute() { DumpMetaData dmd = new DumpMetaData(hiveDumpRoot, conf); // Initialize ReplChangeManager instance since we will require it to encode file URI. ReplChangeManager.getInstance(conf); - Path cmRoot = new Path(conf.getVar(HiveConf.ConfVars.REPLCMDIR)); + Path cmRoot = new Path(conf.getVar(HiveConf.ConfVars.REPL_CM_DIR)); Long lastReplId; LOG.info("Data copy at load enabled : {}", conf.getBoolVar(HiveConf.ConfVars.REPL_RUN_DATA_COPY_TASKS_ON_TARGET)); if (isFailoverTarget) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java index 6ce83ee3e700..690e9a298a0c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplLoadTask.java @@ -92,7 +92,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.LinkedList; @@ -436,8 +435,8 @@ a database ( directory ) } private boolean isReadOnlyHookRegistered() { - return conf.get(HiveConf.ConfVars.PREEXECHOOKS.varname) != null && - conf.get(HiveConf.ConfVars.PREEXECHOOKS.varname).contains(READ_ONLY_HOOK); + return conf.get(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname) != null && + conf.get(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname).contains(READ_ONLY_HOOK); } /** diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/util/PathInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/util/PathInfo.java index 7383d018ece7..cf4c5a57c7a8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/util/PathInfo.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/util/PathInfo.java @@ -41,7 +41,7 @@ public class PathInfo { public PathInfo(HiveConf hiveConf) { this.hiveConf = hiveConf; - stagingDir = HiveConf.getVar(hiveConf, HiveConf.ConfVars.STAGINGDIR); + stagingDir = HiveConf.getVar(hiveConf, HiveConf.ConfVars.STAGING_DIR); } public Map getFsScratchDirs() { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java index 61516a8604da..2c42c9b6156f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/util/ReplUtils.java @@ -70,7 +70,6 @@ import org.apache.hadoop.hive.ql.parse.repl.dump.metric.IncrementalDumpMetricCollector; import org.apache.hadoop.hive.ql.parse.repl.load.metric.BootstrapLoadMetricCollector; import org.apache.hadoop.hive.ql.parse.repl.load.metric.IncrementalLoadMetricCollector; -import org.apache.hadoop.hive.ql.parse.repl.load.metric.PreOptimizedBootstrapLoadMetricCollector; import org.apache.hadoop.hive.ql.parse.repl.metric.ReplicationMetricCollector; import org.apache.hadoop.hive.ql.parse.repl.metric.event.Metadata; import org.apache.hadoop.hive.ql.parse.repl.metric.event.Status; @@ -480,7 +479,7 @@ public static boolean failedWithNonRecoverableError(Path dumpRoot, HiveConf conf } public static Path getEncodedDumpRootPath(HiveConf conf, String dbname) throws UnsupportedEncodingException { - return new Path(conf.getVar(HiveConf.ConfVars.REPLDIR), + return new Path(conf.getVar(HiveConf.ConfVars.REPL_DIR), Base64.getEncoder().encodeToString(dbname .getBytes(StandardCharsets.UTF_8.name()))); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java index b9be333761e6..896be0018d79 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java @@ -372,28 +372,28 @@ private JobConf initializeVertexConf(JobConf baseConf, Context context, MapWork } if (mapWork.getMaxSplitSize() != null) { - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, mapWork.getMaxSplitSize().longValue()); } if (mapWork.getMinSplitSize() != null) { - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, mapWork.getMinSplitSize().longValue()); } if (mapWork.getMinSplitSizePerNode() != null) { - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE_PER_NODE, mapWork.getMinSplitSizePerNode().longValue()); } if (mapWork.getMinSplitSizePerRack() != null) { - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE_PER_RACK, mapWork.getMinSplitSizePerRack().longValue()); } Utilities.setInputAttributes(conf, mapWork); - String inpFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZINPUTFORMAT); + String inpFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_TEZ_INPUT_FORMAT); if (mapWork.isUseBucketizedHiveInputFormat()) { inpFormat = BucketizedHiveInputFormat.class.getName(); @@ -665,10 +665,10 @@ private Map createPartitionerConf(String partitionerClassName, * container size isn't set. */ public static Resource getContainerResource(Configuration conf) { - int memorySizeMb = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVETEZCONTAINERSIZE); + int memorySizeMb = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TEZ_CONTAINER_SIZE); if (memorySizeMb <= 0) { LOG.warn("No Tez container size specified by {}. Falling back to MapReduce container MB {}", - HiveConf.ConfVars.HIVETEZCONTAINERSIZE, MRJobConfig.MAP_MEMORY_MB); + HiveConf.ConfVars.HIVE_TEZ_CONTAINER_SIZE, MRJobConfig.MAP_MEMORY_MB); memorySizeMb = conf.getInt(MRJobConfig.MAP_MEMORY_MB, MRJobConfig.DEFAULT_MAP_MEMORY_MB); // When config is explicitly set to "-1" defaultValue does not work! if (memorySizeMb <= 0) { @@ -676,10 +676,10 @@ public static Resource getContainerResource(Configuration conf) { memorySizeMb = MRJobConfig.DEFAULT_MAP_MEMORY_MB; } } - int cpuCores = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVETEZCPUVCORES); + int cpuCores = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TEZ_CPU_VCORES); if (cpuCores <= 0) { LOG.warn("No Tez VCore size specified by {}. Falling back to MapReduce container VCores {}", - HiveConf.ConfVars.HIVETEZCPUVCORES, MRJobConfig.MAP_CPU_VCORES); + HiveConf.ConfVars.HIVE_TEZ_CPU_VCORES, MRJobConfig.MAP_CPU_VCORES); cpuCores = conf.getInt(MRJobConfig.MAP_CPU_VCORES, MRJobConfig.DEFAULT_MAP_CPU_VCORES); if (cpuCores <= 0) { LOG.warn("Falling back to default container VCores {}", MRJobConfig.DEFAULT_MAP_CPU_VCORES); @@ -705,9 +705,9 @@ Map getContainerEnvironment(Configuration conf, boolean isMap) { * are set */ private static String getContainerJavaOpts(Configuration conf) { - String javaOpts = HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZJAVAOPTS); + String javaOpts = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_TEZ_JAVA_OPTS); - String logLevel = HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZLOGLEVEL); + String logLevel = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_TEZ_LOG_LEVEL); List logProps = Lists.newArrayList(); TezUtils.addLog4jSystemProperties(logLevel, logProps); StringBuilder sb = new StringBuilder(); @@ -716,7 +716,7 @@ private static String getContainerJavaOpts(Configuration conf) { } logLevel = sb.toString(); - if (HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVETEZCONTAINERSIZE) > 0) { + if (HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TEZ_CONTAINER_SIZE) > 0) { if (javaOpts != null) { return javaOpts + " " + logLevel; } else { @@ -724,8 +724,8 @@ private static String getContainerJavaOpts(Configuration conf) { } } else { if (javaOpts != null && !javaOpts.isEmpty()) { - LOG.warn(HiveConf.ConfVars.HIVETEZJAVAOPTS + " will be ignored because " - + HiveConf.ConfVars.HIVETEZCONTAINERSIZE + " is not set!"); + LOG.warn(HiveConf.ConfVars.HIVE_TEZ_JAVA_OPTS + " will be ignored because " + + HiveConf.ConfVars.HIVE_TEZ_CONTAINER_SIZE + " is not set!"); } return logLevel + " " + MRHelpers.getJavaOptsForMRMapper(conf); } @@ -1100,7 +1100,7 @@ public List localizeTempFilesFromConf( String hdfsDirPathStr, Configuration conf) throws IOException, LoginException { List tmpResources = new ArrayList(); - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEADDFILESUSEHDFSLOCATION)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_ADD_FILES_USE_HDFS_LOCATION)) { // reference HDFS based resource directly, to use distribute cache efficiently. addHdfsResource(conf, tmpResources, LocalResourceType.FILE, getHdfsTempFilesFromConf(conf)); // local resources are session based. @@ -1146,7 +1146,7 @@ private static String[] getHdfsTempFilesFromConf(Configuration conf) { private static String[] getLocalTempFilesFromConf(Configuration conf) { String addedFiles = Utilities.getLocalResourceFiles(conf, SessionState.ResourceType.FILE); String addedJars = Utilities.getLocalResourceFiles(conf, SessionState.ResourceType.JAR); - String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS); + String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_AUX_JARS); String reloadableAuxJars = SessionState.get() == null ? null : SessionState.get().getReloadableAuxJars(); String allFiles = HiveStringUtils.joinIgnoringEmpty(new String[]{auxJars, reloadableAuxJars, addedJars, addedFiles}, ','); @@ -1159,13 +1159,13 @@ public static String[] getTempFilesFromConf(Configuration conf) { } String addedFiles = Utilities.getResourceFiles(conf, SessionState.ResourceType.FILE); if (StringUtils.isNotBlank(addedFiles)) { - HiveConf.setVar(conf, ConfVars.HIVEADDEDFILES, addedFiles); + HiveConf.setVar(conf, ConfVars.HIVE_ADDED_FILES, addedFiles); } String addedJars = Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR); if (StringUtils.isNotBlank(addedJars)) { - HiveConf.setVar(conf, ConfVars.HIVEADDEDJARS, addedJars); + HiveConf.setVar(conf, ConfVars.HIVE_ADDED_JARS, addedJars); } - String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS); + String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_AUX_JARS); String reloadableAuxJars = SessionState.get() == null ? null : SessionState.get().getReloadableAuxJars(); // need to localize the additional jars and files @@ -1178,7 +1178,7 @@ public static String[] getTempFilesFromConf(Configuration conf) { private static String[] getTempArchivesFromConf(Configuration conf) { String addedArchives = Utilities.getResourceFiles(conf, SessionState.ResourceType.ARCHIVE); if (StringUtils.isNotBlank(addedArchives)) { - HiveConf.setVar(conf, ConfVars.HIVEADDEDARCHIVES, addedArchives); + HiveConf.setVar(conf, ConfVars.HIVE_ADDED_ARCHIVES, addedArchives); return addedArchives.split(","); } return new String[0]; @@ -1272,7 +1272,7 @@ public String getExecJarPathLocal(Configuration configuration) { } } catch (Exception ignored) {} //Fall back to hive config, if the uri could not get, or it does not point to a .jar file - String jar = configuration.get(ConfVars.HIVEJAR.varname); + String jar = configuration.get(ConfVars.HIVE_JAR.varname); if (!StringUtils.isBlank(jar)) { return jar; } @@ -1466,7 +1466,7 @@ public JobConf createConfiguration(HiveConf hiveConf, boolean skipAMConf) throws conf.set(MRJobConfig.OUTPUT_KEY_CLASS, HiveKey.class.getName()); conf.set(MRJobConfig.OUTPUT_VALUE_CLASS, BytesWritable.class.getName()); - conf.set("mapred.partitioner.class", HiveConf.getVar(conf, HiveConf.ConfVars.HIVEPARTITIONER)); + conf.set("mapred.partitioner.class", HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_PARTITIONER)); conf.set("tez.runtime.partitioner.class", MRPartitioner.class.getName()); // Removing job credential entry/ cannot be set on the tasks @@ -1699,7 +1699,7 @@ public String createDagName(Configuration conf, QueryPlan plan) { } public static String getUserSpecifiedDagName(Configuration conf) { - String name = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYNAME); + String name = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_NAME); return (name != null) ? name : conf.get("mapred.job.name"); } @@ -1712,7 +1712,7 @@ public static String getUserSpecifiedDagName(Configuration conf) { * TODO This method is temporary. Ideally Hive should only need to pass to Tez the amount of memory * it requires to do the map join, and Tez should take care of figuring out how much to allocate * Adjust the percentage of memory to be reserved for the processor from Tez - * based on the actual requested memory by the Map Join, i.e. HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD + * based on the actual requested memory by the Map Join, i.e. HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD * @return the adjusted percentage */ static double adjustMemoryReserveFraction(long memoryRequested, HiveConf conf) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java index 023d4d371d6c..8df866a9ca78 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java @@ -79,7 +79,7 @@ public void init(ExecMapperContext context, MapredContext mrContext, Configurati this.hconf = hconf; this.desc = joinOp.getConf(); this.cacheKey = joinOp.getCacheKey(); - String counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP); + String counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); String vertexName = hconf.get(Operator.CONTEXT_NAME_KEY, ""); String counterName = Utilities.getVertexCounterName(HashTableLoaderCounters.HASHTABLE_LOAD_TIME_MS.name(), vertexName); this.htLoadCounter = tezContext.getTezProcessorContext().getCounters().findCounter(counterGroup, counterName); @@ -100,7 +100,7 @@ public void load(MapJoinTableContainer[] mapJoinTables, } boolean useOptimizedTables = HiveConf.getBoolVar( - hconf, HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE); + hconf, HiveConf.ConfVars.HIVE_MAPJOIN_USE_OPTIMIZED_TABLE); boolean useHybridGraceHashJoin = desc.isHybridHashJoin(); boolean isFirstKey = true; @@ -109,13 +109,13 @@ public void load(MapJoinTableContainer[] mapJoinTables, LOG.info("Memory manager allocates " + totalMapJoinMemory + " bytes for the loading hashtable."); if (totalMapJoinMemory <= 0) { totalMapJoinMemory = HiveConf.getLongVar( - hconf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); + hconf, HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD); } long processMaxMemory = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax(); if (totalMapJoinMemory > processMaxMemory) { float hashtableMemoryUsage = HiveConf.getFloatVar( - hconf, HiveConf.ConfVars.HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE); + hconf, HiveConf.ConfVars.HIVE_HASHTABLE_FOLLOWBY_GBY_MAX_MEMORY_USAGE); LOG.warn("totalMapJoinMemory value of " + totalMapJoinMemory + " is greater than the max memory size of " + processMaxMemory); // Don't want to attempt to grab more memory than we have available .. percentage is a bit arbitrary @@ -153,8 +153,8 @@ public void load(MapJoinTableContainer[] mapJoinTables, int numPartitions = 0; try { numPartitions = HybridHashTableContainer.calcNumPartitions(memory, maxSize, - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINNUMPARTITIONS), - HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINWBSIZE)); + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MIN_NUM_PARTITIONS), + HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MIN_WB_SIZE)); } catch (IOException e) { throw new HiveException(e); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java index 2a68b20c27b8..46df46bbdf66 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java @@ -23,14 +23,11 @@ import java.util.Arrays; import java.util.BitSet; import java.util.Comparator; -import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Optional; import java.util.Set; -import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.BlockLocation; @@ -191,7 +188,7 @@ public List initialize() throws Exception { int availableSlots = getAvailableSlotsCalculator().getAvailableSlots(); - if (HiveConf.getLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 1) <= 1) { + if (HiveConf.getLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 1) <= 1) { // broken configuration from mapred-default.xml final long blockSize = conf.getLongBytes(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT); @@ -199,7 +196,7 @@ public List initialize() throws Exception { TezMapReduceSplitsGrouper.TEZ_GROUPING_SPLIT_MIN_SIZE, TezMapReduceSplitsGrouper.TEZ_GROUPING_SPLIT_MIN_SIZE_DEFAULT); final long preferredSplitSize = Math.min(blockSize / 2, minGrouping); - HiveConf.setLongVar(jobConf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, preferredSplitSize); + HiveConf.setLongVar(jobConf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, preferredSplitSize); LOG.info("The preferred split size is " + preferredSplitSize); } @@ -216,7 +213,7 @@ public List initialize() throws Exception { InputSplit[] splits; if (generateSingleSplit && - conf.get(HiveConf.ConfVars.HIVETEZINPUTFORMAT.varname).equals(HiveInputFormat.class.getName())) { + conf.get(HiveConf.ConfVars.HIVE_TEZ_INPUT_FORMAT.varname).equals(HiveInputFormat.class.getName())) { MapWork mapWork = Utilities.getMapWork(jobConf); List paths = Utilities.getInputPathsTez(jobConf, mapWork); FileSystem fs = paths.get(0).getFileSystem(jobConf); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java index a1593cc8e372..5a31f22b200a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java @@ -99,7 +99,7 @@ public class MapRecordProcessor extends RecordProcessor { public MapRecordProcessor(final JobConf jconf, final ProcessorContext context) throws Exception { super(jconf, context); - String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVE_QUERY_ID); if (LlapProxy.isDaemon()) { setLlapOfFragmentId(context); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java index 2e87b3e44344..6c2fb4be7217 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java @@ -93,7 +93,7 @@ void init( .initialize(); } - String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVE_QUERY_ID); cache = ObjectCacheFactory.getCache(jconf, queryId, true); try { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java index 39c098b13738..c46082988582 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java @@ -86,7 +86,7 @@ public class ReduceRecordProcessor extends RecordProcessor { public ReduceRecordProcessor(final JobConf jconf, final ProcessorContext context) throws Exception { super(jconf, context); - String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(jconf, HiveConf.ConfVars.HIVE_QUERY_ID); cache = ObjectCacheFactory.getCache(jconf, queryId, true); dynamicValueCache = ObjectCacheFactory.getCache(jconf, queryId, false, true); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java index 0cf6bf0e313c..33d4210fb226 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java @@ -298,7 +298,7 @@ private void configureAmRegistry(SessionType session) { bySessionId.put(session.getSessionId(), session); HiveConf conf = session.getConf(); conf.set(ConfVars.LLAP_TASK_SCHEDULER_AM_REGISTRY_NAME.varname, amRegistryName); - conf.set(ConfVars.HIVESESSIONID.varname, session.getSessionId()); + conf.set(ConfVars.HIVE_SESSION_ID.varname, session.getSessionId()); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java index 7fbd1573ee77..c87c968b5c8f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hive.ql.exec.tez; -import org.apache.hadoop.hive.ql.exec.tez.TezSessionState.HiveResources; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -278,7 +276,7 @@ private TezSessionState getSession(HiveConf conf, boolean doOpen) throws Excepti // TODO Session re-use completely disabled for doAs=true. Always launches a new session. boolean nonDefaultUser = conf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS); - boolean jobNameSet = !HiveConf.getVar(conf, ConfVars.HIVETEZJOBNAME).equals("HIVE-%s"); + boolean jobNameSet = !HiveConf.getVar(conf, ConfVars.HIVE_TEZ_JOB_NAME).equals("HIVE-%s"); /* * if the user has specified a queue name themselves or job name is set, we create a new diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java index 015e826d8e0f..c2a9ae5203b1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java @@ -363,7 +363,7 @@ protected void openInternal(String[] additionalFilesNotFromConf, */ HiveConfUtil.updateCredentialProviderPasswordForJobs(tezConfig); - String tezJobNameFormat = HiveConf.getVar(conf, ConfVars.HIVETEZJOBNAME); + String tezJobNameFormat = HiveConf.getVar(conf, ConfVars.HIVE_TEZ_JOB_NAME); final TezClient session = TezClient.newBuilder(String.format(tezJobNameFormat, sessionId), tezConfig) .setIsSession(true).setLocalResources(commonLocalResources) .setCredentials(llapCredentials).setServicePluginDescriptor(servicePluginsDescriptor) @@ -608,7 +608,7 @@ private void setupSessionAcls(Configuration tezConf, HiveConf hiveConf) throws String loginUser = loginUserUgi == null ? null : loginUserUgi.getShortUserName(); boolean addHs2User = - HiveConf.getBoolVar(hiveConf, ConfVars.HIVETEZHS2USERACCESS); + HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_TEZ_HS2_USER_ACCESS); String viewStr = Utilities.getAclStringWithHiveModification(tezConf, TezConfiguration.TEZ_AM_VIEW_ACLS, addHs2User, user, loginUser); @@ -794,12 +794,12 @@ private Path createTezDir(String sessionId, String suffix) throws IOException { // tez needs its own scratch dir (per session) // TODO: De-link from SessionState. A TezSession can be linked to different Hive Sessions via the pool. SessionState sessionState = SessionState.get(); - String hdfsScratchDir = sessionState == null ? HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR) : sessionState + String hdfsScratchDir = sessionState == null ? HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR) : sessionState .getHdfsScratchDirURIString(); Path tezDir = new Path(hdfsScratchDir, TEZ_DIR); tezDir = new Path(tezDir, sessionId + ((suffix == null) ? "" : ("-" + suffix))); FileSystem fs = tezDir.getFileSystem(conf); - FsPermission fsPermission = new FsPermission(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIRPERMISSION)); + FsPermission fsPermission = new FsPermission(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR_PERMISSION)); fs.mkdirs(tezDir, fsPermission); // Make sure the path is normalized (we expect validation to pass since we just created it). tezDir = DagUtils.validateTargetDir(tezDir, conf).getPath(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java index 32942ef98a71..48e907b20ab7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java @@ -148,7 +148,7 @@ public int execute() { Context ctx = null; Ref sessionRef = Ref.from(null); - final String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); + final String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); try { // Get or create Context object. If we create it we have to clean it later as well. @@ -397,7 +397,7 @@ private void collectCommitInformation(TezWork work) throws IOException, TezExcep private void updateNumRows() { if (counters != null) { TezCounter counter = counters.findCounter( - conf.getVar(HiveConf.ConfVars.HIVECOUNTERGROUP), FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN); + conf.getVar(HiveConf.ConfVars.HIVE_COUNTER_GROUP), FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN); if (counter != null) { queryState.setNumModifiedRows(counter.getValue()); } @@ -486,8 +486,8 @@ DAG build(JobConf conf, TezWork tezWork, Path scratchDir, Context ctx, .put("description", ctx.getCmd()); String dagInfo = json.toString(); - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); - dag.setConf(HiveConf.ConfVars.HIVEQUERYID.varname, queryId); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); + dag.setConf(HiveConf.ConfVars.HIVE_QUERY_ID.varname, queryId); LOG.debug("DagInfo: {}", dagInfo); @@ -596,7 +596,7 @@ private static void setAccessControlsForCurrentUser(DAG dag, String queryId, String loginUser = loginUserUgi == null ? null : loginUserUgi.getShortUserName(); boolean addHs2User = - HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVETEZHS2USERACCESS); + HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_TEZ_HS2_USER_ACCESS); // Temporarily re-using the TEZ AM View ACLs property for individual dag access control. // Hive may want to setup it's own parameters if it wants to control per dag access. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java index 33f9a8a34d26..2c71296772fd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import com.fasterxml.jackson.annotation.JsonAutoDetect; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; import com.google.common.annotations.VisibleForTesting; @@ -1568,7 +1567,7 @@ public WmTezSession getSession(TezSessionState session, MappingInput input, Hive WmEvent wmEvent = new WmEvent(WmEvent.EventType.GET); // Note: not actually used for pool sessions; verify some things like doAs are not set. validateConfig(conf); - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); SettableFuture future = SettableFuture.create(); WmTezSession wmSession = checkSessionForReuse(session); GetRequest req = new GetRequest( diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java index c039342d1ed8..045fb13e70be 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java @@ -28,7 +28,6 @@ import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.Vertex; import org.apache.tez.dag.api.client.DAGClient; @@ -73,7 +72,7 @@ class DAGSummary implements PrintSummary { this.dagClient = dagClient; this.dag = dag; this.perfLogger = perfLogger; - this.hiveCountersGroup = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVECOUNTERGROUP); + this.hiveCountersGroup = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); this.hiveCounters = hiveCounters(dagClient); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java index 9cfe9fbec155..47694f241540 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.ConstantVectorExpression; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.FilterDesc; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.VectorDesc; @@ -84,7 +83,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { VectorExpression.doTransientInit(predicateExpression, hconf); try { heartbeatInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVESENDHEARTBEAT); + HiveConf.ConfVars.HIVE_SEND_HEARTBEAT); predicateExpression.init(hconf); } catch (Throwable e) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java index c2dfaeb074eb..505db9e5e611 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java @@ -405,7 +405,7 @@ public void initialize(Configuration hconf) throws HiveException { this.maxHtEntries = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_VECTORIZATION_GROUPBY_MAXENTRIES); this.numRowsCompareHashAggr = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL); + HiveConf.ConfVars.HIVE_GROUPBY_MAP_INTERVAL); } else { this.percentEntriesToFlush = @@ -415,7 +415,7 @@ public void initialize(Configuration hconf) throws HiveException { this.maxHtEntries = HiveConf.ConfVars.HIVE_VECTORIZATION_GROUPBY_MAXENTRIES.defaultIntVal; this.numRowsCompareHashAggr = - HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL.defaultIntVal; + HiveConf.ConfVars.HIVE_GROUPBY_MAP_INTERVAL.defaultIntVal; } minReductionHashAggr = getConf().getMinReductionHashAggr(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java index 836c1477e4f7..c288feb8f9f9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorInBloomFilterColDynamicValue.java @@ -102,7 +102,7 @@ public void init(Configuration conf) { throw new IllegalStateException("Unsupported type " + colVectorType); } - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); runtimeCache = ObjectCacheFactory.getCache(conf, queryId, false, true); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java index 460f05b0e174..cfada606eb16 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; import org.apache.hive.common.util.FixedSizedObjectPool; import org.apache.tez.common.counters.TezCounter; import org.slf4j.Logger; @@ -90,7 +89,7 @@ public VectorMapJoinFastHashTableLoader(TezContext context, Configuration hconf, this.desc = joinOp.getConf(); this.cacheKey = joinOp.getCacheKey(); this.htLoadCounter = this.tezContext.getTezProcessorContext().getCounters().findCounter( - HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP), hconf.get(Operator.CONTEXT_NAME_KEY, "")); + HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_COUNTER_GROUP), hconf.get(Operator.CONTEXT_NAME_KEY, "")); } @Override @@ -100,7 +99,7 @@ public void init(ExecMapperContext context, MapredContext mrContext, this.hconf = hconf; this.desc = joinOp.getConf(); this.cacheKey = joinOp.getCacheKey(); - String counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVECOUNTERGROUP); + String counterGroup = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); String vertexName = hconf.get(Operator.CONTEXT_NAME_KEY, ""); String counterName = Utilities.getVertexCounterName(HashTableLoaderCounters.HASHTABLE_LOAD_TIME_MS.name(), vertexName); this.htLoadCounter = tezContext.getTezProcessorContext().getCounters().findCounter(counterGroup, counterName); @@ -111,7 +110,7 @@ private void initHTLoadingService(long estKeyCount) { // Avoid many small HTs that will rehash multiple times causing GCs this.numLoadThreads = 1; } else { - int initialValue = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEMAPJOINPARALELHASHTABLETHREADS); + int initialValue = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_MAPJOIN_PARALEL_HASHTABLE_THREADS); Preconditions.checkArgument(initialValue > 0, "The number of HT-loading-threads should be positive."); int adjustedValue = Integer.highestOneBit(initialValue); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java index 1b281caf0ecd..20ae779f2215 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java @@ -62,9 +62,9 @@ public VectorMapJoinFastTableContainer(MapJoinDesc desc, Configuration hconf, this.desc = desc; this.hconf = hconf; - keyCountAdj = HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEKEYCOUNTADJUSTMENT); - threshold = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLETHRESHOLD); - loadFactor = HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVEHASHTABLELOADFACTOR); + keyCountAdj = HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_KEY_COUNT_ADJUSTMENT); + threshold = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_THRESHOLD); + loadFactor = HiveConf.getFloatVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_LOAD_FACTOR); this.numHTs = numHTs; this.estimatedKeyCount = estimatedKeys > numHTs ? (estimatedKeys/ numHTs) : estimatedKeys; @@ -98,7 +98,7 @@ private VectorMapJoinFastHashTableContainerBase createHashTables(int newThreshol boolean isFullOuter = vectorDesc.getIsFullOuter(); boolean minMaxEnabled = vectorDesc.getMinMaxEnabled(); - int writeBufferSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEHASHTABLEWBSIZE); + int writeBufferSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVE_HASHTABLE_WB_SIZE); VectorMapJoinFastHashTableContainerBase htWrapper = null; switch (hashTableKeyType) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java index 205a5464f1ed..5261a1beb1c5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistoryImpl.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.IOException; import java.io.PrintWriter; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.Random; @@ -80,7 +79,7 @@ public HiveHistoryImpl(SessionState ss) { try { console = new LogHelper(LOG); String conf_file_loc = ss.getConf().getVar( - HiveConf.ConfVars.HIVEHISTORYFILELOC); + HiveConf.ConfVars.HIVE_HISTORY_FILE_LOC); if ((conf_file_loc == null) || conf_file_loc.length() == 0) { console.printError("No history file location given"); return; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java index 74d6ac4ce017..c0e3bb30054c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java @@ -426,7 +426,7 @@ private HiveHookEventProtoPartialBuilder getPreHookEvent(HookContext hookContext plan.getOptimizedQueryString(), plan.getOptimizedCBOPlan()); return new HiveHookEventProtoPartialBuilder( - builder, explainWork, otherInfo, plan.getQueryStr(), conf.getVar(ConfVars.HIVESTAGEIDREARRANGE)); + builder, explainWork, otherInfo, plan.getQueryStr(), conf.getVar(ConfVars.HIVE_STAGE_ID_REARRANGE)); } private HiveHookEventProtoPartialBuilder getPostHookEvent(HookContext hookContext, boolean success) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java index cd23b247063a..f3fc63ac3e4b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java @@ -49,11 +49,11 @@ public class HookContext { static public enum HookType { - PRE_EXEC_HOOK(HiveConf.ConfVars.PREEXECHOOKS, ExecuteWithHookContext.class, + PRE_EXEC_HOOK(HiveConf.ConfVars.PRE_EXEC_HOOKS, ExecuteWithHookContext.class, "Pre-execution hooks to be invoked for each statement"), - POST_EXEC_HOOK(HiveConf.ConfVars.POSTEXECHOOKS, ExecuteWithHookContext.class, + POST_EXEC_HOOK(HiveConf.ConfVars.POST_EXEC_HOOKS, ExecuteWithHookContext.class, "Post-execution hooks to be invoked for each statement"), - ON_FAILURE_HOOK(HiveConf.ConfVars.ONFAILUREHOOKS, ExecuteWithHookContext.class, + ON_FAILURE_HOOK(HiveConf.ConfVars.ON_FAILURE_HOOKS, ExecuteWithHookContext.class, "On-failure hooks to be invoked for each statement"), QUERY_LIFETIME_HOOKS(HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS, QueryLifeTimeHook.class, "Hooks that will be triggered before/after query compilation and before/after query execution"), @@ -61,7 +61,7 @@ static public enum HookType { "Hooks that invoked before/after Hive performs its own semantic analysis on a statement"), DRIVER_RUN_HOOKS(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, HiveDriverRunHook.class, "Hooks that Will be run at the beginning and end of Driver.run"), - QUERY_REDACTOR_HOOKS(HiveConf.ConfVars.QUERYREDACTORHOOKS, Redactor.class, + QUERY_REDACTOR_HOOKS(HiveConf.ConfVars.QUERY_REDACTOR_HOOKS, Redactor.class, "Hooks to be invoked for each query which can transform the query before it's placed in the job.xml file"), // The HiveSessionHook.class cannot access, use Hook.class instead HIVE_SERVER2_SESSION_HOOK(HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, Hook.class, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java index 102b2b517312..7633ac85612f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecTezSummaryPrinter.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hive.ql.exec.tez.CompileTimeCounters; import org.apache.hadoop.hive.ql.exec.tez.HiveInputCounters; import org.apache.tez.common.counters.FileSystemCounter; -import org.apache.tez.dag.api.client.DAGClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -62,7 +61,7 @@ public void run(HookContext hookContext) throws Exception { LOG.info("Printing summary for tez task: " + tezTask.getName()); TezCounters counters = tezTask.getTezCounters(); if (counters != null) { - String hiveCountersGroup = HiveConf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP); + String hiveCountersGroup = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_COUNTER_GROUP); for (CounterGroup group : counters) { if (hiveCountersGroup.equals(group.getDisplayName())) { console.printInfo(tezTask.getId() + " HIVE COUNTERS:", false); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java index 987411401e4f..ab90142162cf 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -553,7 +553,7 @@ private void addSplitsForGroup(List dirs, TableScanOperator tableScan, Job } else { // if the input is Compressed OR not text we have no way of splitting them! // In that case RecordReader should take care of header/footer skipping! - HiveConf.setLongVar(conf, ConfVars.MAPREDMINSPLITSIZE, Long.MAX_VALUE); + HiveConf.setLongVar(conf, ConfVars.MAPRED_MIN_SPLIT_SIZE, Long.MAX_VALUE); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java b/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java index 6eba2e5b71a6..90bd7339a71d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java @@ -1057,7 +1057,7 @@ void init(Configuration conf, FSDataOutputStream out, this.out = out; this.codec = codec; this.metadata = metadata; - this.useNewMagic = conf.getBoolean(HIVEUSEEXPLICITRCFILEHEADER.varname, true); + this.useNewMagic = conf.getBoolean(HIVE_USE_EXPLICIT_RCFILE_HEADER.varname, true); } /** Returns the compression codec of data in this file. */ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java index 6d93de0b29c4..d6ae00bfa8d4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileRecordReader.java @@ -102,7 +102,7 @@ public RCFileRecordReader(Configuration conf, FileSplit split) this.conf = conf; this.split = split; - useCache = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEUSERCFILESYNCCACHE); + useCache = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_USE_RCFILE_SYNC_CACHE); if (split.getStart() > in.getPosition()) { long oldSync = useCache ? syncCache.get(split) : -1; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java index 7d808c25d254..dac1393976d6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java @@ -118,7 +118,7 @@ public int execute() { String jobName = null; if (noName && this.getQueryPlan() != null) { - int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); + int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVE_JOBNAME_LENGTH); jobName = Utilities.abbreviate(this.getQueryPlan().getQueryStr(), maxlen - 6); } @@ -137,9 +137,9 @@ public int execute() { Utilities.setMapWork(job, work, ctx.getMRTmpPath(), true); // remove pwd from conf file so that job tracker doesn't show this logs - String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); + String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTORE_PWD); if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); + HiveConf.setVar(job, HiveConf.ConfVars.METASTORE_PWD, "HIVE"); } // submit the job diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java index c19c49f266bc..bae96b1b67bc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ExternalCache.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.shims.HadoopShims.HdfsFileStatusWithId; -import org.apache.orc.OrcProto; import org.apache.orc.impl.OrcTail; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -98,7 +97,7 @@ public boolean hasPpd() { public void configure(HiveConf queryConfig) { this.conf = queryConfig; this.sarg = ConvertAstToSearchArg.createFromConf(conf); - this.isPpdEnabled = HiveConf.getBoolVar(conf, ConfVars.HIVEOPTINDEXFILTER) + this.isPpdEnabled = HiveConf.getBoolVar(conf, ConfVars.HIVE_OPT_INDEX_FILTER) && HiveConf.getBoolVar(conf, ConfVars.HIVE_ORC_MS_FOOTER_CACHE_PPD); this.isInTest = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST); this.sargIsOriginal = this.sargNotIsOriginal = null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java index 730ede4e5a02..564836144f0b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hive.ql.io.orc; import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; import org.apache.hadoop.hive.common.BlobStorageUtils; import org.apache.hadoop.hive.common.NoDynamicValuesException; @@ -333,7 +332,7 @@ public static int getRootColumn(boolean isOriginal) { public static void raiseAcidTablesMustBeReadWithAcidReaderException(Configuration conf) throws IOException { - String hiveInputFormat = HiveConf.getVar(conf, ConfVars.HIVEINPUTFORMAT); + String hiveInputFormat = HiveConf.getVar(conf, ConfVars.HIVE_INPUT_FORMAT); if (hiveInputFormat.equals(HiveInputFormat.class.getName())) { throw new IOException(ErrorMsg.ACID_TABLES_MUST_BE_READ_WITH_ACID_READER.getErrorCodedMsg()); } else { @@ -717,8 +716,8 @@ static class Context { this.isVectorMode = Utilities.getIsVectorized(conf); this.forceThreadpool = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST); this.sarg = ConvertAstToSearchArg.createFromConf(conf); - minSize = HiveConf.getLongVar(conf, ConfVars.MAPREDMINSPLITSIZE, DEFAULT_MIN_SPLIT_SIZE); - maxSize = HiveConf.getLongVar(conf, ConfVars.MAPREDMAXSPLITSIZE, DEFAULT_MAX_SPLIT_SIZE); + minSize = HiveConf.getLongVar(conf, ConfVars.MAPRED_MIN_SPLIT_SIZE, DEFAULT_MIN_SPLIT_SIZE); + maxSize = HiveConf.getLongVar(conf, ConfVars.MAPRED_MAX_SPLIT_SIZE, DEFAULT_MAX_SPLIT_SIZE); String ss = conf.get(ConfVars.HIVE_ORC_SPLIT_STRATEGY.varname); if (ss == null || ss.equals(SplitStrategyKind.HYBRID.name())) { splitStrategyKind = SplitStrategyKind.HYBRID; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java index cf73c9551ebd..3710ee71c7c9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java @@ -359,7 +359,7 @@ private static TypeDescription getTypeDescriptionFromTableProperties(Properties writerOptions.getConfiguration().set(OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getAttribute(), "-1.0"); } } - if(!HiveConf.getBoolVar(options.getConfiguration(), HiveConf.ConfVars.HIVETESTMODEACIDKEYIDXSKIP)) { + if(!HiveConf.getBoolVar(options.getConfiguration(), HiveConf.ConfVars.HIVE_TEST_MODE_ACID_KEY_IDX_SKIP)) { writerOptions.fileSystem(fs).callback(indexBuilder); } rowInspector = (StructObjectInspector)options.getInspector(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java index fe98dd0aee4e..43a47a95bb5b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java @@ -78,7 +78,7 @@ public boolean requireLock() { */ @Override public int execute() { - HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, + HiveConf.setVar(job, HiveConf.ConfVars.HIVE_INPUT_FORMAT, BucketizedHiveInputFormat.class.getName()); success = true; HiveFileFormatUtils.prepareJobOutput(job); @@ -105,16 +105,16 @@ public int execute() { job.setBoolean(MRJobConfig.MAP_SPECULATIVE, false); if (work.getMinSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, work + HiveConf.setLongVar(job, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, work .getMinSplitSize().longValue()); } if (work.getInputformat() != null) { - HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, work + HiveConf.setVar(job, HiveConf.ConfVars.HIVE_INPUT_FORMAT, work .getInputformat()); } - String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT); + String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVE_INPUT_FORMAT); LOG.info("Using " + inpFormat); try { @@ -146,7 +146,7 @@ public int execute() { String jobName = null; if (noName && this.getQueryPlan() != null) { - int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); + int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVE_JOBNAME_LENGTH); jobName = Utilities.abbreviate(this.getQueryPlan().getQueryStr(), maxlen - 6); } @@ -166,9 +166,9 @@ public int execute() { // remove the pwd from conf file so that job tracker doesn't show this // logs - String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); + String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTORE_PWD); if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); + HiveConf.setVar(job, HiveConf.ConfVars.METASTORE_PWD, "HIVE"); } JobClient jc = new JobClient(job); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java index b60570b1ec73..c4013fc2c789 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java @@ -268,7 +268,7 @@ long openTxn(Context ctx, String user, TxnType txnType, long delay) throws LockE shouldReallocateWriteIds = false; isExplicitTransaction = false; startTransactionCount = 0; - this.queryId = ctx.getConf().get(HiveConf.ConfVars.HIVEQUERYID.varname); + this.queryId = ctx.getConf().get(HiveConf.ConfVars.HIVE_QUERY_ID.varname); LOG.info("Opened " + JavaUtils.txnIdToString(txnId)); ctx.setHeartbeater(startHeartbeat(delay)); return txnId; @@ -727,7 +727,7 @@ private Heartbeater startHeartbeat(long initialDelay) throws LockException { private ScheduledFuture startHeartbeat(long initialDelay, long heartbeatInterval, Runnable heartbeater) { // For negative testing purpose.. - if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER)) { + if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_HEARTBEATER)) { initialDelay = 0; } else if (initialDelay == 0) { /*make initialDelay a random number in [0, 0.75*heartbeatInterval] so that if a lot @@ -1120,8 +1120,8 @@ public LockException getLockException() { public void run() { try { // For negative testing purpose.. - if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER)) { - throw new LockException(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER.name() + "=true"); + if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_HEARTBEATER)) { + throw new LockException(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_HEARTBEATER.name() + "=true"); } LOG.debug("Heartbeating...for currentUser: " + currentUser); currentUser.doAs((PrivilegedExceptionAction) () -> { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 192fa13ffa1f..b3ef272bbfe5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -3371,8 +3371,8 @@ public Map, Partition> loadDynamicPartitions(final LoadTable LOG.debug("Cancelling " + futures.size() + " dynamic loading tasks"); executor.shutdownNow(); } - if (HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) && HiveConf.getBoolVar(conf, ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION)) { - throw new HiveException(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION.name() + "=true"); + if (HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) && HiveConf.getBoolVar(conf, ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION)) { + throw new HiveException(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION.name() + "=true"); } try { if (isTxnTable) { @@ -3680,7 +3680,7 @@ public Partition getPartition(Table tbl, Map partSpec, for (FieldSchema field : tbl.getPartCols()) { String val = partSpec.get(field.getName()); // enable dynamic partitioning - if ((val == null && !HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONING)) + if ((val == null && !HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING)) || (val != null && val.length() == 0)) { throw new HiveException("get partition: Value for key " + field.getName() + " is null or empty"); @@ -4076,7 +4076,7 @@ public List getPartitionNames(Table tbl, ExprNodeGenericFuncDesc expr, S exprBytes = SerializationUtilities.serializeObjectWithTypeInformation(expr); } try { - String defaultPartitionName = HiveConf.getVar(conf, ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = HiveConf.getVar(conf, ConfVars.DEFAULT_PARTITION_NAME); PartitionsByExprRequest req = new PartitionsByExprRequest(tbl.getDbName(), tbl.getTableName(), ByteBuffer.wrap(exprBytes)); if (defaultPartitionName != null) { @@ -4523,7 +4523,7 @@ public boolean getPartitionsByExpr(Table tbl, ExprNodeDesc expr, HiveConf conf, perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_BY_EXPR); try { Preconditions.checkNotNull(partitions); - String defaultPartitionName = HiveConf.getVar(conf, ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = HiveConf.getVar(conf, ConfVars.DEFAULT_PARTITION_NAME); if (tbl.getStorageHandler() != null && tbl.getStorageHandler().alwaysUnpartitioned()) { partitions.addAll(tbl.getStorageHandler().getPartitionsByExpr(tbl, expr)); return false; @@ -5700,7 +5700,7 @@ private void deleteOldPathForReplace(Path destPath, Path oldPath, HiveConf conf, public void cleanUpOneDirectoryForReplace(Path path, FileSystem fs, PathFilter pathFilter, HiveConf conf, boolean purge, boolean isNeedRecycle) throws IOException, HiveException { - if (isNeedRecycle && conf.getBoolVar(HiveConf.ConfVars.REPLCMENABLED)) { + if (isNeedRecycle && conf.getBoolVar(HiveConf.ConfVars.REPL_CM_ENABLED)) { recycleDirToCmPath(path, purge); } if (!fs.exists(path)) { @@ -5910,7 +5910,7 @@ public synchronized IMetaStoreClient getMSC( } throw ex; } - String metaStoreUris = conf.getVar(HiveConf.ConfVars.METASTOREURIS); + String metaStoreUris = conf.getVar(HiveConf.ConfVars.METASTORE_URIS); if (!org.apache.commons.lang3.StringUtils.isEmpty(metaStoreUris)) { // get a synchronized wrapper if the meta store is remote. metaStoreClient = HiveMetaStoreClient.newSynchronizedClient(metaStoreClient); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreClientWithLocalCache.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreClientWithLocalCache.java index 625dbaeaf3eb..015895da9669 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreClientWithLocalCache.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreClientWithLocalCache.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest; import org.apache.hadoop.hive.metastore.api.UniqueConstraintsResponse; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator; import org.apache.hadoop.hive.ql.util.IncrementalObjectSizeEstimator.ObjectEstimator; @@ -520,7 +519,7 @@ private boolean isCacheEnabledAndInitialized() { protected String getQueryId() { try { - return Hive.get().getConf().get(HiveConf.ConfVars.HIVEQUERYID.varname); + return Hive.get().getConf().get(HiveConf.ConfVars.HIVE_QUERY_ID.varname); } catch (HiveException e) { LOG.error("Error getting query id. Query level and Global HMS caching will be disabled", e); return null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java index 3bb93ed60d1c..31efa27abec1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java @@ -1446,7 +1446,7 @@ public List dropPartitions(String catName, String dbName, String tblN List result = new ArrayList<>(); for (Pair pair : partExprs) { byte[] expr = pair.getRight(); - String filter = generateJDOFilter(table, expr, conf.get(HiveConf.ConfVars.DEFAULTPARTITIONNAME.varname)); + String filter = generateJDOFilter(table, expr, conf.get(HiveConf.ConfVars.DEFAULT_PARTITION_NAME.varname)); List partitions = tt.listPartitionsByFilter(filter); for (Partition p : partitions) { Partition droppedPartition = tt.dropPartition(p.getValues()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java index 8ffd41c49f94..e5109b34ef7e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/VirtualColumn.java @@ -112,7 +112,7 @@ public static List getRegistry(Configuration conf) { ArrayList l = new ArrayList(); l.add(BLOCKOFFSET); l.add(FILENAME); - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEROWOFFSET)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_ROW_OFFSET)) { l.add(ROWOFFSET); } l.add(ROWID); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java index 5d4e6cd8c8ce..bab07f179b1c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java @@ -49,7 +49,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // Throw an error if the user asked for bucketed mapjoin to be enforced and // bucketed mapjoin cannot be performed - if (!convert && conf.getBoolVar(HiveConf.ConfVars.HIVEENFORCEBUCKETMAPJOIN)) { + if (!convert && conf.getBoolVar(HiveConf.ConfVars.HIVE_ENFORCE_BUCKET_MAPJOIN)) { throw new SemanticException(ErrorMsg.BUCKET_MAPJOIN_NOT_POSSIBLE.getMsg()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java index 98a80ab603ee..87b3820f6735 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java @@ -113,7 +113,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor { OptimizeTezProcContext context = (OptimizeTezProcContext) procCtx; - hashTableLoadFactor = context.conf.getFloatVar(ConfVars.HIVEHASHTABLELOADFACTOR); + hashTableLoadFactor = context.conf.getFloatVar(ConfVars.HIVE_HASHTABLE_LOAD_FACTOR); fastHashTableAvailable = context.conf.getBoolVar(ConfVars.HIVE_VECTORIZATION_MAPJOIN_NATIVE_FAST_HASHTABLE_ENABLED); JoinOperator joinOp = (JoinOperator) nd; @@ -133,7 +133,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor { TezBucketJoinProcCtx tezBucketJoinProcCtx = new TezBucketJoinProcCtx(context.conf); - boolean hiveConvertJoin = context.conf.getBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN) & + boolean hiveConvertJoin = context.conf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN) & !context.parseContext.getDisableMapJoin(); if (!hiveConvertJoin) { // we are just converting to a common merge join operator. The shuffle @@ -251,7 +251,7 @@ private boolean selectJoinForLlap(OptimizeTezProcContext context, JoinOperator j TezBucketJoinProcCtx tezBucketJoinProcCtx, LlapClusterStateForCompile llapInfo, MapJoinConversion mapJoinConversion, int numBuckets) throws SemanticException { - if (!context.conf.getBoolVar(HiveConf.ConfVars.HIVEDYNAMICPARTITIONHASHJOIN) + if (!context.conf.getBoolVar(HiveConf.ConfVars.HIVE_DYNAMIC_PARTITION_HASHJOIN) && numBuckets > 1) { // DPHJ is disabled, only attempt BMJ or mapjoin return convertJoinBucketMapJoin(joinOp, context, mapJoinConversion, tezBucketJoinProcCtx); @@ -406,7 +406,7 @@ private static long hashTableDataSizeAdjustment(long numRows, List - HiveConf.getIntVar(context.conf, HiveConf.ConfVars.XPRODSMALLTABLEROWSTHRESHOLD)) { + HiveConf.getIntVar(context.conf, HiveConf.ConfVars.XPROD_SMALL_TABLE_ROWS_THRESHOLD)) { // if any of smaller side is estimated to generate more than // threshold rows we would disable mapjoin return null; @@ -1328,7 +1328,7 @@ public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcCo } MapJoinDesc mapJoinDesc = mapJoinOp.getConf(); mapJoinDesc.setHybridHashJoin(HiveConf.getBoolVar(context.conf, - HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN)); + HiveConf.ConfVars.HIVE_USE_HYBRIDGRACE_HASHJOIN)); List joinExprs = mapJoinDesc.getKeys().values().iterator().next(); if (joinExprs.size() == 0) { // In case of cross join, we disable hybrid grace hash join mapJoinDesc.setHybridHashJoin(false); @@ -1585,8 +1585,8 @@ private boolean convertJoinDynamicPartitionedHashJoin(JoinOperator joinOp, Optim private void fallbackToReduceSideJoin(JoinOperator joinOp, OptimizeTezProcContext context) throws SemanticException { - if (context.conf.getBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN) && - context.conf.getBoolVar(HiveConf.ConfVars.HIVEDYNAMICPARTITIONHASHJOIN)) { + if (context.conf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN) && + context.conf.getBoolVar(HiveConf.ConfVars.HIVE_DYNAMIC_PARTITION_HASHJOIN)) { if (convertJoinDynamicPartitionedHashJoin(joinOp, context)) { return; } @@ -1617,7 +1617,7 @@ private void fallbackToMergeJoin(JoinOperator joinOp, OptimizeTezProcContext con private boolean checkNumberOfEntriesForHashTable(JoinOperator joinOp, int position, OptimizeTezProcContext context) { long max = HiveConf.getLongVar(context.parseContext.getConf(), - HiveConf.ConfVars.HIVECONVERTJOINMAXENTRIESHASHTABLE); + HiveConf.ConfVars.HIVE_CONVERT_JOIN_MAX_ENTRIES_HASHTABLE); if (max < 1) { // Max is disabled, we can safely return true return true; @@ -1652,7 +1652,7 @@ private boolean checkNumberOfEntriesForHashTable(JoinOperator joinOp, int positi private boolean checkShuffleSizeForLargeTable(JoinOperator joinOp, int position, OptimizeTezProcContext context) { long max = HiveConf.getLongVar(context.parseContext.getConf(), - HiveConf.ConfVars.HIVECONVERTJOINMAXSHUFFLESIZE); + HiveConf.ConfVars.HIVE_CONVERT_JOIN_MAX_SHUFFLE_SIZE); if (max < 1) { // Max is disabled, we can safely return false return false; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java index 32edacba7c3e..29e421136129 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java @@ -180,7 +180,7 @@ protected int checkCountDistinct(GroupByOperator mGby, ReduceSinkOperator rs, return -1; } // check if it is potential to trigger nullscan - if (pGraphContext.getConf().getBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES)) { + if (pGraphContext.getConf().getBoolVar(HiveConf.ConfVars.HIVE_METADATA_ONLY_QUERIES)) { for (TableScanOperator tsOp : pGraphContext.getTopOps().values()) { List colIDs = tsOp.getNeededColumnIDs(); TableScanDesc desc = tsOp.getConf(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java index 85a420df8883..bc4bff81d6b8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java @@ -492,16 +492,16 @@ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext pars // do a group by on the list to dedup float groupByMemoryUsage = - HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf.getFloatVar(parseContext.getConf(), - HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf.getFloatVar(parseContext.getConf(), - ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf.getFloatVar(parseContext.getConf(), - ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); List groupByExprs = new ArrayList(); ExprNodeDesc groupByExpr = @@ -613,16 +613,16 @@ private boolean generateSemiJoinOperatorPlan(DynamicListContext ctx, ParseContex // do a group by to aggregate min,max and bloom filter. float groupByMemoryUsage = - HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf.getFloatVar(parseContext.getConf(), - HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf.getFloatVar(parseContext.getConf(), - ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf.getFloatVar(parseContext.getConf(), - ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); // Add min/max and bloom filter aggregations List aggFnOIs = new ArrayList(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index 94dc5a5c26ec..175f868bb82d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -544,7 +544,7 @@ public static void setMapWork(MapWork plan, ParseContext parseCtx, Set fileSinkOp = OperatorFactory.get( parent.getCompilationOpContext(), desc, parent.getSchema()); @@ -1251,9 +1251,9 @@ public static void createMRWorkForMergingFiles(FileSinkOperator fsInput, + " into " + finalName); } - boolean isBlockMerge = (conf.getBoolVar(ConfVars.HIVEMERGERCFILEBLOCKLEVEL) && + boolean isBlockMerge = (conf.getBoolVar(ConfVars.HIVE_MERGE_RCFILE_BLOCK_LEVEL) && fsInputDesc.getTableInfo().getInputFileFormatClass().equals(RCFileInputFormat.class)) || - (conf.getBoolVar(ConfVars.HIVEMERGEORCFILESTRIPELEVEL) && + (conf.getBoolVar(ConfVars.HIVE_MERGE_ORC_FILE_STRIPE_LEVEL) && fsInputDesc.getTableInfo().getInputFileFormatClass().equals(OrcInputFormat.class)); RowSchema inputRS = fsInput.getSchema(); @@ -1268,7 +1268,7 @@ public static void createMRWorkForMergingFiles(FileSinkOperator fsInput, // Create a FileSink operator TableDesc ts = (TableDesc) fsInputDesc.getTableInfo().clone(); Path mergeDest = srcMmWriteId == null ? finalName : finalName.getParent(); - fsOutputDesc = new FileSinkDesc(mergeDest, ts, conf.getBoolVar(ConfVars.COMPRESSRESULT)); + fsOutputDesc = new FileSinkDesc(mergeDest, ts, conf.getBoolVar(ConfVars.COMPRESS_RESULT)); fsOutputDesc.setMmWriteId(srcMmWriteId); fsOutputDesc.setIsMerge(true); // Create and attach the filesink for the merge. @@ -1316,7 +1316,7 @@ public static void createMRWorkForMergingFiles(FileSinkOperator fsInput, cplan = GenMapRedUtils.createMergeTask(fsInputDesc, finalName, dpCtx != null && dpCtx.getNumDPCols() > 0, fsInput.getCompilationOpContext()); if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { - work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf); + work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVE_QUERY_ID), conf); cplan.setName("File Merge"); ((TezWork) work).add(cplan); } else { @@ -1325,7 +1325,7 @@ public static void createMRWorkForMergingFiles(FileSinkOperator fsInput, } else { cplan = createMRWorkForMergingFiles(conf, tsMerge, fsInputDesc); if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { - work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf); + work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVE_QUERY_ID), conf); cplan.setName("File Merge"); ((TezWork)work).add(cplan); } else { @@ -1910,7 +1910,7 @@ public static boolean isMergeRequired(List> mvTasks, HiveConf hco if (currTask.getWork() instanceof TezWork) { // tez blurs the boundary between map and reduce, thus it has it's own config - return hconf.getBoolVar(ConfVars.HIVEMERGETEZFILES); + return hconf.getBoolVar(ConfVars.HIVE_MERGE_TEZ_FILES); } return isMergeRequiredForMr(hconf, fsOp, currTask); } @@ -1918,12 +1918,12 @@ public static boolean isMergeRequired(List> mvTasks, HiveConf hco private static boolean isMergeRequiredForMr(HiveConf hconf, FileSinkOperator fsOp, Task currTask) { if (fsOp.getConf().isLinkedFileSink()) { - // If the user has HIVEMERGEMAPREDFILES set to false, the idea was the + // If the user has HIVE_MERGE_MAPRED_FILES set to false, the idea was the // number of reducers are few, so the number of files anyway are small. // However, with this optimization, we are increasing the number of files // possibly by a big margin. So, merge aggressively. - return (hconf.getBoolVar(ConfVars.HIVEMERGEMAPFILES) || - hconf.getBoolVar(ConfVars.HIVEMERGEMAPREDFILES)); + return (hconf.getBoolVar(ConfVars.HIVE_MERGE_MAPFILES) || + hconf.getBoolVar(ConfVars.HIVE_MERGE_MAPRED_FILES)); } // There are separate configuration parameters to control whether to // merge for a map-only job @@ -1931,9 +1931,9 @@ private static boolean isMergeRequiredForMr(HiveConf hconf, if (currTask.getWork() instanceof MapredWork) { ReduceWork reduceWork = ((MapredWork) currTask.getWork()).getReduceWork(); boolean mergeMapOnly = - hconf.getBoolVar(ConfVars.HIVEMERGEMAPFILES) && reduceWork == null; + hconf.getBoolVar(ConfVars.HIVE_MERGE_MAPFILES) && reduceWork == null; boolean mergeMapRed = - hconf.getBoolVar(ConfVars.HIVEMERGEMAPREDFILES) && + hconf.getBoolVar(ConfVars.HIVE_MERGE_MAPRED_FILES) && reduceWork != null; if (mergeMapOnly || mergeMapRed) { return true; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java index f52d5652b608..ec2a6ccb818c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java @@ -83,7 +83,7 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { Map opRules = new LinkedHashMap(); HiveConf conf = pctx.getConf(); - if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { // process group-by pattern opRules.put(new RuleRegExp("R1", GroupByOperator.getOperatorName() + "%" + @@ -188,7 +188,7 @@ protected void processGroupBy(GroupByOptimizerContext ctx, if (!groupByOpDesc.isDistinct()) { removeReduceSink = true; } - else if (!HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + else if (!HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { // Optimize the query: select count(distinct keys) from T, where // T is bucketized and sorted by T // Partial aggregation can be done by the mappers in this scenario diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java index 4cae3b26a3fd..8b71ea65f283 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java @@ -251,10 +251,10 @@ private static class LimitPushdownContext implements NodeProcessorCtx { private final float threshold; public LimitPushdownContext(HiveConf conf) throws SemanticException { - threshold = conf.getFloatVar(HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE); + threshold = conf.getFloatVar(HiveConf.ConfVars.HIVE_LIMIT_PUSHDOWN_MEMORY_USAGE); if (threshold <= 0 || threshold >= 1) { throw new SemanticException("Invalid memory usage value " + threshold + - " for " + HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE); + " for " + HiveConf.ConfVars.HIVE_LIMIT_PUSHDOWN_MEMORY_USAGE); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java index adf4fbe1b216..839db8f9bbc6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java @@ -413,7 +413,7 @@ private static boolean checkFullOuterMapJoinCompatible(HiveConf hiveConf, boolean isVectorizationMapJoinNativeEnabled = HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_VECTORIZATION_MAPJOIN_NATIVE_ENABLED); boolean isHybridHashJoin = HiveConf.getBoolVar(hiveConf, - HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN); + HiveConf.ConfVars.HIVE_USE_HYBRIDGRACE_HASHJOIN); if (isVectorizationMapJoinNativeEnabled && isHybridHashJoin) { LOG.debug("FULL OUTER MapJoin not enabled: Native Vector MapJoin and Hybrid Grace not supported"); return false; @@ -469,12 +469,12 @@ public static boolean isFullOuterMapEnabled(HiveConf hiveConf, JoinOperator join final boolean isEnabled = HiveConf.getBoolVar( hiveConf, - HiveConf.ConfVars.HIVEMAPJOINFULLOUTER); + HiveConf.ConfVars.HIVE_MAPJOIN_FULL_OUTER); switch (mapJoinFullOuterOverride) { case NONE: { if (!isEnabled) { - LOG.debug("FULL OUTER MapJoin not enabled: {} is false", HiveConf.ConfVars.HIVEMAPJOINFULLOUTER.varname); + LOG.debug("FULL OUTER MapJoin not enabled: {} is false", HiveConf.ConfVars.HIVE_MAPJOIN_FULL_OUTER.varname); return false; } } @@ -483,18 +483,18 @@ public static boolean isFullOuterMapEnabled(HiveConf hiveConf, JoinOperator join if (LOG.isDebugEnabled()) { LOG.debug("FULL OUTER MapJoin not enabled: " + HiveConf.ConfVars.HIVE_TEST_MAPJOINFULLOUTER_OVERRIDE.varname + " is disable (" + - " " + HiveConf.ConfVars.HIVEMAPJOINFULLOUTER.varname + " is " + isEnabled + ")"); + " " + HiveConf.ConfVars.HIVE_MAPJOIN_FULL_OUTER.varname + " is " + isEnabled + ")"); } return false; case ENABLE: // Different parts of the code may rely on this being set... HiveConf.setBoolVar(hiveConf, - HiveConf.ConfVars.HIVEMAPJOINFULLOUTER, true); + HiveConf.ConfVars.HIVE_MAPJOIN_FULL_OUTER, true); if (LOG.isDebugEnabled()) { LOG.debug("FULL OUTER MapJoin is enabled: " + HiveConf.ConfVars.HIVE_TEST_MAPJOINFULLOUTER_OVERRIDE.varname + " is enable (" + - " " + HiveConf.ConfVars.HIVEMAPJOINFULLOUTER.varname + " is " + isEnabled + ")"); + " " + HiveConf.ConfVars.HIVE_MAPJOIN_FULL_OUTER.varname + " is " + isEnabled + ")"); } break; default: @@ -520,9 +520,9 @@ public static boolean isFullOuterMapEnabled(HiveConf hiveConf, JoinOperator join final boolean isOptimizedHashTableEnabled = HiveConf.getBoolVar( hiveConf, - HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE); + HiveConf.ConfVars.HIVE_MAPJOIN_USE_OPTIMIZED_TABLE); if (!isOptimizedHashTableEnabled) { - LOG.debug("FULL OUTER MapJoin not enabled: {} is false", HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE.varname); + LOG.debug("FULL OUTER MapJoin not enabled: {} is false", HiveConf.ConfVars.HIVE_MAPJOIN_USE_OPTIMIZED_TABLE.varname); return false; } @@ -652,8 +652,8 @@ public MapJoinOperator generateMapJoinOperator(ParseContext pctx, JoinOperator o int mapJoinPos) throws SemanticException { HiveConf hiveConf = pctx.getConf(); boolean noCheckOuterJoin = HiveConf.getBoolVar(hiveConf, - HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN) - && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN); + HiveConf.ConfVars.HIVE_OPT_SORT_MERGE_BUCKET_MAPJOIN) + && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_BUCKET_MAPJOIN); MapJoinOperator mapJoinOp = convertMapJoin(pctx.getConf(), op, op.getConf().isLeftInputJoin(), op.getConf().getBaseSrc(), diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index 41bdf77f5fbb..3341be88fd10 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -74,7 +74,7 @@ public void initialize(HiveConf hiveConf) { // Add the transformation that computes the lineage information. Set postExecHooks = Sets.newHashSet( Splitter.on(",").trimResults().omitEmptyStrings().split( - Strings.nullToEmpty(HiveConf.getVar(hiveConf, HiveConf.ConfVars.POSTEXECHOOKS)))); + Strings.nullToEmpty(HiveConf.getVar(hiveConf, HiveConf.ConfVars.POST_EXEC_HOOKS)))); if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_LINEAGE_INFO) || postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.PostExecutePrinter") || postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.LineageLogger") @@ -83,33 +83,33 @@ public void initialize(HiveConf hiveConf) { } // Try to transform OR predicates in Filter into simpler IN clauses first - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZER) && + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_POINT_LOOKUP_OPTIMIZER) && !pctx.getContext().isCboSucceeded()) { final int min = HiveConf.getIntVar(hiveConf, - HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZERMIN); + HiveConf.ConfVars.HIVE_POINT_LOOKUP_OPTIMIZER_MIN); transformations.add(new PointLookupOptimizer(min)); } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEPARTITIONCOLUMNSEPARATOR)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_PARTITION_COLUMN_SEPARATOR)) { transformations.add(new PartitionColumnsSeparator()); } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD) && + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_PPD) && !pctx.getContext().isCboSucceeded()) { transformations.add(new PredicateTransitivePropagate()); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_CONSTANT_PROPAGATION)) { transformations.add(new ConstantPropagate()); } transformations.add(new SyntheticJoinPredicate()); transformations.add(new PredicatePushDown()); - } else if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD) && + } else if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_PPD) && pctx.getContext().isCboSucceeded()) { transformations.add(new SyntheticJoinPredicate()); transformations.add(new SimplePredicatePushDown()); transformations.add(new RedundantDynamicPruningConditionsRemoval()); } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION) && + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_CONSTANT_PROPAGATION) && (!pctx.getContext().isCboSucceeded() || pctx.getContext().getOperation() == Context.Operation.MERGE)) { // We run constant propagation twice because after predicate pushdown, filter expressions // are combined and may become eligible for reduction (like is not null filter). @@ -121,26 +121,26 @@ public void initialize(HiveConf hiveConf) { transformations.add(new SortedDynPartitionTimeGranularityOptimizer()); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_PPD)) { transformations.add(new PartitionPruner()); transformations.add(new PartitionConditionRemover()); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTLISTBUCKETING)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_LIST_BUCKETING)) { /* Add list bucketing pruner. */ transformations.add(new ListBucketingPruner()); } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION) && + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_CONSTANT_PROPAGATION) && !pctx.getContext().isCboSucceeded()) { // PartitionPruner may create more folding opportunities, run ConstantPropagate again. transformations.add(new ConstantPropagate()); } } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTGROUPBY) || + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_GROUPBY) || HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_MAP_GROUPBY_SORT)) { transformations.add(new GroupByOptimizer()); } transformations.add(new ColumnPruner()); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVECOUNTDISTINCTOPTIMIZER) + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_COUNT_DISTINCT_OPTIMIZER) && (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_IN_TEST) || isTezExecEngine)) { transformations.add(new CountDistinctRewriteProc()); } @@ -156,7 +156,7 @@ public void initialize(HiveConf hiveConf) { MapJoinProcessor mapJoinProcessor = new MapJoinProcessor(); transformations.add(mapJoinProcessor); - if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN)) + if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_BUCKET_MAPJOIN)) && !isTezExecEngine) { transformations.add(new BucketMapJoinOptimizer()); bucketMapJoinOptimizer = true; @@ -164,7 +164,7 @@ public void initialize(HiveConf hiveConf) { // If optimize hive.optimize.bucketmapjoin.sortedmerge is set, add both // BucketMapJoinOptimizer and SortedMergeBucketMapJoinOptimizer - if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) + if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_SORT_MERGE_BUCKET_MAPJOIN)) && !isTezExecEngine) { if (!bucketMapJoinOptimizer) { // No need to add BucketMapJoinOptimizer twice @@ -173,20 +173,20 @@ public void initialize(HiveConf hiveConf) { transformations.add(new SortedMergeBucketMapJoinOptimizer()); } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEBUCKETINGSORTING)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_BUCKETING_SORTING)) { transformations.add(new BucketingSortingReduceSinkOptimizer()); } transformations.add(new UnionProcessor()); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.NWAYJOINREORDER)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.N_WAY_JOIN_REORDER)) { transformations.add(new JoinReorder()); } if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.TEZ_OPTIMIZE_BUCKET_PRUNING) - && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD) - && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTINDEXFILTER)) { + && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_PPD) + && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER)) { final boolean compatMode = HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.TEZ_OPTIMIZE_BUCKET_PRUNING_COMPAT); transformations.add(new FixedBucketPruningOptimizer(compatMode)); @@ -194,31 +194,31 @@ public void initialize(HiveConf hiveConf) { transformations.add(new BucketVersionPopulator()); - if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION) && + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_REDUCE_DEDUPLICATION) && !isTezExecEngine) { transformations.add(new ReduceSinkDeDuplication()); } transformations.add(new NonBlockingOpDeDupProc()); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEIDENTITYPROJECTREMOVER) + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_IDENTITY_PROJECT_REMOVER) && !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP)) { transformations.add(new IdentityProjectRemover()); } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVELIMITOPTENABLE)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_LIMIT_OPT_ENABLE)) { transformations.add(new GlobalLimitOptimizer()); } - if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCORRELATION) && - !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEGROUPBYSKEW) && + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_CORRELATION) && + !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_GROUPBY_SKEW) && !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME) && !isTezExecEngine) { transformations.add(new CorrelationOptimizer()); } - if (HiveConf.getFloatVar(hiveConf, HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE) > 0) { + if (HiveConf.getFloatVar(hiveConf, HiveConf.ConfVars.HIVE_LIMIT_PUSHDOWN_MEMORY_USAGE) > 0) { transformations.add(new LimitPushdownOptimizer()); } if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_LIMIT)) { transformations.add(new OrderlessLimitPushDownOptimizer()); } - if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES)) { + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES)) { transformations.add(new StatsOptimizer()); } if (pctx.getContext().isExplainSkipExecution() && !isTezExecEngine) { @@ -226,11 +226,11 @@ public void initialize(HiveConf hiveConf) { transformations.add(new AnnotateWithOpTraits()); } - if (!HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVEFETCHTASKCONVERSION).equals("none")) { + if (!HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION).equals("none")) { transformations.add(new SimpleFetchOptimizer()); // must be called last } - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEFETCHTASKAGGR)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_FETCH_TASK_AGGR)) { transformations.add(new SimpleFetchAggregation()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SemiJoinReductionMerge.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SemiJoinReductionMerge.java index 1201418bcccc..d3764dcc2365 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SemiJoinReductionMerge.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SemiJoinReductionMerge.java @@ -399,10 +399,10 @@ private static GroupByOperator createGroupBy(SelectOperator selectOp, Operator stack, ReduceSinkOperator sink = (ReduceSinkOperator) nd; ReduceSinkDesc desc = sink.getConf(); - long bytesPerReducer = context.conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER); - int maxReducers = context.conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); - int constantReducers = context.conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); + long bytesPerReducer = context.conf.getLongVar(HiveConf.ConfVars.BYTES_PER_REDUCER); + int maxReducers = context.conf.getIntVar(HiveConf.ConfVars.MAX_REDUCERS); + int constantReducers = context.conf.getIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS); if (context.visitedReduceSinks.contains(sink)) { // skip walking the children diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java index 83f6d9e7226a..fb1f6a1c7952 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java @@ -131,7 +131,7 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { private FetchTask optimize(ParseContext pctx, String alias, TableScanOperator source) throws Exception { String mode = HiveConf.getVar( - pctx.getConf(), HiveConf.ConfVars.HIVEFETCHTASKCONVERSION); + pctx.getConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION); boolean aggressive = "more".equals(mode); final int limit = pctx.getQueryProperties().getOuterQueryLimit(); @@ -144,7 +144,7 @@ private FetchTask optimize(ParseContext pctx, String alias, TableScanOperator so FetchWork fetchWork = fetch.convertToWork(); FetchTask fetchTask = (FetchTask) TaskFactory.get(fetchWork); fetchTask.setCachingEnabled(HiveConf.getBoolVar(pctx.getConf(), - HiveConf.ConfVars.HIVEFETCHTASKCACHING)); + HiveConf.ConfVars.HIVE_FETCH_TASK_CACHING)); fetchWork.setSink(fetch.completed(pctx, fetchWork)); fetchWork.setSource(source); fetchWork.setLimit(limit); @@ -154,7 +154,7 @@ private FetchTask optimize(ParseContext pctx, String alias, TableScanOperator so } private boolean checkThreshold(FetchData data, int limit, ParseContext pctx) throws Exception { - boolean cachingEnabled = HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVEFETCHTASKCACHING); + boolean cachingEnabled = HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CACHING); if (!cachingEnabled) { if (limit > 0) { if (data.hasOnlyPruningFilter()) { @@ -176,7 +176,7 @@ private boolean checkThreshold(FetchData data, int limit, ParseContext pctx) thr } // if caching is enabled we apply the treshold in all cases long threshold = HiveConf.getLongVar(pctx.getConf(), - HiveConf.ConfVars.HIVEFETCHTASKCONVERSIONTHRESHOLD); + HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION_THRESHOLD); if (threshold < 0) { return true; } @@ -209,7 +209,7 @@ private FetchData checkTree(boolean aggressive, ParseContext pctx, String alias, } boolean bypassFilter = false; - if (HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVEOPTPPD)) { + if (HiveConf.getBoolVar(pctx.getConf(), HiveConf.ConfVars.HIVE_OPT_PPD)) { ExprNodeDesc pruner = pctx.getOpToPartPruner().get(ts); if (PartitionPruner.onlyContainsPartnCols(table, pruner)) { bypassFilter = !pctx.getPrunedPartitions(alias, ts).hasUnknownPartitions(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java index b57ddd8e6c4e..cebb937c1209 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java @@ -836,7 +836,7 @@ private ArrayList getPositionsToExprNodes(List pos, private boolean shouldDo(List partitionPos, Operator fsParent) { int threshold = HiveConf.getIntVar(this.parseCtx.getConf(), - HiveConf.ConfVars.HIVEOPTSORTDYNAMICPARTITIONTHRESHOLD); + HiveConf.ConfVars.HIVE_OPT_SORT_DYNAMIC_PARTITION_THRESHOLD); long MAX_WRITERS = -1; switch (threshold) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java index cbfb749fc628..e8cec2fceefc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java @@ -55,7 +55,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // and sort merge bucketed mapjoin cannot be performed if (!convert && pGraphContext.getConf().getBoolVar( - HiveConf.ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN)) { + HiveConf.ConfVars.HIVE_ENFORCE_SORT_MERGE_BUCKET_MAPJOIN)) { throw new SemanticException(ErrorMsg.SORTMERGE_MAPJOIN_FAILED.getMsg()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java index 5b6570a1bad2..7e7c1489cc1c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java @@ -82,7 +82,7 @@ private RelMetadataProvider init(HiveConf hiveConf, List>(); - trustScript = pctx.getConf().getBoolVar(HIVESCRIPTOPERATORTRUST); - minReducer = pctx.getConf().getIntVar(HIVEOPTREDUCEDEDUPLICATIONMINREDUCER); - isMapAggr = pctx.getConf().getBoolVar(HIVEMAPSIDEAGGREGATE); + trustScript = pctx.getConf().getBoolVar(HIVE_SCRIPT_OPERATOR_TRUST); + minReducer = pctx.getConf().getIntVar(HIVE_OPT_REDUCE_DEDUPLICATION_MIN_REDUCER); + isMapAggr = pctx.getConf().getBoolVar(HIVE_MAPSIDE_AGGREGATE); this.pctx = pctx; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java index 43870562a3d7..19a2295a0c10 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java @@ -178,7 +178,7 @@ private void findPossibleAutoConvertedJoinOperators() throws SemanticException { } long ThresholdOfSmallTblSizeSum = HiveConf.getLongVar(pCtx.getConf(), - HiveConf.ConfVars.HIVESMALLTABLESFILESIZE); + HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE); for (int i = 0; i < numAliases; i++) { // this table cannot be big table if (!bigTableCandidates.contains(i)) { @@ -212,7 +212,7 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { pCtx = pctx; - if (HiveConf.getBoolVar(pCtx.getConf(),HiveConf.ConfVars.HIVECONVERTJOIN)) { + if (HiveConf.getBoolVar(pCtx.getConf(),HiveConf.ConfVars.HIVE_CONVERT_JOIN)) { findPossibleAutoConvertedJoinOperators(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java index dc5c97d5a32f..51af5b7cd161 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hive.ql.optimizer.correlation; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVECONVERTJOIN; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_CONVERT_JOIN; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONALTASK; import java.util.ArrayList; import java.util.LinkedHashMap; @@ -73,10 +73,10 @@ public ParseContext transform(ParseContext pctx) throws SemanticException { ReduceSinkDeduplicateProcCtx cppCtx = new ReduceSinkDeduplicateProcCtx(pGraphContext); // for auto convert map-joins, it not safe to dedup in here (todo) - boolean mergeJoins = !pctx.getConf().getBoolVar(HIVECONVERTJOIN) && - !pctx.getConf().getBoolVar(HIVECONVERTJOINNOCONDITIONALTASK) && + boolean mergeJoins = !pctx.getConf().getBoolVar(HIVE_CONVERT_JOIN) && + !pctx.getConf().getBoolVar(HIVE_CONVERT_JOIN_NOCONDITIONALTASK) && !pctx.getConf().getBoolVar(ConfVars.HIVE_CONVERT_JOIN_BUCKET_MAPJOIN_TEZ) && - !pctx.getConf().getBoolVar(ConfVars.HIVEDYNAMICPARTITIONHASHJOIN); + !pctx.getConf().getBoolVar(ConfVars.HIVE_DYNAMIC_PARTITION_HASHJOIN); // If multiple rules can be matched with same cost, last rule will be choosen as a processor // see DefaultRuleDispatcher#dispatch() @@ -329,7 +329,7 @@ public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, start, ReduceSinkOperator.class, dedupCtx.trustScript()); if (pRS != null && ReduceSinkDeDuplicationUtils .merge(dedupCtx.getPctx().getConf(), cRS, pRS, dedupCtx.minReducer())) { - if (dedupCtx.getPctx().getConf().getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + if (dedupCtx.getPctx().getConf().getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { return false; } CorrelationUtilities.removeReduceSinkForGroupBy(cRS, cGBY, dedupCtx.getPctx(), dedupCtx); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java index 9327c643d768..27e56dfead17 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hive.ql.optimizer.correlation; -import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; import java.util.Map.Entry; @@ -33,14 +32,12 @@ import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.optimizer.correlation.ReduceSinkDeDuplication.ReduceSinkDeduplicateProcCtx; import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.ColStatistics; import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import org.apache.hadoop.hive.ql.plan.Statistics; import org.apache.hadoop.hive.ql.plan.TableDesc; import com.google.common.collect.ImmutableList; @@ -149,7 +146,7 @@ public static boolean merge(HiveConf hiveConf, ReduceSinkOperator cRS, ReduceSin // child RS but Sorting order of the child RS is more specific than // that of the parent RS. throw new SemanticException("Sorting columns and order don't match. " + - "Try set " + HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION + "=false;"); + "Try set " + HiveConf.ConfVars.HIVE_OPT_REDUCE_DEDUPLICATION + "=false;"); } pRS.getConf().setOrder(cRS.getConf().getOrder()); pRS.getConf().setNullOrder(cRS.getConf().getNullOrder()); @@ -210,7 +207,7 @@ private static long estimateReducers(HiveConf conf, ReduceSinkOperator rs) { if (rs.getConf().getNumReducers() > 0) { return rs.getConf().getNumReducers(); } - int constantReducers = conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); + int constantReducers = conf.getIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS); if (constantReducers > 0) { return constantReducers; } @@ -221,8 +218,8 @@ private static long estimateReducers(HiveConf conf, ReduceSinkOperator rs) { inputTotalBytes = StatsUtils.safeAdd(inputTotalBytes, sibling.getStatistics().getDataSize()); } } - int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); - long bytesPerReducer = conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER); + int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAX_REDUCERS); + long bytesPerReducer = conf.getLongVar(HiveConf.ConfVars.BYTES_PER_REDUCER); return Utilities.estimateReducers(inputTotalBytes, bytesPerReducer, maxReducers, false); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java index 28e8d12dcead..86dcb10b557c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java @@ -143,7 +143,7 @@ private long calculateLocalTableTotalSize(MapredLocalWork localWork) { * Check if the total size of local tables will be under * the limit after we merge localWork1 and localWork2. * The limit of the total size of local tables is defined by - * HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD. + * HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD. * @param conf * @param localWorks * @return @@ -152,7 +152,7 @@ private boolean isLocalTableTotalSizeUnderLimitAfterMerge( Configuration conf, MapredLocalWork... localWorks) { final long localTableTotalSizeLimit = HiveConf.getLongVar(conf, - HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); + HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD); long localTableTotalSize = 0; for (int i = 0; i < localWorks.length; i++) { final long localWorkTableTotalSize = calculateLocalTableTotalSize(localWorks[i]); @@ -166,7 +166,7 @@ private boolean isLocalTableTotalSizeUnderLimitAfterMerge( if (localTableTotalSize > localTableTotalSizeLimit) { // The total size of local tables after we merge localWorks // is larger than the limit set by - // HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD. + // HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD. return false; } @@ -431,12 +431,12 @@ public Task processCurrentTask(MapRedTask currTask, // If sizes of at least n-1 tables in a n-way join is known, and their sum is smaller than // the threshold size, convert the join into map-join and don't create a conditional task boolean convertJoinMapJoin = HiveConf.getBoolVar(conf, - HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK); + HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONALTASK); int bigTablePosition = -1; if (convertJoinMapJoin) { // This is the threshold that the user has specified to fit in mapjoin long mapJoinSize = HiveConf.getLongVar(conf, - HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); + HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD); Long bigTableSize = null; Set aliases = aliasToWork.keySet(); @@ -480,7 +480,7 @@ public Task processCurrentTask(MapRedTask currTask, } long ThresholdOfSmallTblSizeSum = HiveConf.getLongVar(conf, - HiveConf.ConfVars.HIVESMALLTABLESFILESIZE); + HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE); for (int pos = 0; pos < joinOp.getNumParent(); pos++) { // this table cannot be big table if (!bigTableCandidates.contains(pos)) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java index 9c9dac07a671..d846428f78ae 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java @@ -147,7 +147,7 @@ public static void processSkewJoin(JoinOperator joinOp, joinDescriptor.setBigKeysDirMap(bigKeysDirMap); joinDescriptor.setSmallKeysDirMap(smallKeysDirMap); joinDescriptor.setSkewKeyDefinition(HiveConf.getIntVar(parseCtx.getConf(), - HiveConf.ConfVars.HIVESKEWJOINKEY)); + HiveConf.ConfVars.HIVE_SKEWJOIN_KEY)); HashMap> bigKeysDirToTaskMap = new HashMap>(); @@ -323,9 +323,9 @@ public static void processSkewJoin(JoinOperator joinOp, GenMRSkewJoinProcessor.class); newPlan.setNumMapTasks(HiveConf - .getIntVar(jc, HiveConf.ConfVars.HIVESKEWJOINMAPJOINNUMMAPTASK)); + .getIntVar(jc, HiveConf.ConfVars.HIVE_SKEWJOIN_MAPJOIN_NUM_MAP_TASK)); newPlan - .setMinSplitSize(HiveConf.getLongVar(jc, HiveConf.ConfVars.HIVESKEWJOINMAPJOINMINSPLIT)); + .setMinSplitSize(HiveConf.getLongVar(jc, HiveConf.ConfVars.HIVE_SKEWJOIN_MAPJOIN_MIN_SPLIT)); newPlan.setInputformat(HiveInputFormat.class.getName()); MapredWork w = new MapredWork(); @@ -366,7 +366,7 @@ public static void processSkewJoin(JoinOperator joinOp, public static boolean skewJoinEnabled(HiveConf conf, JoinOperator joinOp) { - if (conf != null && !conf.getBoolVar(HiveConf.ConfVars.HIVESKEWJOIN)) { + if (conf != null && !conf.getBoolVar(HiveConf.ConfVars.HIVE_SKEW_JOIN)) { return false; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java index 666da19e5813..2fef7f570b15 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java @@ -173,7 +173,7 @@ private void adjustAutoParallelism(BaseWork work) { clusterState.initClusterInfo(); final int targetCount; final int executorCount; - final int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); + final int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAX_REDUCERS); if (!clusterState.hasClusterInfo()) { LOG.warn("Cannot determine LLAP cluster information"); executorCount = executorsPerNode; // assume 1 node @@ -190,7 +190,7 @@ private void adjustAutoParallelism(BaseWork work) { if (newMin < reduceWork.getMaxReduceTasks()) { reduceWork.setMinReduceTasks(newMin); reduceWork.getEdgePropRef().setAutoReduce(conf, true, newMin, - reduceWork.getMaxReduceTasks(), conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER), + reduceWork.getMaxReduceTasks(), conf.getLongVar(HiveConf.ConfVars.BYTES_PER_REDUCER), reduceWork.getMinSrcFraction(), reduceWork.getMaxSrcFraction()); } else { reduceWork.setAutoReduceParallelism(false); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java index ca840d7f4eb4..9d8985fe62ec 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java @@ -96,7 +96,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. context.setFollowedByGroupBy(true); GroupByOperator groupByOp = (GroupByOperator) nd; float groupByMemoryUsage = context.getParseCtx().getConf().getFloatVar( - HiveConf.ConfVars.HIVEMAPJOINFOLLOWEDBYMAPAGGRHASHMEMORY); + HiveConf.ConfVars.HIVE_MAPJOIN_FOLLOWEDBY_MAP_AGGR_HASH_MEMORY); groupByOp.getConf().setGroupByMemoryUsage(groupByMemoryUsage); return null; } @@ -130,10 +130,10 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. float hashtableMemoryUsage; if (context.isFollowedByGroupBy()) { hashtableMemoryUsage = conf.getFloatVar( - HiveConf.ConfVars.HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE); + HiveConf.ConfVars.HIVE_HASHTABLE_FOLLOWBY_GBY_MAX_MEMORY_USAGE); } else { hashtableMemoryUsage = conf.getFloatVar( - HiveConf.ConfVars.HIVEHASHTABLEMAXMEMORYUSAGE); + HiveConf.ConfVars.HIVE_HASHTABLE_MAX_MEMORY_USAGE); } mapJoinDesc.setHashTableMemoryUsage(hashtableMemoryUsage); LOG.info("Setting max memory usage to " + hashtableMemoryUsage + " for table sink " @@ -148,7 +148,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, Object.. // todo: support tez/vectorization boolean useNontaged = conf.getBoolVar( - HiveConf.ConfVars.HIVECONVERTJOINUSENONSTAGED) && + HiveConf.ConfVars.HIVE_CONVERT_JOIN_USE_NONSTAGED) && conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("mr") && !conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java index 8e51417f12c6..14f322e55a95 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java @@ -73,9 +73,9 @@ public class MemoryCalculator implements SemanticDispatcher { public MemoryCalculator(PhysicalContext pctx) { this.pctx = pctx; - this.totalAvailableMemory = HiveConf.getLongVar(pctx.conf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); - this.minimumHashTableSize = HiveConf.getIntVar(pctx.conf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINNUMPARTITIONS) - * HiveConf.getIntVar(pctx.conf, HiveConf.ConfVars.HIVEHYBRIDGRACEHASHJOINMINWBSIZE); + this.totalAvailableMemory = HiveConf.getLongVar(pctx.conf, HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD); + this.minimumHashTableSize = HiveConf.getIntVar(pctx.conf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MIN_NUM_PARTITIONS) + * HiveConf.getIntVar(pctx.conf, HiveConf.ConfVars.HIVE_HYBRIDGRACE_HASHJOIN_MIN_WB_SIZE); this.inflationFactor = HiveConf.getFloatVar(pctx.conf, HiveConf.ConfVars.HIVE_HASH_TABLE_INFLATION_FACTOR); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java index 8903eb738107..0ed3b3589381 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; /** @@ -46,7 +45,7 @@ public PhysicalOptimizer(PhysicalContext pctx, HiveConf hiveConf) { */ private void initialize(HiveConf hiveConf) { resolvers = new ArrayList(); - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN)) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN)) { resolvers.add(new CommonJoinResolver()); // The joins have been automatically converted to map-joins. @@ -56,18 +55,18 @@ private void initialize(HiveConf hiveConf) { resolvers.add(new SortMergeJoinResolver()); } } - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVESKEWJOIN)) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SKEW_JOIN)) { resolvers.add(new SkewJoinResolver()); } resolvers.add(new MapJoinResolver()); - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES)) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_METADATA_ONLY_QUERIES)) { resolvers.add(new MetadataOnlyOptimizer()); } - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVENULLSCANOPTIMIZE)) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_NULL_SCAN_OPTIMIZE)) { resolvers.add(new NullScanOptimizer()); } - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVESAMPLINGFORORDERBY)) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SAMPLING_FOR_ORDERBY)) { resolvers.add(new SamplingOptimizer()); } @@ -91,7 +90,7 @@ private void initialize(HiveConf hiveConf) { "enable")) { resolvers.add(new Vectorizer()); } - if (!"none".equalsIgnoreCase(hiveConf.getVar(HiveConf.ConfVars.HIVESTAGEIDREARRANGE))) { + if (!"none".equalsIgnoreCase(hiveConf.getVar(HiveConf.ConfVars.HIVE_STAGE_ID_REARRANGE))) { resolvers.add(new StageIDsRearranger()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java index 54373effe23f..d67c5d720388 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java @@ -274,7 +274,7 @@ public Task processCurrentTask(MapRedTask currTask, pathToAliases, aliasToSize); long ThresholdOfSmallTblSizeSum = HiveConf.getLongVar(conf, - HiveConf.ConfVars.HIVESMALLTABLESFILESIZE); + HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE); for (int bigTablePosition = 0; bigTablePosition < numAliases; bigTablePosition++) { // this table cannot be big table diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/StageIDsRearranger.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/StageIDsRearranger.java index 02bbf6a99511..51a79650e602 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/StageIDsRearranger.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/StageIDsRearranger.java @@ -52,7 +52,7 @@ public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { } private static List getExplainOrder(PhysicalContext pctx) { - List tasks = getExplainOrder(pctx.getRootTasks(), pctx.getConf().getVar(HiveConf.ConfVars.HIVESTAGEIDREARRANGE)); + List tasks = getExplainOrder(pctx.getRootTasks(), pctx.getConf().getVar(HiveConf.ConfVars.HIVE_STAGE_ID_REARRANGE)); if (pctx.getFetchTask() != null) { tasks.add(pctx.getFetchTask()); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index ce2e31d58800..a33e5627b4bd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -3912,7 +3912,7 @@ private boolean canSpecializeMapJoin(Operator op, MapJoi vectorMapJoinInfo.setBigTableFilterExpressions(bigTableFilterExpressions); boolean useOptimizedTable = - HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE); + HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_MAPJOIN_USE_OPTIMIZED_TABLE); // Remember the condition variables for EXPLAIN regardless of whether we specialize or not. vectorDesc.setVectorMapJoinInfo(vectorMapJoinInfo); @@ -5330,7 +5330,7 @@ public Operator validateAndVectorizeOperator(Operator partitio List partNames = Hive.get().getPartitionNames( tab.getDbName(), tab.getTableName(), (short) -1); - String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); List partCols = extractPartColNames(tab); List partColTypeInfos = extractPartColTypes(tab); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java index d7744587e689..416443479fa3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java @@ -1528,7 +1528,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, AnnotateStatsProcCtx aspCtx = (AnnotateStatsProcCtx) procCtx; HiveConf conf = aspCtx.getConf(); - long maxSplitSize = HiveConf.getLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE); + long maxSplitSize = HiveConf.getLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE); List aggDesc = gop.getConf().getAggregators(); Map colExprMap = gop.getColumnExprMap(); RowSchema rs = gop.getSchema(); @@ -1577,7 +1577,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // be updated to bytes per reducer (1GB default) if (top == null) { inputSize = parentStats.getDataSize(); - maxSplitSize = HiveConf.getLongVar(conf, HiveConf.ConfVars.BYTESPERREDUCER); + maxSplitSize = HiveConf.getLongVar(conf, HiveConf.ConfVars.BYTES_PER_REDUCER); } else { inputSize = top.getConf().getStatistics().getDataSize(); } @@ -1875,7 +1875,7 @@ private long getParentNumRows(GroupByOperator op, List gbyKeys, Hi /** * This method does not take into account many configs used at runtime to - * disable hash aggregation like HIVEMAPAGGRHASHMINREDUCTION. This method + * disable hash aggregation like HIVE_MAP_AGGR_HASH_MIN_REDUCTION. This method * roughly estimates the number of rows and size of each row to see if it * can fit in hashtable for aggregation. * @param gop - group by operator @@ -1891,8 +1891,8 @@ private boolean checkMapSideAggregation(GroupByOperator gop, GroupByDesc.Mode mode = desc.getMode(); if (mode.equals(GroupByDesc.Mode.HASH)) { - float hashAggMem = conf.getFloatVar(HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); - float hashAggMaxThreshold = conf.getFloatVar(HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + float hashAggMem = conf.getFloatVar(HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); + float hashAggMaxThreshold = conf.getFloatVar(HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); // get available map memory in bytes long totalMemory = DagUtils.getContainerResource(conf).getMemorySize() * 1024L * 1024L; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java index bc36832b6d1a..773cafd01c6c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java @@ -1197,9 +1197,9 @@ public TableSpec(Hive db, HiveConf conf, ASTNode ast, boolean allowDynamicPartit try { // get table metadata tableName = HiveTableName.withNoDefault(getUnescapedName((ASTNode)ast.getChild(0))); - boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODE); + boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE); if (testMode) { - tableName = TableName.fromString(String.join("", conf.getVar(HiveConf.ConfVars.HIVETESTMODEPREFIX), + tableName = TableName.fromString(String.join("", conf.getVar(HiveConf.ConfVars.HIVE_TEST_MODE_PREFIX), tableName.getTable()), tableName.getCat(), tableName.getDb()); // not that elegant, but hard to refactor } if (ast.getToken().getType() != HiveParser.TOK_CREATETABLE && @@ -1274,7 +1274,7 @@ public TableSpec(Hive db, HiveConf conf, ASTNode ast, boolean allowDynamicPartit numStaPart = parts.size() - numDynParts; } if (numStaPart == 0 && - conf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { + conf.getVar(HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE).equalsIgnoreCase("strict")) { throw new SemanticException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg()); } @@ -1613,7 +1613,7 @@ private static boolean getPartExprNodeDesc(ASTNode astNode, HiveConf conf, } TypeCheckCtx typeCheckCtx = new TypeCheckCtx(null); - String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULT_PARTITION_NAME); boolean result = true; for (Node childNode : astNode.getChildren()) { ASTNode childASTNode = (ASTNode)childNode; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java index b105de8174ee..c23e94e6e165 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java @@ -506,9 +506,9 @@ public static RelOptPlanner createPlanner(HiveConf conf) { private static RelOptPlanner createPlanner( HiveConf conf, StatsSource statsSource, boolean isExplainPlan) { final Double maxSplitSize = (double) HiveConf.getLongVar( - conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE); + conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE); final Double maxMemory = (double) HiveConf.getLongVar( - conf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); + conf, HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD); HiveAlgorithmsConf algorithmsConf = new HiveAlgorithmsConf(maxSplitSize, maxMemory); HiveRulesRegistry registry = new HiveRulesRegistry(); Properties calciteConfigProperties = new Properties(); @@ -1745,7 +1745,7 @@ protected RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataPr PerfLogger perfLogger = SessionState.getPerfLogger(); final int maxCNFNodeCount = conf.getIntVar(HiveConf.ConfVars.HIVE_CBO_CNF_NODES_LIMIT); - final int minNumORClauses = conf.getIntVar(HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZERMIN); + final int minNumORClauses = conf.getIntVar(HiveConf.ConfVars.HIVE_POINT_LOOKUP_OPTIMIZER_MIN); final boolean allowDisjunctivePredicates = conf.getBoolVar(ConfVars.HIVE_JOIN_DISJ_TRANSITIVE_PREDICATES_PUSHDOWN); final HepProgramBuilder program = new HepProgramBuilder(); @@ -1790,7 +1790,7 @@ protected RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataPr } // Run this optimization early, since it is expanding the operator pipeline. if (!conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("mr") && - conf.getBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEDISTINCTREWRITE)) { + conf.getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_DISTINCT_REWRITE)) { // Its not clear, if this rewrite is always performant on MR, since extra map phase // introduced for 2nd MR job may offset gains of this multi-stage aggregation. // We need a cost model for MR to enable this on MR. @@ -1809,7 +1809,7 @@ protected RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataPr // 3. Run exhaustive PPD, add not null filters, transitive inference, // constant propagation, constant folding List rules = Lists.newArrayList(); - if (conf.getBoolVar(HiveConf.ConfVars.HIVEOPTPPD_WINDOWING)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_OPT_PPD_WINDOWING)) { rules.add(HiveFilterProjectTransposeRule.DETERMINISTIC_WINDOWING); } else { rules.add(HiveFilterProjectTransposeRule.DETERMINISTIC); @@ -1836,7 +1836,7 @@ protected RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataPr rules.add(HiveReduceExpressionsRule.SEMIJOIN_INSTANCE); rules.add(HiveAggregateReduceFunctionsRule.INSTANCE); rules.add(HiveAggregateReduceRule.INSTANCE); - if (conf.getBoolVar(HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZER)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_POINT_LOOKUP_OPTIMIZER)) { rules.add(new HivePointLookupOptimizerRule.FilterCondition(minNumORClauses)); rules.add(new HivePointLookupOptimizerRule.JoinCondition(minNumORClauses)); rules.add(new HivePointLookupOptimizerRule.ProjectionExpressions(minNumORClauses)); @@ -2156,7 +2156,7 @@ private RelNode applyJoinOrderingTransform(RelNode basePlan, RelMetadataProvider rules.add(HiveJoinProjectTransposeRule.RIGHT_PROJECT_BTW_JOIN); rules.add(HiveProjectMergeRule.INSTANCE); if (profilesCBO.contains(ExtendedCBOProfile.REFERENTIAL_CONSTRAINTS)) { - rules.add(conf.getBoolVar(HiveConf.ConfVars.HIVEOPTPPD_WINDOWING) ? + rules.add(conf.getBoolVar(HiveConf.ConfVars.HIVE_OPT_PPD_WINDOWING) ? HiveFilterProjectTransposeRule.DETERMINISTIC_WINDOWING_ON_NON_FILTERING_JOIN : HiveFilterProjectTransposeRule.DETERMINISTIC_ON_NON_FILTERING_JOIN); rules.add(HiveFilterJoinRule.FILTER_ON_NON_FILTERING_JOIN); @@ -2899,7 +2899,7 @@ private RelNode genTableLogicalPlan(String tableAlias, QB qb) throws SemanticExc // 2. if returnpath is on and hivetestmode is on bail if (qb.getParseInfo().getTabSample(tableAlias) != null || getNameToSplitSampleMap().containsKey(tableAlias) - || (conf.getBoolVar(HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP)) && (conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODE)) ) { + || (conf.getBoolVar(HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP)) && (conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE)) ) { String msg = String.format("Table Sample specified for %s." + " Currently we don't support Table Sample clauses in CBO," + " turn off cbo for queries on tableSamples.", tableAlias); @@ -3663,16 +3663,16 @@ private RelNode genGBLogicalPlan(QB qb, RelNode srcRel) throws SemanticException || !qbp.getDestGroupingSets().isEmpty() || !qbp.getDestCubes().isEmpty()); // 2. Sanity check - if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW) && qbp.getDistinctFuncExprsForClause(destClauseName).size() > 1) { throw new SemanticException(ErrorMsg.UNSUPPORTED_MULTIPLE_DISTINCTS.getMsg()); } if (cubeRollupGrpSetPresent) { - if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE)) { + if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MAPSIDE_AGGREGATE)) { throw new SemanticException(ErrorMsg.HIVE_GROUPING_SETS_AGGR_NOMAPAGGR.getMsg()); } - if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { if (qbp.getDestGroupingSets().size() > conf .getIntVar(HiveConf.ConfVars.HIVE_NEW_JOB_GROUPING_SET_CARDINALITY)) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java index 029ea483a7fe..535d8d855ed3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java @@ -205,9 +205,9 @@ public void analyzeInternal(ASTNode root) throws SemanticException { this.prepareQuery = false; // reset config - String queryId = this.conf.getVar(HiveConf.ConfVars.HIVEQUERYID); + String queryId = this.conf.getVar(HiveConf.ConfVars.HIVE_QUERY_ID); this.conf.syncFromConf(cachedPlan.getQueryState().getConf()); - this.conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId); + this.conf.setVar(HiveConf.ConfVars.HIVE_QUERY_ID, queryId); // set rest of the params this.inputs = cachedPlan.getInputs(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java index 4add301fb0ae..91cda5591981 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java @@ -259,8 +259,8 @@ private EximUtil() { */ public static URI getValidatedURI(HiveConf conf, String dcPath) throws SemanticException { try { - boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODE) - || conf.getBoolVar(HiveConf.ConfVars.HIVEEXIMTESTMODE); + boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE) + || conf.getBoolVar(HiveConf.ConfVars.HIVE_EXIM_TEST_MODE); URI uri = new Path(dcPath).toUri(); FileSystem fs = FileSystem.get(uri, conf); // Get scheme from FileSystem @@ -316,8 +316,8 @@ static void validateTable(org.apache.hadoop.hive.ql.metadata.Table table) throws public static String relativeToAbsolutePath(HiveConf conf, String location) throws SemanticException { try { - boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODE) - || conf.getBoolVar(HiveConf.ConfVars.HIVEEXIMTESTMODE);; + boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE) + || conf.getBoolVar(HiveConf.ConfVars.HIVE_EXIM_TEST_MODE);; if (testMode) { URI uri = new Path(location).toUri(); FileSystem fs = FileSystem.get(uri, conf); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java index baa31c224ea8..ddd42d1590af 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java @@ -221,7 +221,7 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { ctx.getCalcitePlan()); work.setAppendTaskType( - HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEEXPLAINDEPENDENCYAPPENDTASKTYPES)); + HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_EXPLAIN_DEPENDENCY_APPEND_TASK_TYPES)); ExplainTask explTask = (ExplainTask) TaskFactory.get(work); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java index 6042c0982149..6688a4256f7c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java @@ -173,7 +173,7 @@ public GenTezProcContext(HiveConf conf, ParseContext parseContext, this.inputs = inputs; this.outputs = outputs; this.currentTask = (TezTask) TaskFactory.get( - new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID), conf)); + new TezWork(conf.getVar(HiveConf.ConfVars.HIVE_QUERY_ID), conf)); this.leafOperatorToFollowingWork = new LinkedHashMap, BaseWork>(); this.linkOpWithWorkMap = new LinkedHashMap, Map>(); this.linkWorkWithReduceSinkMap = new LinkedHashMap>(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java index 1d2eab7e1449..a2512500e37b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.*; -import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType; @@ -96,7 +95,7 @@ public static ReduceWork createReduceWork( float maxPartitionFactor = context.conf.getFloatVar(HiveConf.ConfVars.TEZ_MAX_PARTITION_FACTOR); float minPartitionFactor = context.conf.getFloatVar(HiveConf.ConfVars.TEZ_MIN_PARTITION_FACTOR); - long bytesPerReducer = context.conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER); + long bytesPerReducer = context.conf.getLongVar(HiveConf.ConfVars.BYTES_PER_REDUCER); int defaultTinyBufferSize = context.conf.getIntVar(HiveConf.ConfVars.TEZ_SIMPLE_CUSTOM_EDGE_TINY_BUFFER_SIZE_MB); ReduceWork reduceWork = new ReduceWork(Utilities.REDUCENAME + context.nextSequenceNumber()); @@ -126,7 +125,7 @@ public static ReduceWork createReduceWork( if (isAutoReduceParallelism && reduceSink.getConf().getReducerTraits().contains(AUTOPARALLEL)) { // configured limit for reducers - final int maxReducers = context.conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); + final int maxReducers = context.conf.getIntVar(HiveConf.ConfVars.MAX_REDUCERS); // estimated number of reducers final int nReducers = reduceSink.getConf().getNumReducers(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java index 736e562c1afb..e112b55031a4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java @@ -372,7 +372,7 @@ public Object process(Node nd, Stack stack, if (context.leafOperatorToFollowingWork.containsKey(operator)) { BaseWork followingWork = context.leafOperatorToFollowingWork.get(operator); - long bytesPerReducer = context.conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER); + long bytesPerReducer = context.conf.getLongVar(HiveConf.ConfVars.BYTES_PER_REDUCER); LOG.debug("Second pass. Leaf operator: "+operator +" has common downstream work: "+followingWork); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java index 3a18b7af729b..ce036a2b63e8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java @@ -351,7 +351,7 @@ private void analyzeLoad(ASTNode ast) throws SemanticException { // for managed tables, make sure the file formats match if (TableType.MANAGED_TABLE.equals(ts.tableHandle.getTableType()) - && conf.getBoolVar(HiveConf.ConfVars.HIVECHECKFILEFORMAT)) { + && conf.getBoolVar(HiveConf.ConfVars.HIVE_CHECK_FILEFORMAT)) { ensureFileFormatsMatch(ts, files, fromURI); } inputs.add(toReadEntity(new Path(fromURI))); @@ -557,7 +557,7 @@ private void reparseAndSuperAnalyze(Table table, URI fromURI) throws SemanticExc // Step 3 : parse the query // Set dynamic partitioning to nonstrict so that queries do not need any partition // references. - HiveConf.setVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + HiveConf.setVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); // Parse the rewritten query string Context rewrittenCtx; rewrittenCtx = new Context(conf); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java index 2a3b60e52bb6..376ba8708ca1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java @@ -185,7 +185,7 @@ private static int getNumberOfReducers(MapredWork mrwork, HiveConf conf) { return mrwork.getReduceWork().getNumReduceTasks(); } - return conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); + return conf.getIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS); } @Override @@ -200,7 +200,7 @@ protected void decideExecMode(List> rootTasks, Context ctx, // user has told us to run in local mode or doesn't want auto-local mode if (ctx.isLocalOnlyExecutionMode() || - !conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) { + !conf.getBoolVar(HiveConf.ConfVars.LOCAL_MODE_AUTO)) { return; } @@ -230,11 +230,11 @@ public boolean accept(Path file) { // estimated Input = (num_limit * max_size_per_row) * (estimated_map + 2) // long sizePerRow = HiveConf.getLongVar(conf, - HiveConf.ConfVars.HIVELIMITMAXROWSIZE); + HiveConf.ConfVars.HIVE_LIMIT_MAX_ROW_SIZE); estimatedInput = (globalLimitCtx.getGlobalOffset() + globalLimitCtx.getGlobalLimit()) * sizePerRow; long minSplitSize = HiveConf.getLongVar(conf, - HiveConf.ConfVars.MAPREDMINSPLITSIZE); + HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE); long estimatedNumMap = inputSummary.getLength() / minSplitSize + 1; estimatedInput = estimatedInput * (estimatedNumMap + 1); } else { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java index db959192db71..972d7997673d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java @@ -571,7 +571,7 @@ private static CalcitePlanner getAnalyzer(HiveConf conf, Context ctx) throws Sem */ public static Map> getFullPartitionSpecs( CommonTree ast, Table table, Configuration conf, boolean canGroupExprs) throws SemanticException { - String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULT_PARTITION_NAME); Map colTypes = new HashMap<>(); List partitionKeys = table.getStorageHandler() != null && table.getStorageHandler().alwaysUnpartitioned() ? table.getStorageHandler().getPartitionKeys(table) : table.getPartitionKeys(); @@ -697,7 +697,7 @@ public static ReparseResult parseRewrittenQuery(Context ctx, // Set dynamic partitioning to nonstrict so that queries do not need any partition // references. // TODO: this may be a perf issue as it prevents the optimizer.. or not - HiveConf.setVar(ctx.getConf(), HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + HiveConf.setVar(ctx.getConf(), HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); // Disable LLAP IO wrapper; doesn't propagate extra ACID columns correctly. HiveConf.setBoolVar(ctx.getConf(), HiveConf.ConfVars.LLAP_IO_ROW_WRAPPER_ENABLED, false); // Parse the rewritten query string diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java index f4514d700c0f..62bd84f9f912 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java @@ -56,7 +56,7 @@ import java.util.Collections; import java.util.Objects; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVEQUERYID; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_QUERY_ID; import static org.apache.hadoop.hive.ql.exec.repl.ReplAck.LOAD_ACKNOWLEDGEMENT; import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DBNAME; import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_REPL_CONFIG; @@ -450,7 +450,7 @@ private void setConfigs(ASTNode node) throws SemanticException { for (Map.Entry config : replConfigs.entrySet()) { String key = config.getKey(); // don't set the query id in the config - if (key.equalsIgnoreCase(HIVEQUERYID.varname)) { + if (key.equalsIgnoreCase(HIVE_QUERY_ID.varname)) { String queryTag = config.getValue(); if (!StringUtils.isEmpty(queryTag)) { QueryState.setApplicationTag(conf, queryTag); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 769ab25e43e9..3c4dd24e70b6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -21,8 +21,8 @@ import static java.util.Objects.nonNull; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_TABLE; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.DYNAMICPARTITIONCONVERT; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVEARCHIVEENABLED; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.DYNAMIC_PARTITION_CONVERT; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ARCHIVE_ENABLED; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_DEFAULT_STORAGE_HANDLER; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_LOCATION; @@ -1206,7 +1206,7 @@ private String processTable(QB qb, ASTNode tabref) throws SemanticException { throw new SemanticException(generateErrorMessage((ASTNode) numerator, "Sampling percentage should be between 0 and 100")); } - int seedNum = conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM); + int seedNum = conf.getIntVar(ConfVars.HIVE_SAMPLE_RANDOM_NUM); sample = new SplitSample(percent, seedNum); } else if (type.getType() == HiveParser.TOK_ROWCOUNT) { sample = new SplitSample(Integer.parseInt(value)); @@ -1222,7 +1222,7 @@ private String processTable(QB qb, ASTNode tabref) throws SemanticException { } else if (last == 'g' || last == 'G') { length <<= 30; } - int seedNum = conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM); + int seedNum = conf.getIntVar(ConfVars.HIVE_SAMPLE_RANDOM_NUM); sample = new SplitSample(length, seedNum); } String alias_id = getAliasId(alias, qb); @@ -1270,8 +1270,8 @@ Map getNameToSplitSampleMap() { private void assertCombineInputFormat(Tree numerator, String message) throws SemanticException { String inputFormat = conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") ? - HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZINPUTFORMAT): - HiveConf.getVar(conf, HiveConf.ConfVars.HIVEINPUTFORMAT); + HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_TEZ_INPUT_FORMAT): + HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_INPUT_FORMAT); if (!inputFormat.equals(CombineHiveInputFormat.class.getName())) { throw new SemanticException(generateErrorMessage((ASTNode) numerator, message + " sampling is not supported in " + inputFormat)); @@ -1984,8 +1984,8 @@ boolean doPhase1(ASTNode ast, QB qb, Phase1Ctx ctx_1, PlannerContext plannerCtx, qb.getParseInfo().setIsAnalyzeCommand(true); qb.getParseInfo().setNoScanAnalyzeCommand(this.noscan); // Allow analyze the whole table and dynamic partitions - HiveConf.setVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + HiveConf.setVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); + HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); break; @@ -4298,7 +4298,7 @@ private Operator genScriptPlan(ASTNode trfm, QB qb, Operator input) TableDesc outInfo; TableDesc errInfo; TableDesc inInfo; - String defaultSerdeName = conf.getVar(HiveConf.ConfVars.HIVESCRIPTSERDE); + String defaultSerdeName = conf.getVar(HiveConf.ConfVars.HIVE_SCRIPT_SERDE); Class serde; try { @@ -4309,7 +4309,7 @@ private Operator genScriptPlan(ASTNode trfm, QB qb, Operator input) } int fieldSeparator = Utilities.tabCode; - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESCRIPTESCAPE)) { + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SCRIPT_ESCAPE)) { fieldSeparator = Utilities.ctrlaCode; } @@ -4373,7 +4373,7 @@ private Class getRecordReader(ASTNode node) String name; if (node.getChildCount() == 0) { - name = conf.getVar(HiveConf.ConfVars.HIVESCRIPTRECORDREADER); + name = conf.getVar(HiveConf.ConfVars.HIVE_SCRIPT_RECORD_READER); } else { name = unescapeSQLString(node.getChild(0).getText()); } @@ -4390,7 +4390,7 @@ private Class getDefaultRecordReader() throws SemanticException { String name; - name = conf.getVar(HiveConf.ConfVars.HIVESCRIPTRECORDREADER); + name = conf.getVar(HiveConf.ConfVars.HIVE_SCRIPT_RECORD_READER); try { return (Class) Class.forName(name, true, @@ -4405,7 +4405,7 @@ private Class getRecordWriter(ASTNode node) String name; if (node.getChildCount() == 0) { - name = conf.getVar(HiveConf.ConfVars.HIVESCRIPTRECORDWRITER); + name = conf.getVar(HiveConf.ConfVars.HIVE_SCRIPT_RECORD_WRITER); } else { name = unescapeSQLString(node.getChild(0).getText()); } @@ -5377,13 +5377,13 @@ private Operator genGroupByPlanGroupByOperator(QBParseInfo parseInfo, genericUDAFEvaluators.put(entry.getKey(), genericUDAFEvaluator); } } - float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf - .getFloatVar(conf, ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + .getFloatVar(conf, ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( new GroupByDesc(mode, outputColumnNames, groupByKeys, aggregations, @@ -5646,13 +5646,13 @@ private Operator genGroupByPlanGroupByOperator1(QBParseInfo parseInfo, groupByOutputRowResolver.putExpression(value, new ColumnInfo( field, udaf.returnType, "", false)); } - float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf - .getFloatVar(conf, ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + .getFloatVar(conf, ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); // Nothing special needs to be done for grouping sets if // this is the final group by operator, and multiple rows corresponding to the @@ -5827,13 +5827,13 @@ private Operator genGroupByPlanMapGroupByOperator(QB qb, genericUDAFEvaluators.put(entry.getKey(), genericUDAFEvaluator); } } - float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf - .getFloatVar(conf, ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + .getFloatVar(conf, ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( new GroupByDesc(GroupByDesc.Mode.HASH, outputColumnNames, groupByKeys, aggregations, false, groupByMemoryUsage, memoryThreshold, minReductionHashAggr, minReductionHashAggrLowerBound, @@ -6364,13 +6364,13 @@ private Operator genGroupByPlanGroupByOperator2MR(QBParseInfo parseInfo, groupByOutputRowResolver2.putExpression(value, new ColumnInfo( field, udaf.returnType, "", false)); } - float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf - .getFloatVar(conf, ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + .getFloatVar(conf, ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( new GroupByDesc(GroupByDesc.Mode.FINAL, outputColumnNames, groupByKeys, aggregations, @@ -7108,9 +7108,9 @@ private Operator genBucketingSortingDest(String dest, Operator input, QB qb, if (enforceBucketing) { Operation acidOp = AcidUtils.isFullAcidTable(dest_tab) ? getAcidType(table_desc.getOutputFileFormatClass(), dest, AcidUtils.isInsertOnlyTable(dest_tab)) : Operation.NOT_ACID; - int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); - if (conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS) > 0) { - maxReducers = conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); + int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAX_REDUCERS); + if (conf.getIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS) > 0) { + maxReducers = conf.getIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS); } int numBuckets = dest_tab.getNumBuckets(); if (numBuckets > maxReducers) { @@ -7856,8 +7856,8 @@ protected Operator genFileSinkPlan(String dest, QB qb, Operator input) cols = ct.cols; colTypes = ct.colTypes; dpCtx = new DynamicPartitionCtx(partitionColumnNames, - conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME), - conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTSPERNODE)); + conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME), + conf.getIntVar(HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS_PER_NODE)); qbm.setDPCtx(dest, dpCtx); isPartitioned = true; } else { @@ -8472,7 +8472,7 @@ private FileSinkDesc createFileSinkDesc(String dest, TableDesc table_desc, throw new IllegalStateException("Unexpected dest_type=" + dest_tab); } FileSinkDesc fileSinkDesc = new FileSinkDesc(queryTmpdir, table_desc, - conf.getBoolVar(HiveConf.ConfVars.COMPRESSRESULT), currentTableId, rsCtx.isMultiFileSpray(), + conf.getBoolVar(HiveConf.ConfVars.COMPRESS_RESULT), currentTableId, rsCtx.isMultiFileSpray(), canBeMerged, rsCtx.getNumFiles(), rsCtx.getTotalFiles(), rsCtx.getPartnCols(), dpCtx, dest_path, mmWriteId, isMmCtas, isInsertOverwrite, qb.getIsQuery(), qb.isCTAS() || qb.isMaterializedView(), isDirectInsert, acidOperation, @@ -8706,8 +8706,8 @@ private DynamicPartitionCtx checkDynPart(QB qb, QBMetaData qbm, Table dest_tab, if (dpCtx == null) { dest_tab.validatePartColumnNames(partSpec, false); dpCtx = new DynamicPartitionCtx(partSpec, - conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME), - conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTSPERNODE)); + conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME), + conf.getIntVar(HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS_PER_NODE)); qbm.setDPCtx(dest, dpCtx); } @@ -8720,7 +8720,7 @@ private DynamicPartitionCtx checkDynPart(QB qb, QBMetaData qbm, Table dest_tab, } private static void verifyDynamicPartitionEnabled(HiveConf conf, QB qb, String dest) throws SemanticException { - if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONING)) { // allow DP + if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING)) { // allow DP throw new SemanticException(generateErrorMessage(qb.getParseInfo().getDestForClause(dest), ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg())); } @@ -8757,8 +8757,8 @@ String fixCtasColumnName(String colName) { private void checkAcidConstraints() { /* LOG.info("Modifying config values for ACID write"); - conf.setBoolVar(ConfVars.HIVEOPTREDUCEDEDUPLICATION, true); - conf.setIntVar(ConfVars.HIVEOPTREDUCEDEDUPLICATIONMINREDUCER, 1); + conf.setBoolVar(ConfVars.HIVE_OPT_REDUCE_DEDUPLICATION, true); + conf.setIntVar(ConfVars.HIVE_OPT_REDUCE_DEDUPLICATION_MIN_REDUCER, 1); These props are now enabled elsewhere (see commit diffs). It would be better instead to throw if they are not set. For exmaple, if user has set hive.optimize.reducededuplication=false for some reason, we'll run a query contrary to what they wanted... But throwing now would be @@ -8872,7 +8872,7 @@ private Operator genConversionSelectOperator(String dest, QB qb, Operator input, // Check column number List tableFields = oi.getAllStructFieldRefs(); - boolean dynPart = HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONING); + boolean dynPart = HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING); List rowFields = opParseCtx.get(input).getRowResolver().getColumnInfos(); int inColumnCnt = rowFields.size(); int outColumnCnt = tableFields.size(); @@ -8943,7 +8943,7 @@ private Operator genConversionSelectOperator(String dest, QB qb, Operator input, new ExprNodeColumnDesc(inputTypeInfo, inputColumn.getInternalName(), "", true); // Cast input column to destination column type if necessary. - if (conf.getBoolVar(DYNAMICPARTITIONCONVERT)) { + if (conf.getBoolVar(DYNAMIC_PARTITION_CONVERT)) { if (parts != null && !parts.isEmpty()) { String destPartitionName = dpCtx.getDPColNames().get(dpColIdx); FieldSchema destPartitionFieldSchema = parts.stream() @@ -10102,13 +10102,13 @@ private Operator genMapGroupByForSemijoin(List fields, Operator inpu } // Generate group-by operator - float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float groupByMemoryUsage = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY); float memoryThreshold = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_MEMORY_THRESHOLD); float minReductionHashAggr = HiveConf - .getFloatVar(conf, HiveConf.ConfVars.HIVEMAPAGGRHASHMINREDUCTION); + .getFloatVar(conf, HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION); float minReductionHashAggrLowerBound = HiveConf - .getFloatVar(conf, ConfVars.HIVEMAPAGGRHASHMINREDUCTIONLOWERBOUND); + .getFloatVar(conf, ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_LOWER_BOUND); Operator op = putOpInsertMap(OperatorFactory.getAndMakeChild( new GroupByDesc(GroupByDesc.Mode.HASH, outputColumnNames, groupByKeys, aggregations, false, groupByMemoryUsage, memoryThreshold, minReductionHashAggr, minReductionHashAggrLowerBound, @@ -10212,7 +10212,7 @@ private List getMapSideJoinTables(QB qb) { ASTNode hint = (ASTNode) hints.getChild(pos); if (((ASTNode) hint.getChild(0)).getToken().getType() == HintParser.TOK_MAPJOIN) { // the user has specified to ignore mapjoin hint - if (!conf.getBoolVar(HiveConf.ConfVars.HIVEIGNOREMAPJOINHINT) + if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_IGNORE_MAPJOIN_HINT) && !conf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { ASTNode hintTblNames = (ASTNode) hint.getChild(1); int numCh = hintTblNames.getChildCount(); @@ -11364,7 +11364,7 @@ private Operator genBodyPlan(QB qb, Operator input, Map aliasT // If we can put multiple group bys in a single reducer, determine suitable groups of // expressions, otherwise treat all the expressions as a single group - if (conf.getBoolVar(HiveConf.ConfVars.HIVEMULTIGROUPBYSINGLEREDUCER)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_MULTI_GROUPBY_SINGLE_REDUCER)) { try { commonGroupByDestGroups = getCommonGroupByDestGroups(qb, inputs); } catch (SemanticException e) { @@ -11396,8 +11396,8 @@ private Operator genBodyPlan(QB qb, Operator input, Map aliasT if (commonGroupByDestGroup.size() == 1 || (qbp.getAggregationExprsForClause(firstDest).size() == 0 && getGroupByForClause(qbp, firstDest).size() == 0) || - conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) || - !conf.getBoolVar(HiveConf.ConfVars.HIVEMULTIGROUPBYSINGLEREDUCER)) { + conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW) || + !conf.getBoolVar(HiveConf.ConfVars.HIVE_MULTI_GROUPBY_SINGLE_REDUCER)) { // Go over all the destination tables for (String dest : commonGroupByDestGroup) { @@ -11415,7 +11415,7 @@ private Operator genBodyPlan(QB qb, Operator input, Map aliasT && (qbp.getSelForClause(dest).getToken().getType() != HiveParser.TOK_SELECTDI || qbp.getWindowingExprsForClause(dest) == null)) { // multiple distincts is not supported with skew in data - if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) && + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW) && qbp.getDistinctFuncExprsForClause(dest).size() > 1) { throw new SemanticException(ErrorMsg.UNSUPPORTED_MULTIPLE_DISTINCTS. getMsg()); @@ -11440,13 +11440,13 @@ private Operator genBodyPlan(QB qb, Operator input, Map aliasT qbp.setSelExprForClause(dest, genSelectDIAST(rr)); } } - if (conf.getBoolVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE)) { - if (!conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_MAPSIDE_AGGREGATE)) { + if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { curr = genGroupByPlanMapAggrNoSkew(dest, qb, curr); } else { curr = genGroupByPlanMapAggr2MR(dest, qb, curr); } - } else if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + } else if (conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { curr = genGroupByPlan2MR(dest, qb, curr); } else { curr = genGroupByPlan1MR(dest, qb, curr); @@ -11501,13 +11501,13 @@ private Operator genPostGroupByBodyPlan(Operator curr, String dest, QB qb, || getGroupByForClause(qbp, dest).size() > 0) && qbp.getSelForClause(dest).getToken().getType() == HiveParser.TOK_SELECTDI && qbp.getWindowingExprsForClause(dest) != null) { - if (conf.getBoolVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE)) { - if (!conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_MAPSIDE_AGGREGATE)) { + if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { curr = genGroupByPlanMapAggrNoSkew(dest, qb, curr); } else { curr = genGroupByPlanMapAggr2MR(dest, qb, curr); } - } else if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { + } else if (conf.getBoolVar(HiveConf.ConfVars.HIVE_GROUPBY_SKEW)) { curr = genGroupByPlan2MR(dest, qb, curr); } else { curr = genGroupByPlan1MR(dest, qb, curr); @@ -12086,13 +12086,13 @@ samplePredicate, true, new SampleDesc(ts.getNumerator(), new RowSchema(rwsch.getColumnInfos()), top); } } else { - boolean testMode = conf.getBoolVar(ConfVars.HIVETESTMODE); + boolean testMode = conf.getBoolVar(ConfVars.HIVE_TEST_MODE); if (testMode) { String tabName = tab.getTableName(); // has the user explicitly asked not to sample this table String unSampleTblList = conf - .getVar(ConfVars.HIVETESTMODENOSAMPLE); + .getVar(ConfVars.HIVE_TEST_MODE_NOSAMPLE); String[] unSampleTbls = unSampleTblList.split(","); boolean unsample = false; for (String unSampleTbl : unSampleTbls) { @@ -12121,7 +12121,7 @@ samplePredicate, true, new SampleDesc(ts.getNumerator(), LOG.info("No need for sample filter"); } else { // The table is not bucketed, add a dummy filter :: rand() - int freq = conf.getIntVar(ConfVars.HIVETESTMODESAMPLEFREQ); + int freq = conf.getIntVar(ConfVars.HIVE_TEST_MODE_SAMPLE_FREQ); TableSample tsSample = new TableSample(1, freq); tsSample.setInputPruning(false); qb.getParseInfo().setTabSample(alias, tsSample); @@ -13203,7 +13203,7 @@ void analyzeInternal(ASTNode ast, Supplier pcf) throws SemanticE // Add the transformation that computes the lineage information. Set postExecHooks = Sets.newHashSet(Splitter.on(",").trimResults() .omitEmptyStrings() - .split(Strings.nullToEmpty(HiveConf.getVar(conf, HiveConf.ConfVars.POSTEXECHOOKS)))); + .split(Strings.nullToEmpty(HiveConf.getVar(conf, HiveConf.ConfVars.POST_EXEC_HOOKS)))); if (postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.PostExecutePrinter") || postExecHooks.contains("org.apache.hadoop.hive.ql.hooks.LineageLogger") || postExecHooks.contains("org.apache.atlas.hive.hook.HiveHook")) { @@ -13667,7 +13667,7 @@ public void validate() throws SemanticException { LOG.debug("validated " + usedp.getName()); LOG.debug(usedp.getTable().getTableName()); - if (!AcidUtils.isTransactionalTable(tbl) && conf.getBoolVar(HIVEARCHIVEENABLED)) { + if (!AcidUtils.isTransactionalTable(tbl) && conf.getBoolVar(HIVE_ARCHIVE_ENABLED)) { // Do not check for ACID; it does not create new parts and this is expensive as hell. // TODO: add an API to get table name list for archived parts with a single call; // nobody uses this so we could skip the whole thing. @@ -14021,7 +14021,7 @@ ASTNode analyzeCreateTable( throw new SemanticException(ErrorMsg.CTAS_COLLST_COEXISTENCE.getMsg()); } if (partCols.size() != 0 || bucketCols.size() != 0) { - boolean dynPart = HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONING); + boolean dynPart = HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING); if (dynPart == false) { throw new SemanticException(ErrorMsg.CTAS_PARCOL_COEXISTENCE.getMsg()); } else { @@ -14338,7 +14338,7 @@ ASTNode analyzeCreateTable( if(lStats != null && lStats.length != 0) { // Don't throw an exception if the target location only contains the staging-dirs for (FileStatus lStat : lStats) { - if (!lStat.getPath().getName().startsWith(HiveConf.getVar(conf, HiveConf.ConfVars.STAGINGDIR))) { + if (!lStat.getPath().getName().startsWith(HiveConf.getVar(conf, HiveConf.ConfVars.STAGING_DIR))) { throw new SemanticException(ErrorMsg.CTAS_LOCATION_NONEMPTY.getMsg(location)); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java index b01be0740631..50f08ff1f4ed 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java @@ -180,9 +180,9 @@ public void processStorageFormat(String name) throws SemanticException { if (serde == null) { // RCFile supports a configurable SerDe if (name.equalsIgnoreCase(IOConstants.RCFILE)) { - serde = ensureClassExists(HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE)); + serde = ensureClassExists(HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_RCFILE_SERDE)); } else { - serde = ensureClassExists(HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTSERDE)); + serde = ensureClassExists(HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_SERDE)); } } } @@ -197,8 +197,8 @@ public void fillDefaultStorageFormat(boolean isExternal, boolean isMaterializedV HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_FILE_FORMAT); serde = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_SERDE); } else { - defaultFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT); - defaultManagedFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT); + defaultFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_FILEFORMAT); + defaultManagedFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_MANAGED_FILEFORMAT); } if (!isExternal && !"none".equals(defaultManagedFormat)) { @@ -211,7 +211,7 @@ public void fillDefaultStorageFormat(boolean isExternal, boolean isMaterializedV } else { processStorageFormat(defaultFormat); if (defaultFormat.equalsIgnoreCase(IOConstants.RCFILE)) { - serde = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE); + serde = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_RCFILE_SERDE); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java index 1095b13e1a3e..062a3fb4d344 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java @@ -261,7 +261,7 @@ public void compile(final ParseContext pCtx, // For the FetchTask, the limit optimization requires we fetch all the rows // in memory and count how many rows we get. It's not practical if the // limit factor is too big - int fetchLimit = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVELIMITOPTMAXFETCH); + int fetchLimit = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_LIMIT_OPT_MAX_FETCH); if (globalLimitCtx.isEnable() && globalLimitCtx.getGlobalLimit() > fetchLimit) { LOG.info("For FetchTask, LIMIT " + globalLimitCtx.getGlobalLimit() + " > " + fetchLimit + ". Doesn't qualify limit optimization."); @@ -764,12 +764,12 @@ protected abstract void generateTaskTree(List> rootTasks, ParseContext p protected void runDynPartitionSortOptimizations(ParseContext parseContext, HiveConf hConf) throws SemanticException { // run Sorted dynamic partition optimization - if(HiveConf.getBoolVar(hConf, HiveConf.ConfVars.DYNAMICPARTITIONING) && - HiveConf.getVar(hConf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equals("nonstrict") && - !HiveConf.getBoolVar(hConf, HiveConf.ConfVars.HIVEOPTLISTBUCKETING)) { + if(HiveConf.getBoolVar(hConf, HiveConf.ConfVars.DYNAMIC_PARTITIONING) && + HiveConf.getVar(hConf, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE).equals("nonstrict") && + !HiveConf.getBoolVar(hConf, HiveConf.ConfVars.HIVE_OPT_LIST_BUCKETING)) { new SortedDynPartitionOptimizer().transform(parseContext); - if(HiveConf.getBoolVar(hConf, HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION)) { + if(HiveConf.getBoolVar(hConf, HiveConf.ConfVars.HIVE_OPT_REDUCE_DEDUPLICATION)) { // Dynamic sort partition adds an extra RS therefore need to de-dup new ReduceSinkDeDuplication().transform(parseContext); // there is an issue with dedup logic wherein SELECT is created with wrong columns diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java index 57f1f491b315..6c17e9878893 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java @@ -195,7 +195,7 @@ protected void optimizeOperatorPlan(ParseContext pCtx) throws SemanticException } // need to run this; to get consistent filterop conditions(for operator tree matching) - if (procCtx.conf.getBoolVar(ConfVars.HIVEOPTCONSTANTPROPAGATION)) { + if (procCtx.conf.getBoolVar(ConfVars.HIVE_OPT_CONSTANT_PROPAGATION)) { new ConstantPropagate(ConstantPropagateOption.SHORTCUT).transform(procCtx.parseContext); } @@ -205,15 +205,15 @@ protected void optimizeOperatorPlan(ParseContext pCtx) throws SemanticException perfLogger.perfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Setup stats in the operator plan"); // run Sorted dynamic partition optimization - if(HiveConf.getBoolVar(procCtx.conf, HiveConf.ConfVars.DYNAMICPARTITIONING) && - HiveConf.getVar(procCtx.conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equals("nonstrict") && - !HiveConf.getBoolVar(procCtx.conf, HiveConf.ConfVars.HIVEOPTLISTBUCKETING)) { + if(HiveConf.getBoolVar(procCtx.conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING) && + HiveConf.getVar(procCtx.conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE).equals("nonstrict") && + !HiveConf.getBoolVar(procCtx.conf, HiveConf.ConfVars.HIVE_OPT_LIST_BUCKETING)) { perfLogger.perfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER); new SortedDynPartitionOptimizer().transform(procCtx.parseContext); perfLogger.perfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Sorted dynamic partition optimization"); } - if(HiveConf.getBoolVar(procCtx.conf, HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION)) { + if(HiveConf.getBoolVar(procCtx.conf, HiveConf.ConfVars.HIVE_OPT_REDUCE_DEDUPLICATION)) { perfLogger.perfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER); // Dynamic sort partition adds an extra RS therefore need to de-dup new ReduceSinkDeDuplication().transform(procCtx.parseContext); @@ -233,7 +233,7 @@ protected void optimizeOperatorPlan(ParseContext pCtx) throws SemanticException new BucketVersionPopulator().transform(pCtx); perfLogger.perfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER); - if(procCtx.conf.getBoolVar(ConfVars.HIVEOPTJOINREDUCEDEDUPLICATION)) { + if(procCtx.conf.getBoolVar(ConfVars.HIVE_OPT_JOIN_REDUCE_DEDUPLICATION)) { new ReduceSinkJoinDeDuplication().transform(procCtx.parseContext); } perfLogger.perfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Run reduce sink after join algorithm selection"); @@ -253,7 +253,7 @@ protected void optimizeOperatorPlan(ParseContext pCtx) throws SemanticException // of "and true and true" conditions. // Rather than run the full constant folding just need to shortcut AND/OR expressions // involving constant true/false values. - if(procCtx.conf.getBoolVar(ConfVars.HIVEOPTCONSTANTPROPAGATION)) { + if(procCtx.conf.getBoolVar(ConfVars.HIVE_OPT_CONSTANT_PROPAGATION)) { new ConstantPropagate(ConstantPropagateOption.SHORTCUT).transform(procCtx.parseContext); } @@ -481,7 +481,7 @@ private void runStatsDependentOptimizations(OptimizeTezProcContext procCtx) thro new SetReducerParallelism()); opRules.put(new RuleRegExp("Convert Join to Map-join", JoinOperator.getOperatorName() + "%"), new ConvertJoinMapJoin()); - if (procCtx.conf.getBoolVar(ConfVars.HIVEMAPAGGRHASHMINREDUCTIONSTATSADJUST)) { + if (procCtx.conf.getBoolVar(ConfVars.HIVE_MAP_AGGR_HASH_MIN_REDUCTION_STATS_ADJUST)) { opRules.put(new RuleRegExp("Set min reduction - GBy (Hash)", GroupByOperator.getOperatorName() + "%"), new SetHashGroupByMinReduction()); @@ -681,7 +681,7 @@ protected void generateTaskTree(List> rootTasks, ParseContext pCtx, for (BaseWork w : baseWorkList) { // work should be the smallest unit for memory allocation w.setReservedMemoryMB( - (int)(conf.getLongVar(ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD) / (1024 * 1024))); + (int)(conf.getLongVar(ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD) / (1024 * 1024))); } } @@ -788,13 +788,13 @@ protected void optimizeTaskPlan(List> rootTasks, ParseContext pCtx, PhysicalContext physicalCtx = new PhysicalContext(conf, pCtx, pCtx.getContext(), rootTasks, pCtx.getFetchTask()); - if (conf.getBoolVar(HiveConf.ConfVars.HIVENULLSCANOPTIMIZE)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_NULL_SCAN_OPTIMIZE)) { physicalCtx = new NullScanOptimizer().resolve(physicalCtx); } else { LOG.debug("Skipping null scan query optimization"); } - if (conf.getBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_METADATA_ONLY_QUERIES)) { physicalCtx = new MetadataOnlyOptimizer().resolve(physicalCtx); } else { LOG.debug("Skipping metadata only query optimization"); @@ -818,14 +818,14 @@ protected void optimizeTaskPlan(List> rootTasks, ParseContext pCtx, LOG.debug("Skipping vectorization"); } - if (!"none".equalsIgnoreCase(conf.getVar(HiveConf.ConfVars.HIVESTAGEIDREARRANGE))) { + if (!"none".equalsIgnoreCase(conf.getVar(HiveConf.ConfVars.HIVE_STAGE_ID_REARRANGE))) { physicalCtx = new StageIDsRearranger().resolve(physicalCtx); } else { LOG.debug("Skipping stage id rearranger"); } if ((conf.getBoolVar(HiveConf.ConfVars.HIVE_TEZ_ENABLE_MEMORY_MANAGER)) - && (conf.getBoolVar(HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN))) { + && (conf.getBoolVar(HiveConf.ConfVars.HIVE_USE_HYBRIDGRACE_HASHJOIN))) { physicalCtx = new MemoryDecider().resolve(physicalCtx); } @@ -1488,7 +1488,7 @@ private void removeSemiJoinEdgesForUnion(OptimizeTezProcContext procCtx) throws */ private void removeSemijoinsParallelToMapJoin(OptimizeTezProcContext procCtx) throws SemanticException { - if (!procCtx.conf.getBoolVar(ConfVars.HIVECONVERTJOIN)) { + if (!procCtx.conf.getBoolVar(ConfVars.HIVE_CONVERT_JOIN)) { // Not needed without mapjoin conversion return; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java index c1b9b27ff481..59f40b203bae 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverCommonJoin.java @@ -169,7 +169,7 @@ protected Task resolveMapJoinTask( Map aliasToKnownSize = ctx.getAliasToKnownSize(); Map, Set> taskToAliases = ctx.getTaskToAliases(); - long threshold = HiveConf.getLongVar(conf, HiveConf.ConfVars.HIVESMALLTABLESFILESIZE); + long threshold = HiveConf.getLongVar(conf, HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE); Long bigTableSize = null; Long smallTablesSize = null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java index 0e6816ae4056..c5aecaa9cae1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java @@ -134,9 +134,9 @@ public List> getTasks(HiveConf conf, Object objCtx) { List> resTsks = new ArrayList>(); // check if a map-reduce job is needed to merge the files // If the current size is smaller than the target, merge - long trgtSize = conf.getLongVar(HiveConf.ConfVars.HIVEMERGEMAPFILESSIZE); + long trgtSize = conf.getLongVar(HiveConf.ConfVars.HIVE_MERGE_MAP_FILES_SIZE); long avgConditionSize = conf - .getLongVar(HiveConf.ConfVars.HIVEMERGEMAPFILESAVGSIZE); + .getLongVar(HiveConf.ConfVars.HIVE_MERGE_MAP_FILES_AVG_SIZE); trgtSize = Math.max(trgtSize, avgConditionSize); Task mvTask = ctx.getListTasks().get(0); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicValue.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicValue.java index 04129a6adaa5..033df4e9ceff 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicValue.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicValue.java @@ -24,10 +24,7 @@ import org.apache.hadoop.hive.ql.exec.DynamicValueRegistry; import org.apache.hadoop.hive.ql.exec.ObjectCache; import org.apache.hadoop.hive.ql.exec.ObjectCacheFactory; -import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.sarg.LiteralDelegate; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; @@ -113,7 +110,7 @@ public Object getValue() { try { // Get object cache - String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); + String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID); ObjectCache cache = ObjectCacheFactory.getCache(conf, queryId, false, true); if (cache == null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java index 738ace050600..d5117fe768cf 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.MemoryMonitorInfo; import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectSerDeContext; -import org.apache.hadoop.hive.ql.optimizer.signature.Signature; import org.apache.hadoop.hive.ql.plan.Explain.Level; import org.apache.hadoop.hive.ql.plan.Explain.Vectorization; import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc.HashTableImplementationType; @@ -531,7 +530,7 @@ private VectorizationCondition[] createNativeConditions() { conditionList.add( new VectorizationCondition( vectorMapJoinDesc.getUseOptimizedTable(), - HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE.varname)); + HiveConf.ConfVars.HIVE_MAPJOIN_USE_OPTIMIZED_TABLE.varname)); conditionList.add( new VectorizationCondition( enabled, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java index 61d3da32a3f6..188ec71a57ea 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java @@ -1026,7 +1026,7 @@ public static String stripQuotes(String val) { * @return */ public static String removePrefixFromWarehouseConfig(String origiKey) { - String prefix = SessionState.get().getConf().getVar(HiveConf.ConfVars.METASTOREWAREHOUSE); + String prefix = SessionState.get().getConf().getVar(HiveConf.ConfVars.METASTORE_WAREHOUSE); if ((prefix != null) && (prefix.length() > 0)) { //Local file system is using pfile:/// {@link ProxyLocalFileSystem} prefix = prefix.replace("pfile:///", "pfile:/"); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java index b5fd1814c2d6..90a7ff727a10 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java @@ -405,7 +405,7 @@ private static void extractFinalCandidates(ExprNodeDesc expr, exprInfo.convertedExpr : expr); return; } else if (!FunctionRegistry.isOpAnd(expr) && - HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { ctx.addNonFinalCandidate(exprInfo != null ? exprInfo.alias : null, expr); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java index fa7f5710f6fd..01bb077daab7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java @@ -174,7 +174,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, OpWalkerInfo owi = (OpWalkerInfo) procCtx; ExprWalkerInfo childInfo = getChildWalkerInfo((Operator) nd, owi); if (childInfo != null && HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { ExprWalkerInfo unpushedPreds = mergeChildrenPred(nd, owi, null, false); return createFilter((Operator)nd, unpushedPreds, owi); } @@ -214,7 +214,7 @@ private void pushRankLimit(PTFOperator ptfOp, OpWalkerInfo owi) throws SemanticE return; } - float threshold = owi.getParseContext().getConf().getFloatVar(HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE); + float threshold = owi.getParseContext().getConf().getFloatVar(HiveConf.ConfVars.HIVE_LIMIT_PUSHDOWN_MEMORY_USAGE); if (threshold <= 0 || threshold >= 1) { return; } @@ -356,7 +356,7 @@ private void pushRankLimitToRedSink(PTFOperator ptfOp, HiveConf conf, int rLimit return; } - float threshold = conf.getFloatVar(HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE); + float threshold = conf.getFloatVar(HiveConf.ConfVars.HIVE_LIMIT_PUSHDOWN_MEMORY_USAGE); ReduceSinkOperator rSink = (ReduceSinkOperator) gP; ReduceSinkDesc rDesc = rSink.getConf(); @@ -392,7 +392,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object o = super.process(nd, stack, procCtx, nodeOutputs); OpWalkerInfo owi = (OpWalkerInfo) procCtx; if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { // The lateral view join is allowed to have a filter pushed through it. // We need to remove the filter candidate here once it has been applied. // If we do not remove it here, the candidates will be cleared out through @@ -440,7 +440,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, TableScanOperator tsOp = (TableScanOperator) nd; mergeWithChildrenPred(tsOp, owi, null, null); if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { // remove all the candidate filter operators // when we get to the TS removeAllCandidates(owi); @@ -495,7 +495,7 @@ Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, logExpr(nd, ewi); owi.putPrunedPreds((Operator) nd, ewi); if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { // add this filter for deletion, if it does not have non-final candidates owi.addCandidateFilterOp((FilterOperator)op); Map> residual = ewi.getResidualPredicates(true); @@ -505,7 +505,7 @@ Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, // merge it with children predicates boolean hasUnpushedPredicates = mergeWithChildrenPred(nd, owi, ewi, null); if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { if (hasUnpushedPredicates) { ExprWalkerInfo unpushedPreds = mergeChildrenPred(nd, owi, null, false); return createFilter((Operator)nd, unpushedPreds, owi); @@ -611,7 +611,7 @@ protected Set getAliases(Node nd) throws SemanticException { protected Object handlePredicates(Node nd, ExprWalkerInfo prunePreds, OpWalkerInfo owi) throws SemanticException { if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { return createFilter((Operator)nd, prunePreds.getResidualPredicates(true), owi); } return null; @@ -685,7 +685,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, if (operator.getNumChild() == 1 && operator.getChildOperators().get(0) instanceof JoinOperator) { if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDRECOGNIZETRANSITIVITY)) { + HiveConf.ConfVars.HIVE_PPD_RECOGNIZE_TRANSITIVITY)) { JoinOperator child = (JoinOperator) operator.getChildOperators().get(0); int targetPos = child.getParentOperators().indexOf(operator); applyFilterTransitivity(child, targetPos, owi); @@ -724,7 +724,7 @@ private void applyFilterTransitivity(JoinOperator join, int targetPos, OpWalkerI ExprWalkerInfo rsPreds = owi.getPrunedPreds(target); boolean recogniseColumnEqualities = HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPD_RECOGNIZE_COLUMN_EQUALITIES); + HiveConf.ConfVars.HIVE_PPD_RECOGNIZE_COLUMN_EQUALITIES); for (int sourcePos = 0; sourcePos < parentOperators.size(); sourcePos++) { ReduceSinkOperator source = (ReduceSinkOperator) parentOperators.get(sourcePos); List sourceKeys = source.getConf().getKeyCols(); @@ -1095,7 +1095,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Set includes = getQualifiedAliases((Operator) nd, owi); boolean hasUnpushedPredicates = mergeWithChildrenPred(nd, owi, null, includes); if (hasUnpushedPredicates && HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { if (includes != null || nd instanceof ReduceSinkOperator) { owi.getCandidateFilterOps().clear(); } else { @@ -1257,7 +1257,7 @@ protected static Object createFilter(Operator op, boolean pushFilterToStorage; HiveConf hiveConf = owi.getParseContext().getConf(); pushFilterToStorage = - hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTPPD_STORAGE); + hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_OPT_PPD_STORAGE); if (pushFilterToStorage) { condn = pushFilterToStorageHandler( (TableScanOperator) op, @@ -1288,7 +1288,7 @@ protected static Object createFilter(Operator op, } if (HiveConf.getBoolVar(owi.getParseContext().getConf(), - HiveConf.ConfVars.HIVEPPDREMOVEDUPLICATEFILTERS)) { + HiveConf.ConfVars.HIVE_PPD_REMOVE_DUPLICATE_FILTERS)) { // remove the candidate filter ops removeCandidates(op, owi); } @@ -1326,7 +1326,7 @@ private static ExprNodeGenericFuncDesc pushFilterToStorageHandler( TableScanDesc tableScanDesc = tableScanOp.getConf(); Table tbl = tableScanDesc.getTableMetadata(); - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTINDEXFILTER)) { + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER)) { // attach the original predicate to the table scan operator for index // optimizations that require the pushed predicate before pcr & later // optimizations are applied diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java index fbaabe06f258..87547116fed1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java @@ -234,7 +234,7 @@ public Map getHiveVariable() { return ss.getHiveVariables(); } }).substitute(conf, varValue); - if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CONF_VALIDATION)) { HiveConf.ConfVars confVars = HiveConf.getConfVars(key); if (confVars != null) { if (!confVars.isType(value)) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java b/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java index d3e6a4722c27..cfa50d5e6369 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/scheduled/ScheduledQueryExecutionService.java @@ -227,7 +227,7 @@ private void processQuery(ScheduledQueryPollResponse q) { conf.set(Constants.SCHEDULED_QUERY_SCHEDULENAME, q.getScheduleKey().getScheduleName()); conf.set(Constants.SCHEDULED_QUERY_USER, q.getUser()); conf.set(Constants.SCHEDULED_QUERY_EXECUTIONID, Long.toString(q.getExecutionId())); - conf.unset(HiveConf.ConfVars.HIVESESSIONID.varname); + conf.unset(HiveConf.ConfVars.HIVE_SESSION_ID.varname); state = new SessionState(conf, q.getUser()); state.setIsHiveServerQuery(true); SessionState.start(state); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/fallback/FallbackHiveAuthorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/fallback/FallbackHiveAuthorizer.java index 71a0d22bd6b2..eac288bd88d2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/fallback/FallbackHiveAuthorizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/fallback/FallbackHiveAuthorizer.java @@ -221,15 +221,15 @@ public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPl if (sessionCtx.getClientType() == HiveAuthzSessionContext.CLIENT_TYPE.HIVESERVER2 && hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) { - // Configure PREEXECHOOKS with DisallowTransformHook to disallow transform queries - String hooks = hiveConf.getVar(HiveConf.ConfVars.PREEXECHOOKS).trim(); + // Configure PRE_EXEC_HOOKS with DisallowTransformHook to disallow transform queries + String hooks = hiveConf.getVar(HiveConf.ConfVars.PRE_EXEC_HOOKS).trim(); if (hooks.isEmpty()) { hooks = DisallowTransformHook.class.getName(); } else { hooks = hooks + "," + DisallowTransformHook.class.getName(); } LOG.debug("Configuring hooks : " + hooks); - hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, hooks); + hiveConf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, hooks); SettableConfigUpdater.setHiveConfWhiteList(hiveConf); String curBlackList = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java index 12ebba63acb4..f1203dc36610 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java @@ -628,15 +628,15 @@ public void applyAuthorizationConfigPolicy(HiveConf hiveConf) throws HiveAuthzPl if (sessionCtx.getClientType() == CLIENT_TYPE.HIVESERVER2 && hiveConf.getBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED)) { - // Configure PREEXECHOOKS with DisallowTransformHook to disallow transform queries - String hooks = hiveConf.getVar(ConfVars.PREEXECHOOKS).trim(); + // Configure PRE_EXEC_HOOKS with DisallowTransformHook to disallow transform queries + String hooks = hiveConf.getVar(ConfVars.PRE_EXEC_HOOKS).trim(); if (hooks.isEmpty()) { hooks = DisallowTransformHook.class.getName(); } else { hooks = hooks + "," + DisallowTransformHook.class.getName(); } LOG.debug("Configuring hooks : " + hooks); - hiveConf.setVar(ConfVars.PREEXECHOOKS, hooks); + hiveConf.setVar(ConfVars.PRE_EXEC_HOOKS, hooks); SettableConfigUpdater.setHiveConfWhiteList(hiveConf); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/ClearDanglingScratchDir.java b/ql/src/java/org/apache/hadoop/hive/ql/session/ClearDanglingScratchDir.java index 62105dcec5a6..576a38d19603 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/ClearDanglingScratchDir.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/ClearDanglingScratchDir.java @@ -19,11 +19,9 @@ import java.io.File; import java.io.IOException; -import java.io.OutputStream; import java.util.ArrayList; import java.util.List; -import com.google.common.annotations.VisibleForTesting; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; @@ -101,7 +99,7 @@ public static void main(String[] args) throws Exception { if (cli.hasOption("s")) { rootHDFSDir = cli.getOptionValue("s"); } else { - rootHDFSDir = HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR); + rootHDFSDir = HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR); } ClearDanglingScratchDir clearDanglingScratchDirMain = new ClearDanglingScratchDir(dryRun, verbose, true, rootHDFSDir, conf); @@ -186,7 +184,7 @@ public void run() { return; } consoleMessage("Removing " + scratchDirToRemove.size() + " scratch directories"); - String localTmpDir = HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR); + String localTmpDir = HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR); for (Path scratchDir : scratchDirToRemove) { if (dryRun) { System.out.println(scratchDir); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index cb7ed48a58a5..3ba5fb3361f1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -391,7 +391,7 @@ public void deleteTmpErrOutputFile() { public boolean getIsSilent() { if(sessionConf != null) { - return sessionConf.getBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT); + return sessionConf.getBoolVar(HiveConf.ConfVars.HIVE_SESSION_SILENT); } else { return isSilent; } @@ -407,7 +407,7 @@ public boolean isHiveServerQuery() { public void setIsSilent(boolean isSilent) { if(sessionConf != null) { - sessionConf.setBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT, isSilent); + sessionConf.setBoolVar(HiveConf.ConfVars.HIVE_SESSION_SILENT, isSilent); } this.isSilent = isSilent; } @@ -463,13 +463,13 @@ public SessionState(HiveConf conf, String userName, CleanupService cleanupServic if (LOG.isDebugEnabled()) { LOG.debug("SessionState user: " + userName); } - isSilent = conf.getBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT); + isSilent = conf.getBoolVar(HiveConf.ConfVars.HIVE_SESSION_SILENT); resourceMaps = new ResourceMaps(); // Must be deterministic order map for consistent q-test output across Java versions overriddenConfigurations = new LinkedHashMap(); // if there isn't already a session name, go ahead and create it. - if (StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HIVESESSIONID))) { - conf.setVar(HiveConf.ConfVars.HIVESESSIONID, makeSessionId()); + if (StringUtils.isEmpty(conf.getVar(HiveConf.ConfVars.HIVE_SESSION_ID))) { + conf.setVar(HiveConf.ConfVars.HIVE_SESSION_ID, makeSessionId()); getConsole().printInfo("Hive Session ID = " + getSessionId()); } // Using system classloader as the parent. Using thread context @@ -500,7 +500,7 @@ public void setHiveVariables(Map hiveVariables) { } public String getSessionId() { - return (sessionConf.getVar(HiveConf.ConfVars.HIVESESSIONID)); + return (sessionConf.getVar(HiveConf.ConfVars.HIVE_SESSION_ID)); } public void updateThreadName() { @@ -781,14 +781,14 @@ private void createSessionDirs(String userName) throws IOException { HiveConf conf = getConf(); Path rootHDFSDirPath = createRootHDFSDir(conf); // Now create session specific dirs - String scratchDirPermission = HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIRPERMISSION); + String scratchDirPermission = HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR_PERMISSION); Path path; // 1. HDFS scratch dir path = new Path(rootHDFSDirPath, userName); hdfsScratchDirURIString = path.toUri().toString(); createPath(conf, path, scratchDirPermission, false, false); // 2. Local scratch dir - path = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR)); + path = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); createPath(conf, path, scratchDirPermission, true, false); // 3. Download resources dir path = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.DOWNLOADED_RESOURCES_DIR)); @@ -812,7 +812,7 @@ private void createSessionDirs(String userName) throws IOException { hdfsSessionPathLockFile = fs.create(new Path(hdfsSessionPath, LOCK_FILE_NAME), true); } // 6. Local session path - localSessionPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR), sessionId); + localSessionPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR), sessionId); createPath(conf, localSessionPath, scratchDirPermission, true, true); conf.set(LOCAL_SESSION_PATH_KEY, localSessionPath.toUri().toString()); // 7. HDFS temp table space @@ -837,7 +837,7 @@ private void createSessionDirs(String userName) throws IOException { * @throws IOException */ private Path createRootHDFSDir(HiveConf conf) throws IOException { - Path rootHDFSDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIR)); + Path rootHDFSDirPath = new Path(HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCH_DIR)); Utilities.ensurePathIsWritable(rootHDFSDirPath, conf); return rootHDFSDirPath; } @@ -1077,8 +1077,8 @@ public void setActiveAuthorizer(Object authorizer) { * @throws IOException */ private static File createTempFile(HiveConf conf) throws IOException { - String lScratchDir = HiveConf.getVar(conf, HiveConf.ConfVars.LOCALSCRATCHDIR); - String sessionID = conf.getVar(HiveConf.ConfVars.HIVESESSIONID); + String lScratchDir = HiveConf.getVar(conf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR); + String sessionID = conf.getVar(HiveConf.ConfVars.HIVE_SESSION_ID); return FileUtils.createTempFile(lScratchDir, sessionID, ".pipeout"); } @@ -1462,10 +1462,10 @@ public void loadAuxJars() throws IOException { public void loadReloadableAuxJars() throws IOException { LOG.info("Reloading auxiliary JAR files"); - final String renewableJarPath = sessionConf.getVar(ConfVars.HIVERELOADABLEJARS); + final String renewableJarPath = sessionConf.getVar(ConfVars.HIVE_RELOADABLE_JARS); // do nothing if this property is not specified or empty if (StringUtils.isBlank(renewableJarPath)) { - LOG.warn("Configuration {} not specified", ConfVars.HIVERELOADABLEJARS); + LOG.warn("Configuration {} not specified", ConfVars.HIVE_RELOADABLE_JARS); return; } @@ -2113,7 +2113,7 @@ public void setupQueryCurrentTimestamp() { // Provide a facility to set current timestamp during tests if (sessionConf.getBoolVar(ConfVars.HIVE_IN_TEST)) { String overrideTimestampString = - HiveConf.getVar(sessionConf, HiveConf.ConfVars.HIVETESTCURRENTTIMESTAMP, (String)null); + HiveConf.getVar(sessionConf, HiveConf.ConfVars.HIVE_TEST_CURRENT_TIMESTAMP, (String)null); if (overrideTimestampString != null && overrideTimestampString.length() > 0) { TimestampTZ zonedDateTime = TimestampTZUtil.convert( Timestamp.valueOf(overrideTimestampString), sessionConf.getLocalTimeZone()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsNoJobTask.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsNoJobTask.java index 723f64f594c7..244d5cd0b710 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsNoJobTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsNoJobTask.java @@ -231,7 +231,7 @@ public void run() { } ThreadPoolExecutor tpE = null; List> futures = null; - int numThreadsFactor = HiveConf.getIntVar(jc, HiveConf.ConfVars.BASICSTATSTASKSMAXTHREADSFACTOR); + int numThreadsFactor = HiveConf.getIntVar(jc, HiveConf.ConfVars.BASIC_STATS_TASKS_MAX_THREADS_FACTOR); if (fileList.size() > 1 && numThreadsFactor > 0) { int numThreads = Math.min(fileList.size(), numThreadsFactor * Runtime.getRuntime().availableProcessors()); ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("Basic-Stats-Thread-%d").build(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java index e2777a128bc4..e6f945bc864f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java @@ -148,7 +148,7 @@ private boolean constructColumnStatsFromPackedRows(Table tbl, List op : baseWork.getAllOperators()) { String operatorId = op.getOperatorId(); OperatorStats operatorStats = null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java index 1ba231546b46..239f57b69b3e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java @@ -605,7 +605,7 @@ public static ColStatistics getColStatsForPartCol(ColumnInfo ci,PartitionIterabl partCS.setAvgColLen(StatsUtils.getAvgColLenOf(conf, ci.getObjectInspector(), partCS.getColumnType())); partCS.setRange(getRangePartitionColumn(partList, ci.getInternalName(), - ci.getType().getTypeName(), conf.getVar(ConfVars.DEFAULTPARTITIONNAME))); + ci.getType().getTypeName(), conf.getVar(ConfVars.DEFAULT_PARTITION_NAME))); return partCS; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java index 5bf1c0bffb14..3443ea204558 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java @@ -425,7 +425,7 @@ private void setColumnTypes(JobConf job, List cols) { } job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, colNames.toString()); job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, colTypes.toString()); - HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + HiveConf.setVar(job, HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); } public JobConf getMrJob() { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java index 814ed9cd046b..9d1885ad9c3a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java @@ -68,8 +68,8 @@ protected boolean isDynPartAbort(Table t, CompactionInfo ci) { } protected void failCompactionIfSetForTest() { - if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION)) { - throw new RuntimeException(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION.name() + "=true"); + if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION)) { + throw new RuntimeException(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION.name() + "=true"); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRegExp.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRegExp.java index 8522abd0ee68..63fbcc0e7af1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRegExp.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRegExp.java @@ -64,7 +64,7 @@ public class GenericUDFRegExp extends GenericUDF { @Override public void configure(MapredContext context) { if (context != null) { - if(HiveConf.getBoolVar(context.getJobConf(), HiveConf.ConfVars.HIVEUSEGOOGLEREGEXENGINE)){ + if(HiveConf.getBoolVar(context.getJobConf(), HiveConf.ConfVars.HIVE_USE_GOOGLE_REGEX_ENGINE)){ this.useGoogleRegexEngine=true; } } @@ -75,7 +75,7 @@ public void configure(MapredContext context) { public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { SessionState ss = SessionState.get(); if (ss != null) { - this.useGoogleRegexEngine = ss.getConf().getBoolVar(HiveConf.ConfVars.HIVEUSEGOOGLEREGEXENGINE); + this.useGoogleRegexEngine = ss.getConf().getBoolVar(HiveConf.ConfVars.HIVE_USE_GOOGLE_REGEX_ENGINE); } checkArgsSize(arguments, 2, 2); @@ -93,7 +93,7 @@ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumen regexConst = getConstantStringValue(arguments, 1); if (regexConst != null) { if(!useGoogleRegexEngine){ - //if(!HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVEUSEGOOGLEREGEXENGINE)){ + //if(!HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_USE_GOOGLE_REGEX_ENGINE)){ patternConst = Pattern.compile(regexConst); }else{ patternConstR2j = com.google.re2j.Pattern.compile(regexConst); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java index 97d04676e76b..b1c3b767deca 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java @@ -238,7 +238,7 @@ protected SplitResult getSplitResult(boolean generateLightWeightSplits) + ConfVars.LLAP_HS2_ENABLE_COORDINATOR.varname + " enabled"); } ApplicationId extClientAppId = coordinator.createExtClientAppId(); - String externalDagName = SessionState.get().getConf().getVar(ConfVars.HIVEQUERYNAME); + String externalDagName = SessionState.get().getConf().getVar(ConfVars.HIVE_QUERY_NAME); StringBuilder sb = new StringBuilder(); sb.append("Generated appID ").append(extClientAppId.toString()).append(" for LLAP splits"); @@ -271,8 +271,8 @@ private PlanFragment createPlanFragment(String query, ApplicationId splitsAppId) throws HiveException { HiveConf conf = new HiveConf(SessionState.get().getConf()); - HiveConf.setVar(conf, ConfVars.HIVEFETCHTASKCONVERSION, "none"); - HiveConf.setVar(conf, ConfVars.HIVEQUERYRESULTFILEFORMAT, PlanUtils.LLAP_OUTPUT_FORMAT_KEY); + HiveConf.setVar(conf, ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); + HiveConf.setVar(conf, ConfVars.HIVE_QUERY_RESULT_FILEFORMAT, PlanUtils.LLAP_OUTPUT_FORMAT_KEY); String originalMode = HiveConf.getVar(conf, ConfVars.HIVE_EXECUTION_MODE); @@ -451,7 +451,7 @@ private SplitResult getSplits(JobConf job, TezWork work, Schema schema, Applicat // Update the queryId to use the generated extClientAppId. See comment below about // why this is done. - HiveConf.setVar(wxConf, HiveConf.ConfVars.HIVEQUERYID, extClientAppId.toString()); + HiveConf.setVar(wxConf, HiveConf.ConfVars.HIVE_QUERY_ID, extClientAppId.toString()); Vertex wx = utils.createVertex(wxConf, mapWork, scratchDir, work, DagUtils.createTezLrMap(appJarLr, null)); String vertexName = wx.getName(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java index 8bf51e5b8c3c..fb342e27153d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java @@ -213,7 +213,7 @@ private boolean streamingPossible(Configuration cfg, WindowFunctionDef wFnDef) return true; } - int windowLimit = HiveConf.getIntVar(cfg, ConfVars.HIVEJOINCACHESIZE); + int windowLimit = HiveConf.getIntVar(cfg, ConfVars.HIVE_JOIN_CACHE_SIZE); if (windowLimit < (start.getAmt() + end.getAmt() + 1)) { return false; @@ -271,7 +271,7 @@ private int[] setCanAcceptInputAsStream(Configuration cfg) throws HiveException return null; } - int windowLimit = HiveConf.getIntVar(cfg, ConfVars.HIVEJOINCACHESIZE); + int windowLimit = HiveConf.getIntVar(cfg, ConfVars.HIVE_JOIN_CACHE_SIZE); if (windowLimit < (endPos - startPos + 1)) { return null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java b/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java index 1dd9d8bf9db9..1d4b8fa22d03 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java @@ -574,7 +574,7 @@ private void readControlConfigs(FileSystem fs, Path path) { }); if (runOptions.shouldModifyManagedTableLocation || runOptions.shouldMoveExternal) { Configuration oldConf = new Configuration(conf); - HiveConf.setVar(oldConf, HiveConf.ConfVars.METASTOREWAREHOUSE, runOptions.oldWarehouseRoot); + HiveConf.setVar(oldConf, HiveConf.ConfVars.METASTORE_WAREHOUSE, runOptions.oldWarehouseRoot); oldWh = ThreadLocal.withInitial(() -> { try { @@ -650,7 +650,7 @@ static WarehouseRootCheckResult checkOldWarehouseRoot(RunOptions runOptions, Hiv shouldMoveExternal = false; } else { String currentPathString = shouldModifyManagedTableLocation ? - HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREWAREHOUSE) : + HiveConf.getVar(conf, HiveConf.ConfVars.METASTORE_WAREHOUSE) : HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL); if (arePathsEqual(conf, runOptions.oldWarehouseRoot, currentPathString)) { LOG.info("oldWarehouseRoot is the same as the target path {}." diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java b/ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java index 67df6a7bcec4..8871bb54e349 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/util/UpgradeTool.java @@ -175,7 +175,7 @@ private static IMetaStoreClient getHMS(HiveConf conf) { return RetryingMetaStoreClient.getProxy(conf, true); } catch (MetaException e) { throw new RuntimeException("Error connecting to Hive Metastore URI: " - + conf.getVar(HiveConf.ConfVars.METASTOREURIS) + ". " + e.getMessage(), e); + + conf.getVar(HiveConf.ConfVars.METASTORE_URIS) + ". " + e.getMessage(), e); } } /** diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java index 4f9f1d8e2fcb..8de9cbe93e10 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java @@ -1469,9 +1469,9 @@ public void showLocks() throws Exception { * 5.1.30 * * 2. Hack in the c'tor of this class - * conf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, "jdbc:mysql://localhost/metastore"); + * conf.setVar(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY, "jdbc:mysql://localhost/metastore"); * conf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME, "hive"); - * conf.setVar(HiveConf.ConfVars.METASTOREPWD, "hive"); + * conf.setVar(HiveConf.ConfVars.METASTORE_PWD, "hive"); * conf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, "com.mysql.jdbc.Driver"); * 3. Remove TxnDbUtil.prepDb(); in TxnHandler.checkQFileTestHack() * diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java index 1e2fe3303afe..46c8e8244564 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java @@ -65,7 +65,6 @@ import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; import org.apache.hadoop.hive.metastore.api.TxnInfo; import org.apache.hadoop.hive.metastore.api.TxnState; -import org.apache.hadoop.hive.metastore.api.AbortCompactResponse; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; @@ -123,7 +122,7 @@ void initHiveConf() { //TestTxnCommandsWithSplitUpdateAndVectorization has the vectorized version //of these tests. HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false); - HiveConf.setVar(hiveConf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + HiveConf.setVar(hiveConf, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_ACID_DROP_PARTITION_USE_BASE, false); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_ACID_RENAME_PARTITION_MAKE_COPY, false); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX, false); @@ -163,9 +162,9 @@ public void testInsertOverwrite() throws Exception { List rs = runStatementOnDriver("select a from " + Table.ACIDTBL + " where b = 2"); Assert.assertEquals(1, rs.size()); Assert.assertEquals("1", rs.get(0)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert overwrite table " + Table.ACIDTBL + " values(3,2)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); runStatementOnDriver("insert into " + Table.ACIDTBL + " values(5,6)"); rs = runStatementOnDriver("select a from " + Table.ACIDTBL + " order by a"); Assert.assertEquals(2, rs.size()); @@ -1463,9 +1462,9 @@ public void testMoreBucketsThanReducers() throws Exception { // todo: try using set VerifyNumReducersHook.num.reducers=10; d.destroy(); HiveConf hc = new HiveConf(hiveConf); - hc.setIntVar(HiveConf.ConfVars.MAXREDUCERS, 1); + hc.setIntVar(HiveConf.ConfVars.MAX_REDUCERS, 1); //this is used in multiple places, SemanticAnalyzer.getBucketingSortingDest() among others - hc.setIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS, 1); + hc.setIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS, 1); hc.setBoolVar(HiveConf.ConfVars.HIVE_EXPLAIN_USER, false); d = new Driver(hc); d.setMaxRows(10000); @@ -1483,9 +1482,9 @@ public void testMoreBucketsThanReducers2() throws Exception { //see bucket_num_reducers.q bucket_num_reducers2.q d.destroy(); HiveConf hc = new HiveConf(hiveConf); - hc.setIntVar(HiveConf.ConfVars.MAXREDUCERS, 2); + hc.setIntVar(HiveConf.ConfVars.MAX_REDUCERS, 2); //this is used in multiple places, SemanticAnalyzer.getBucketingSortingDest() among others - hc.setIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS, 2); + hc.setIntVar(HiveConf.ConfVars.HADOOP_NUM_REDUCERS, 2); d = new Driver(hc); d.setMaxRows(10000); runStatementOnDriver("create table fourbuckets (a int, b int) clustered by (a) into 4 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); @@ -2589,15 +2588,15 @@ public void testFetchTaskCachingWithConversion() throws Exception { runStatementOnDriver("insert into table fetch_task_table values (1,2), (3,4), (5,6)"); List expectedRes = runStatementOnDriver("select * from fetch_task_table"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEFETCHTASKCACHING, true); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CACHING, true); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); d.run("select * from fetch_task_table"); Assert.assertFalse(d.getFetchTask().isCachingEnabled()); d.getFetchTask().fetch(actualRes); Assert.assertEquals(actualRes, expectedRes); actualRes.clear(); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "more"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "more"); d.run("select * from fetch_task_table"); Assert.assertTrue(d.getFetchTask().isCachingEnabled()); d.getFetchTask().fetch(actualRes); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java index b96ec876db20..9b2edfa10f57 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java @@ -159,7 +159,7 @@ void initHiveConf() { HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false); //TestTxnCommands2WithAbortCleanupUsingCompactionCycle has the tests with abort cleanup in compaction cycle MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, true); - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, false); HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_ACID_TRUNCATE_USE_BASE, false); } @@ -204,8 +204,8 @@ public void testOrcNoPPD() throws Exception { * @throws Exception */ private void testOrcPPD(boolean enablePPD) throws Exception { - boolean originalPpd = hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, enablePPD);//enables ORC PPD + boolean originalPpd = hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER, enablePPD);//enables ORC PPD //create delta_0001_0001_0000 (should push predicate here) runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(new int[][]{{1, 2}, {3, 4}})); List explain; @@ -263,7 +263,7 @@ private void testOrcPPD(boolean enablePPD) throws Exception { List rs1 = runStatementOnDriver(query); int [][] resultData = new int[][] {{3, 5}, {5, 6}, {9, 10}}; Assert.assertEquals("Update failed", stringifyValues(resultData), rs1); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, originalPpd); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER, originalPpd); } static void assertExplainHasString(String string, List queryPlan, String errMsg) { @@ -1076,15 +1076,15 @@ public void testValidTxnsBookkeeping() throws Exception { @Test public void testSimpleRead() throws Exception { - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "more"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "more"); int[][] tableData = {{1,2},{3,3}}; runStatementOnDriver("insert into " + Table.ACIDTBL + " " + makeValuesClause(tableData)); int[][] tableData2 = {{5,3}}; //this will cause next txn to be marked aborted but the data is still written to disk - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + " " + makeValuesClause(tableData2)); assert hiveConf.get(ValidTxnWriteIdList.VALID_TABLES_WRITEIDS_KEY) == null : "previous txn should've cleaned it"; - //so now if HIVEFETCHTASKCONVERSION were to use a stale value, it would use a + //so now if HIVE_FETCH_TASK_CONVERSION were to use a stale value, it would use a //ValidWriteIdList with HWM=MAX_LONG, i.e. include the data for aborted txn List rs = runStatementOnDriver("select * from " + Table.ACIDTBL); Assert.assertEquals("Extra data", 2, rs.size()); @@ -1225,7 +1225,7 @@ void testInitiatorWithMultipleFailedCompactionsForVariousTblProperties(String tb //generate enough delta files so that Initiator can trigger auto compaction runStatementOnDriver("insert into " + tblName + " values(" + (i + 1) + ", 'foo'),(" + (i + 2) + ", 'bar'),(" + (i + 3) + ", 'baz')"); } - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, true); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true); MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true); @@ -1267,7 +1267,7 @@ void testInitiatorWithMultipleFailedCompactionsForVariousTblProperties(String tb MetastoreConf.getIntVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED) + MetastoreConf.getIntVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_DID_NOT_INITIATE)), countCompacts(txnHandler)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, false); txnHandler.compact(new CompactionRequest("default", tblName, CompactionType.MINOR)); //at this point "show compactions" should have (COMPACTOR_HISTORY_RETENTION_FAILED) failed + 1 initiated (explicitly by user) checkCompactionState(new CompactionsByState( @@ -1563,9 +1563,9 @@ private void writeBetweenWorkerAndCleanerForVariousTblProperties(String tblPrope runWorker(hiveConf); //delete something, but make sure txn is rolled back - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("delete from " + tblName + " where a = 1"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); List expected = new ArrayList<>(); expected.add("1\tfoo"); @@ -1622,7 +1622,7 @@ private void writeBetweenWorkerAndCleanerForVariousTblProperties(String tblPrope public void testFailHeartbeater() throws Exception { // Fail heartbeater, so that we can get a RuntimeException from the query. // More specifically, it's the original IOException thrown by either MR's or Tez's progress monitoring loop. - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_HEARTBEATER, true); Exception exception = null; try { runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(new int[][]{{1, 2}, {3, 4}})); @@ -1630,7 +1630,7 @@ public void testFailHeartbeater() throws Exception { exception = e; } Assert.assertNotNull(exception); - Assert.assertTrue(exception.getMessage().contains("HIVETESTMODEFAILHEARTBEATER=true")); + Assert.assertTrue(exception.getMessage().contains("HIVE_TEST_MODE_FAIL_HEARTBEATER=true")); } @Test @@ -1692,9 +1692,9 @@ public void testCompactWithDelete() throws Exception { public void testNoHistory() throws Exception { int[][] tableData = {{1,2},{3,4}}; runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'"); runWorker(hiveConf); @@ -1764,7 +1764,7 @@ protected void testACIDwithSchemaEvolutionForVariousTblProperties(String tblProp } protected void createAbortLowWaterMark() throws Exception{ - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("select * from " + Table.ACIDTBL); // wait for metastore.txn.opentxn.timeout Thread.sleep(1000); @@ -1774,7 +1774,7 @@ protected void createAbortLowWaterMark() throws Exception{ @Test public void testETLSplitStrategyForACID() throws Exception { hiveConf.setVar(HiveConf.ConfVars.HIVE_ORC_SPLIT_STRATEGY, "ETL"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPT_INDEX_FILTER, true); runStatementOnDriver("insert into " + Table.ACIDTBL + " values(1,2)"); runStatementOnDriver("alter table " + Table.ACIDTBL + " compact 'MAJOR'"); runWorker(hiveConf); @@ -2437,9 +2437,9 @@ public void testCleanerForTxnToWriteId() throws Exception { int[][] tableData5 = {{5, 6}}; runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p=3) (a,b) " + makeValuesClause(tableData3)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData4)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // Keep an open txn which refers to the aborted txn. Context ctx = new Context(hiveConf); @@ -2448,9 +2448,9 @@ public void testCleanerForTxnToWriteId() throws Exception { txnMgr.getValidTxns(); // Start an INSERT statement transaction and roll back this transaction. - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData5)); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData5)); @@ -2505,9 +2505,9 @@ public void testMmTableAbortWithCompaction() throws Exception { Assert.assertEquals("1", r1.get(0)); // 2. Let a transaction be aborted - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(3,4)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // There should be 1 delta and 1 base directory. The base one is the aborted one. verifyDeltaDirAndResult(2, Table.MMTBL.toString(), "", resultData1); @@ -2541,9 +2541,9 @@ public void testMmTableAbortWithCompaction() throws Exception { // 7. add few more rows runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(7,8)"); // 8. add one more aborted delta - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(9,10)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // 9. Perform a MAJOR compaction, expectation is it should remove aborted base dir runStatementOnDriver("alter table "+ Table.MMTBL + " compact 'MAJOR'"); @@ -2575,9 +2575,9 @@ public void testMmTableAbortWithCompactionNoCleanup() throws Exception { Assert.assertEquals("2", r1.get(0)); // 2. Let a transaction be aborted - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.MMTBL + " values(3,4)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // There should be 1 delta and 1 base directory. The base one is the aborted one. verifyDeltaDirAndResult(3, Table.MMTBL.toString(), "", resultData1); r1 = runStatementOnDriver("select count(*) from " + Table.MMTBL); @@ -2597,9 +2597,9 @@ public void testMmTableAbortWithCompactionNoCleanup() throws Exception { // 4. add few more rows runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(7,8)"); // 5. add one more aborted delta - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(9,10)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); verifyDeltaDirAndResult(5, Table.MMTBL.toString(), "", resultData3); // 6. Perform a MAJOR compaction, expectation is it should remove aborted delta dir @@ -2629,9 +2629,9 @@ public void testDynPartInsertWithAborts() throws Exception { verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData); // forcing a txn to abort before addDynamicPartitions - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, true); runStatementOnDriverWithAbort("insert into " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, false); verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData); int count = TestTxnDbUtil @@ -2666,10 +2666,10 @@ public void testDynPartInsertWithMultiPartitionAborts() throws Exception { Assert.assertEquals("4", r1.get(0)); // forcing a txn to abort before addDynamicPartitions - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, true); runStatementOnDriverWithAbort("insert into " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')"); runStatementOnDriverWithAbort("insert into " + Table.ACIDTBLPART + " partition(p) values(3,3,'p2'),(4,4,'p2')"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, false); verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData); verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p2", resultData); r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART); @@ -2709,9 +2709,9 @@ public void testDynPartIOWWithAborts() throws Exception { verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData); // forcing a txn to abort before addDynamicPartitions - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, true); runStatementOnDriverWithAbort("insert overwrite table " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, false); verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData); verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p1"); @@ -2748,10 +2748,10 @@ public void testDynPartIOWWithMultiPartitionAborts() throws Exception { Assert.assertEquals("4", r1.get(0)); // forcing a txn to abort before addDynamicPartitions - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, true); runStatementOnDriverWithAbort("insert overwrite table " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')"); runStatementOnDriverWithAbort("insert overwrite table " + Table.ACIDTBLPART + " partition(p) values(3,3,'p2'),(4,4,'p2')"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, false); verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData); verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p1"); verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p2", resultData); @@ -2796,9 +2796,9 @@ public void testDynPartUpdateWithAborts() throws Exception { verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData1); // forcing a txn to abort before addDynamicPartitions - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, true); runStatementOnDriverWithAbort("update " + Table.ACIDTBLPART + " set b=a+2 where a<5"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, false); verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData1); verifyDeleteDeltaDir(1, Table.ACIDTBLPART.toString(), "p=p1"); @@ -2836,12 +2836,12 @@ public void testDynPartMergeWithAborts() throws Exception { runStatementOnDriver("insert into " + TestTxnCommands2.Table.NONACIDORCTBL + " " + makeValuesClause(sourceVals)); // forcing a txn to abort before addDynamicPartitions - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, true); runStatementOnDriverWithAbort("merge into " + Table.ACIDTBLPART + " using " + TestTxnCommands2.Table.NONACIDORCTBL + " as s ON " + Table.ACIDTBLPART + ".a = s.a " + "when matched then update set b = s.b " + "when not matched then insert values(s.a, s.b, 'newpart')"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_LOAD_DYNAMIC_PARTITION, false); verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData); verifyDeleteDeltaDir(1, Table.ACIDTBLPART.toString(), "p=p1"); verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=newpart", resultData); @@ -2886,9 +2886,9 @@ public void testFullACIDAbortWithMinorMajorCompaction() throws Exception { Assert.assertEquals("1", r1.get(0)); // 2. Let a transaction be aborted - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(3,4)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // There should be 2 delta directories. verifyDeltaDirAndResult(2, Table.ACIDTBL.toString(), "", resultData1); @@ -2921,9 +2921,9 @@ public void testFullACIDAbortWithMinorMajorCompaction() throws Exception { runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(7,8)"); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(9,10)"); // 7. add one more aborted delta - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(11,12)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // 8. Perform a MAJOR compaction runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'"); @@ -2953,9 +2953,9 @@ public void testFullACIDAbortWithMajorCompaction() throws Exception { Assert.assertEquals("2", r1.get(0)); // 2. Let a transaction be aborted - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(5,6)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // There should be 2 delta and 1 base directory. The base one is the aborted one. verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData1); r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBL); @@ -2987,9 +2987,9 @@ public void testFullACIDAbortWithCompactionNoCleanup() throws Exception { verifyDeltaDirAndResult(2, Table.ACIDTBL.toString(), "", resultData1); // 2. abort one txns - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(5,6)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData1); // 3. Perform a MAJOR compaction. @@ -3019,14 +3019,14 @@ public void testFullACIDAbortWithManyPartitions() throws Exception { verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p3", resultData1); // 2. abort two txns in each partition - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p1') (a,b) values(5,6)"); runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p2') (a,b) values(5,6)"); runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p3') (a,b) values(5,6)"); runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p1') (a,b) values(5,6)"); runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p2') (a,b) values(5,6)"); runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p3') (a,b) values(5,6)"); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p1", resultData1); verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p2", resultData1); verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p3", resultData1); @@ -3398,7 +3398,7 @@ public void testCompactionOutputDirectoryNamesOnPartitionsAndOldDeltasDeleted() public void testShowCompactionOrder() throws Exception { d.destroy(); - hiveConf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + hiveConf.setVar(HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); d = new Driver(hiveConf); //generate some compaction history runStatementOnDriver("drop database if exists mydb1 cascade"); @@ -3434,7 +3434,7 @@ public void testShowCompactionOrder() throws Exception { runStatementOnDriver("insert into T values(1,4)");//makes delta_2_2 in T2 //create failed compaction attempt so that compactor txn is aborted - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, true); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, true); runStatementOnDriver("alter table T compact 'minor'"); TestTxnCommands2.runWorker(hiveConf); // Verify compaction order @@ -3454,7 +3454,7 @@ public void testShowCompactionOrder() throws Exception { public void testAbortCompaction() throws Exception { d.destroy(); - hiveConf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + hiveConf.setVar(HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); d = new Driver(hiveConf); //generate some compaction history runStatementOnDriver("drop database if exists mydb1 cascade"); @@ -3491,7 +3491,7 @@ public void testAbortCompaction() throws Exception { runStatementOnDriver("insert into myT1 values(1,4)");//makes delta_2_2 in T2 //create failed compaction attempt so that compactor txn is aborted - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, true); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, true); runStatementOnDriver("alter table myT1 compact 'minor'"); TestTxnCommands2.runWorker(hiveConf); // Verify compaction order diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java index c5a2639bdafc..b3fe87a8be4d 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java @@ -426,7 +426,7 @@ public void testCompactionAbort() throws Exception { runStatementOnDriver("insert into T values(1,4)");//makes delta_2_2 in T2 //create failed compaction attempt so that compactor txn is aborted - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, true); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, true); runStatementOnDriver("alter table T compact 'minor'"); runWorker(hiveConf); @@ -456,7 +456,7 @@ public void testCompactionAbort() throws Exception { Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_WRITEID=" + highestCompactWriteId)); //now make a successful compactor run so that next Cleaner run actually cleans - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, false); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, false); runStatementOnDriver("alter table T compact 'minor'"); runWorker(hiveConf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java index ba2b75cd9b07..5b243d2022b1 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.FileUtils; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.junit.Assert; @@ -88,9 +87,9 @@ public void setUp() throws Exception { } void setUpInternalExtended(boolean isOrcFormat) throws Exception { - hiveConf.setBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING, true); - hiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setBoolVar(HiveConf.ConfVars.DYNAMIC_PARTITIONING, true); + hiveConf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "true"); hiveConf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java index 95cf36f6e3ba..9b7fab9ac704 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnExIm.java @@ -584,9 +584,9 @@ public void testMMExportAborted() throws Exception { runStatementOnDriver("create table T (a int, b int)"); runStatementOnDriver("create table Tstage (a int, b int)"); - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); runStatementOnDriver("insert into Tstage" + TestTxnCommands2.makeValuesClause(dataAbort)); - HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); runStatementOnDriver("insert into Tstage" + TestTxnCommands2.makeValuesClause(data)); runStatementOnDriver("export table Tstage to '" + getWarehouseDir() + "/1'"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java index a00886bb9cd9..7409179f4368 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java @@ -19,10 +19,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; @@ -41,7 +38,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -311,7 +307,7 @@ public void testCtasEmpty() throws Exception { @Test public void testInsertToAcidWithUnionRemove() throws Exception { hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_UNION_REMOVE, true); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); d.close(); d = new Driver(hiveConf); int[][] values = {{1,2},{3,4},{5,6},{7,8},{9,10}}; @@ -350,7 +346,7 @@ public void testInsertToAcidWithUnionRemove() throws Exception { @Test public void testInsertOverwriteToAcidWithUnionRemove() throws Exception { hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_UNION_REMOVE, true); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); d.close(); d = new Driver(hiveConf); int[][] values = {{1, 2}, {3, 4}, {5, 6}, {7, 8}, {9, 10}}; @@ -374,9 +370,9 @@ public void testInsertOverwriteToAcidWithUnionRemove() throws Exception { @Test public void testToAcidConversionMultiBucket() throws Exception { //need to disable these so that automatic merge doesn't merge the files - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEMERGEMAPFILES, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEMERGEMAPREDFILES, false); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEMERGETEZFILES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_MERGE_MAPFILES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_MERGE_MAPRED_FILES, false); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_MERGE_TEZ_FILES, false); d.close(); d = new Driver(hiveConf); @@ -404,7 +400,7 @@ public void testToAcidConversionMultiBucket() throws Exception { //now do Insert from Union here to create data files in sub dirs hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_UNION_REMOVE, true); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); d.close(); d = new Driver(hiveConf); runStatementOnDriver("insert into T(a,b) select a * 10, b * 10 from " + Table.ACIDTBL + @@ -665,7 +661,7 @@ public void testCtasPartitioned() throws Exception { @Test public void testNonAcidToAcidVectorzied() throws Exception { hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, true); - hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHTASKCONVERSION, "none"); + hiveConf.setVar(HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); //this enables vectorization of ROW__ID hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ROW_IDENTIFIER_ENABLED, true);//HIVE-12631 runStatementOnDriver("drop table if exists T"); @@ -773,7 +769,7 @@ private void checkExpected(List rs, String[][] expected, String msg) { */ @Test public void testCompactStatsGather() throws Exception { - hiveConf.setIntVar(HiveConf.ConfVars.HIVEOPTSORTDYNAMICPARTITIONTHRESHOLD, -1); + hiveConf.setIntVar(HiveConf.ConfVars.HIVE_OPT_SORT_DYNAMIC_PARTITION_THRESHOLD, -1); runStatementOnDriver("drop table if exists T"); runStatementOnDriver("create table T(a int, b int) partitioned by (p int, q int) " + "stored as orc TBLPROPERTIES ('transactional'='true')"); @@ -916,7 +912,7 @@ public void testEmptyCompactionResult() throws Exception { */ @Test public void testGetPartitionsNoSession() throws Exception { - hiveConf.setIntVar(HiveConf.ConfVars.HIVEOPTSORTDYNAMICPARTITIONTHRESHOLD, -1); + hiveConf.setIntVar(HiveConf.ConfVars.HIVE_OPT_SORT_DYNAMIC_PARTITION_THRESHOLD, -1); runStatementOnDriver("drop table if exists T"); runStatementOnDriver("create table T(a int, b int) partitioned by (p int, q int) " + "stored as orc TBLPROPERTIES ('transactional'='true')"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java index 3a9b0cb754cd..b7acefc531c4 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java @@ -119,10 +119,10 @@ void setUpInternal() throws Exception { + File.separator + "mapred" + File.separator + "staging"); hiveConf.set("mapred.temp.dir", workDir + File.separator + this.getClass().getSimpleName() + File.separator + "mapred" + File.separator + "temp"); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, getWarehouseDir()); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, getWarehouseDir()); + hiveConf.setVar(HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); hiveConf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java index 801133d85c61..00a58c4cea6d 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java @@ -89,8 +89,8 @@ public class TestExecDriver { try { queryState = new QueryState.Builder().withHiveConf(new HiveConf(ExecDriver.class)).build(); conf = queryState.getConf(); - conf.setBoolVar(HiveConf.ConfVars.SUBMITVIACHILD, true); - conf.setBoolVar(HiveConf.ConfVars.SUBMITLOCALTASKVIACHILD, true); + conf.setBoolVar(HiveConf.ConfVars.SUBMIT_VIA_CHILD, true); + conf.setBoolVar(HiveConf.ConfVars.SUBMIT_LOCAL_TASK_VIA_CHILD, true); conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java index 2ea15cf4924f..099c12bd686c 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java @@ -210,7 +210,7 @@ public void testGetJSONDependenciesJsonShhouldMatch() throws Exception { @Test public void testGetJSONPlan() throws Exception { - uut.conf.setVar(HiveConf.ConfVars.HIVESTAGEIDREARRANGE, "EXECUTION"); + uut.conf.setVar(HiveConf.ConfVars.HIVE_STAGE_ID_REARRANGE, "EXECUTION"); Task mockTask = mockTask(); when(mockTask.getId()).thenReturn("mockTaskId"); ExplainWork explainWorkMock = mockExplainWork(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestLimitOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestLimitOperator.java index 681435c65c8b..0c5679b223ec 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestLimitOperator.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestLimitOperator.java @@ -64,7 +64,7 @@ private void testGlobalLimitReachedInDaemonOrContainer(boolean isDaemon, int off } HiveConf conf = new HiveConf(); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYID, "query-" + random.nextInt(10000)); + HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID, "query-" + random.nextInt(10000)); HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "tez"); conf.set(TezProcessor.HIVE_TEZ_VERTEX_NAME, "Map 1"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java index c82fdf3a1d9a..dcf0483cf057 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java @@ -200,11 +200,11 @@ public void testScriptOperatorEnvVarsProcessing() throws Throwable { hconf.set("name", hugeEnvVar); Map env = new HashMap(); - HiveConf.setBoolVar(hconf, HiveConf.ConfVars.HIVESCRIPTTRUNCATEENV, false); + HiveConf.setBoolVar(hconf, HiveConf.ConfVars.HIVE_SCRIPT_TRUNCATE_ENV, false); scriptOperator.addJobConfToEnvironment(hconf, env); assertEquals(20*1024+1, env.get("name").length()); - HiveConf.setBoolVar(hconf, HiveConf.ConfVars.HIVESCRIPTTRUNCATEENV, true); + HiveConf.setBoolVar(hconf, HiveConf.ConfVars.HIVE_SCRIPT_TRUNCATE_ENV, true); scriptOperator.addJobConfToEnvironment(hconf, env); assertEquals(20*1024, env.get("name").length()); @@ -223,7 +223,7 @@ public void testScriptOperatorBlacklistedEnvVarsProcessing() { Map env = new HashMap(); - HiveConf.setVar(hconf, HiveConf.ConfVars.HIVESCRIPT_ENV_BLACKLIST, "foobar"); + HiveConf.setVar(hconf, HiveConf.ConfVars.HIVE_SCRIPT_ENV_BLACKLIST, "foobar"); hconf.set("foobar", "foobar"); hconf.set("barfoo", "barfoo"); scriptOperator.addJobConfToEnvironment(hconf, env); @@ -423,7 +423,7 @@ public InputSplit[] getSplits(JobConf job, int splits) throws IOException { public void testFetchOperatorContext() throws Exception { HiveConf conf = new HiveConf(); conf.set("hive.support.concurrency", "false"); - conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + conf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); SessionState.start(conf); @@ -459,7 +459,7 @@ public void testNoConditionalTaskSizeForLlap() { ConvertJoinMapJoin convertJoinMapJoin = new ConvertJoinMapJoin(); long defaultNoConditionalTaskSize = 1024L * 1024L * 1024L; HiveConf hiveConf = new HiveConf(); - hiveConf.setLongVar(HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD, defaultNoConditionalTaskSize); + hiveConf.setLongVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN_NOCONDITIONAL_TASK_THRESHOLD, defaultNoConditionalTaskSize); LlapClusterStateForCompile llapInfo = null; if ("llap".equalsIgnoreCase(hiveConf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_MODE))) { @@ -577,7 +577,7 @@ public void testLlapMemoryOversubscriptionMaxExecutorsPerQueryCalculation() { // 5. Configure hive conf and Build group by operator HiveConf hconf = new HiveConf(); - HiveConf.setIntVar(hconf, HiveConf.ConfVars.HIVEGROUPBYMAPINTERVAL, 1); + HiveConf.setIntVar(hconf, HiveConf.ConfVars.HIVE_GROUPBY_MAP_INTERVAL, 1); // 6. test hash aggr without grouping sets System.out.println("---------------Begin to test hash group by without grouping sets-------------"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java index 15106909734e..f87d6c40f17f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java @@ -36,7 +36,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; @@ -345,7 +344,7 @@ public void testGetInputPathsWithEmptyPartitions() throws Exception { List inputPaths = new ArrayList<>(); try { - Path scratchDir = new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCALSCRATCHDIR)); + Path scratchDir = new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); List inputPaths1 = Utilities.getInputPaths(jobConf, mapWork1, scratchDir, mock(Context.class), false); @@ -414,7 +413,7 @@ public void testGetInputPathsWithMultipleThreadsAndEmptyPartitions() throws Exce try { fs.mkdirs(testTablePath); List inputPaths = Utilities.getInputPaths(jobConf, mapWork, - new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCALSCRATCHDIR)), mock(Context.class), false); + new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)), mock(Context.class), false); assertEquals(inputPaths.size(), numPartitions); for (int i = 0; i < numPartitions; i++) { @@ -542,7 +541,7 @@ public void testGetInputPathsWithMultipleThreads() throws Exception { private void runTestGetInputPaths(JobConf jobConf, int numOfPartitions) throws Exception { MapWork mapWork = new MapWork(); - Path scratchDir = new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCALSCRATCHDIR)); + Path scratchDir = new Path(HiveConf.getVar(jobConf, HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); Map> pathToAliasTable = new LinkedHashMap<>(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/errors/TestTaskLogProcessor.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/errors/TestTaskLogProcessor.java index e95e9c32918d..404ac4a41e07 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/errors/TestTaskLogProcessor.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/errors/TestTaskLogProcessor.java @@ -231,7 +231,7 @@ public void testMapAggrMemErrorHeuristic() throws Exception { String solution = eas.getSolution(); assertNotNull(solution); assertTrue(solution.length() > 0); - String confName = HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY.toString(); + String confName = HiveConf.ConfVars.HIVE_MAP_AGGR_HASH_MEMORY.toString(); assertTrue(solution.contains(confName)); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java index 40712eb96d48..d4f983c62c03 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java @@ -69,7 +69,7 @@ public void mrTaskSumbitViaChildWithImpersonation() throws IOException, LoginExc QueryState queryState = new QueryState.Builder().build(); HiveConf conf= queryState.getConf(); - conf.setBoolVar(HiveConf.ConfVars.SUBMITVIACHILD, true); + conf.setBoolVar(HiveConf.ConfVars.SUBMIT_VIA_CHILD, true); MapredWork mrWork = new MapredWork(); mrWork.setMapWork(Mockito.mock(MapWork.class)); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java index 74a4ad0b61b9..ef2232c626ab 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java @@ -87,7 +87,7 @@ private void testGlobalLimitReachedInDaemonOrContainer(boolean isDaemon, int off } HiveConf conf = new HiveConf(); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEQUERYID, "query-" + random.nextInt(10000)); + HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_QUERY_ID, "query-" + random.nextInt(10000)); HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "tez"); conf.set(TezProcessor.HIVE_TEZ_VERTEX_NAME, "Map 1"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveHooks.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveHooks.java index f5bc5bab5a25..b6284aff329f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveHooks.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHiveHooks.java @@ -61,9 +61,9 @@ public static class SemanticAnalysisHook implements HiveSemanticAnalyzerHook { @Test public void testLoadHooksFromConf() throws Exception { HiveConf hiveConf = new HiveConf(); - hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, + hiveConf.setVar(HiveConf.ConfVars.PRE_EXEC_HOOKS, PreExecHook.class.getName() + "," + PreExecHook.class.getName()); - hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, + hiveConf.setVar(HiveConf.ConfVars.POST_EXEC_HOOKS, PostExecHook.class.getName()); hiveConf.setVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, SemanticAnalysisHook.class.getName()); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHooks.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHooks.java index f1a9a44e1fe5..e0dccc9f5834 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHooks.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestHooks.java @@ -55,7 +55,7 @@ public void testRedactLogString() throws Exception { HiveConf conf = new HiveConf(TestHooks.class); String str; - HiveConf.setVar(conf, HiveConf.ConfVars.QUERYREDACTORHOOKS, SimpleQueryRedactor.class.getName()); + HiveConf.setVar(conf, HiveConf.ConfVars.QUERY_REDACTOR_HOOKS, SimpleQueryRedactor.class.getName()); str = HookUtils.redactLogString(null, null); assertEquals(str, null); @@ -70,7 +70,7 @@ public void testRedactLogString() throws Exception { @Test public void testQueryRedactor() throws Exception { HiveConf conf = new HiveConf(TestHooks.class); - HiveConf.setVar(conf, HiveConf.ConfVars.QUERYREDACTORHOOKS, + HiveConf.setVar(conf, HiveConf.ConfVars.QUERY_REDACTOR_HOOKS, SimpleQueryRedactor.class.getName()); conf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java b/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java index 0adaf0a700eb..a929bb9d820f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java @@ -26,16 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.io.RCFileInputFormat; -import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; -import org.apache.hadoop.hive.ql.io.RCFileStorageFormatDescriptor; -import org.apache.hadoop.hive.ql.io.StorageFormatDescriptor; -import org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat; -import org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcSerde; -import org.apache.hadoop.hive.serde2.avro.AvroSerDe; import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; -import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; import static org.junit.Assert.assertTrue; @@ -104,9 +95,9 @@ public static Collection asParameters() { String serdeClass = descriptor.getSerde(); if (serdeClass == null) { if (descriptor instanceof RCFileStorageFormatDescriptor) { - serdeClass = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE); + serdeClass = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_RCFILE_SERDE); } else { - serdeClass = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTSERDE); + serdeClass = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DEFAULT_SERDE); } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java index 73096b36720b..073b930531b2 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java @@ -643,7 +643,7 @@ public void testSync() throws IOException { RCFileInputFormat inputFormat = new RCFileInputFormat(); JobConf jobconf = new JobConf(cloneConf); jobconf.set("mapred.input.dir", testDir.toString()); - HiveConf.setLongVar(jobconf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, fileLen); + HiveConf.setLongVar(jobconf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, fileLen); InputSplit[] splits = inputFormat.getSplits(jobconf, 1); RCFileRecordReader rr = new RCFileRecordReader(jobconf, (FileSplit)splits[0]); long lastSync = 0; @@ -710,7 +710,7 @@ private void writeThenReadByRecordReader(int intervalRecordCount, RCFileInputFormat inputFormat = new RCFileInputFormat(); JobConf jonconf = new JobConf(cloneConf); jonconf.set("mapred.input.dir", testDir.toString()); - HiveConf.setLongVar(jonconf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, minSplitSize); + HiveConf.setLongVar(jonconf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, minSplitSize); InputSplit[] splits = inputFormat.getSplits(jonconf, splitNumber); assertEquals("splits length should be " + splitNumber, splitNumber, splits.length); int readCount = 0; @@ -796,7 +796,7 @@ public void testRCFileHeader(char[] expected, Configuration conf) @Test public void testNonExplicitRCFileHeader() throws IOException, SerDeException { Configuration conf = new Configuration(); - conf.setBoolean(HiveConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, false); + conf.setBoolean(HiveConf.ConfVars.HIVE_USE_EXPLICIT_RCFILE_HEADER.varname, false); char[] expected = new char[] {'S', 'E', 'Q'}; testRCFileHeader(expected, conf); } @@ -804,7 +804,7 @@ public void testNonExplicitRCFileHeader() throws IOException, SerDeException { @Test public void testExplicitRCFileHeader() throws IOException, SerDeException { Configuration conf = new Configuration(); - conf.setBoolean(HiveConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, true); + conf.setBoolean(HiveConf.ConfVars.HIVE_USE_EXPLICIT_RCFILE_HEADER.varname, true); char[] expected = new char[] {'R', 'C', 'F'}; testRCFileHeader(expected, conf); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java index e885c634c054..c986d10ab4cf 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java @@ -1728,8 +1728,8 @@ public void testSplitGenerator() throws Exception { new MockBlock("host0", "host3-2", "host3-3"), new MockBlock("host4-1", "host4-2", "host4-3"), new MockBlock("host5-1", "host5-2", "host5-3"))); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 300); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 200); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 300); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 200); OrcInputFormat.Context context = new OrcInputFormat.Context(conf); OrcInputFormat.SplitGenerator splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs, @@ -1752,8 +1752,8 @@ public void testSplitGenerator() throws Exception { assertEquals(1800, result.getStart()); assertEquals(200, result.getLength()); // test min = 0, max = 0 generates each stripe - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 0); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 0); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 0); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 0); context = new OrcInputFormat.Context(conf); splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs, fs.getFileStatus(new Path("/a/file")), null, null, true, @@ -1777,8 +1777,8 @@ public void testProjectedColumnSize() throws Exception { new MockBlock("host0", "host3-2", "host3-3"), new MockBlock("host4-1", "host4-2", "host4-3"), new MockBlock("host5-1", "host5-2", "host5-3"))); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 300); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 200); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 300); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 200); conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false); conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0"); OrcInputFormat.Context context = new OrcInputFormat.Context(conf); @@ -1802,8 +1802,8 @@ public void testProjectedColumnSize() throws Exception { assertEquals(43792, result.getProjectedColumnsUncompressedSize()); // test min = 0, max = 0 generates each stripe - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 0); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 0); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 0); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 0); context = new OrcInputFormat.Context(conf); splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs, fs.getFileStatus(new Path("/a/file")), null, null, true, @@ -1822,8 +1822,8 @@ public void testProjectedColumnSize() throws Exception { } // single split - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 1000); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 100000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 1000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 100000); context = new OrcInputFormat.Context(conf); splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs, fs.getFileStatus(new Path("/a/file")), null, null, true, @@ -3785,13 +3785,13 @@ public void testRowNumberUniquenessInDifferentSplits() throws Exception { // Save the conf variable values so that they can be restored later. long oldDefaultStripeSize = conf.getLong(OrcConf.STRIPE_SIZE.getHiveConfName(), -1L); - long oldMaxSplitSize = conf.getLong(HiveConf.ConfVars.MAPREDMAXSPLITSIZE.varname, -1L); + long oldMaxSplitSize = conf.getLong(HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE.varname, -1L); // Set the conf variable values for this test. long newStripeSize = 10000L; // 10000 bytes per stripe long newMaxSplitSize = 100L; // 1024 bytes per split conf.setLong(OrcConf.STRIPE_SIZE.getHiveConfName(), newStripeSize); - conf.setLong(HiveConf.ConfVars.MAPREDMAXSPLITSIZE.varname, newMaxSplitSize); + conf.setLong(HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE.varname, newMaxSplitSize); AbstractSerDe serde = new OrcSerde(); HiveOutputFormat outFormat = new OrcOutputFormat(); @@ -3838,10 +3838,10 @@ public void testRowNumberUniquenessInDifferentSplits() throws Exception { conf.unset(OrcConf.STRIPE_SIZE.getHiveConfName()); } if (oldMaxSplitSize != -1L) { - conf.setLong(HiveConf.ConfVars.MAPREDMAXSPLITSIZE.varname, oldMaxSplitSize); + conf.setLong(HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE.varname, oldMaxSplitSize); } else { // this means that nothing was set for default stripe size previously, so we should unset it. - conf.unset(HiveConf.ConfVars.MAPREDMAXSPLITSIZE.varname); + conf.unset(HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE.varname); } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java index f046191ae47e..0b6d57636d38 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java @@ -127,8 +127,8 @@ public void testSplitEliminationSmallMaxSplit() throws Exception { 100000, CompressionKind.NONE, 10000, 10000); writeData(writer); writer.close(); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 1000); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 5000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 1000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 5000); InputFormat in = new OrcInputFormat(); FileInputFormat.setInputPaths(conf, testFilePath.toString()); @@ -197,8 +197,8 @@ public void testSplitEliminationLargeMaxSplit() throws Exception { 100000, CompressionKind.NONE, 10000, 10000); writeData(writer); writer.close(); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 1000); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 150000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 1000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 150000); InputFormat in = new OrcInputFormat(); FileInputFormat.setInputPaths(conf, testFilePath.toString()); @@ -278,8 +278,8 @@ public void testSplitEliminationComplexExpr() throws Exception { 100000, CompressionKind.NONE, 10000, 10000); writeData(writer); writer.close(); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 1000); - HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, 150000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE, 1000); + HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE, 150000); InputFormat in = new OrcInputFormat(); FileInputFormat.setInputPaths(conf, testFilePath.toString()); @@ -696,10 +696,10 @@ private static String toString(FileSplit fs) { private void setupExternalCacheConfig(boolean isPpd, String paths) { FileInputFormat.setInputPaths(conf, paths); conf.set(ConfVars.HIVE_ORC_SPLIT_STRATEGY.varname, "ETL"); - conf.setLong(HiveConf.ConfVars.MAPREDMINSPLITSIZE.varname, 1000); - conf.setLong(HiveConf.ConfVars.MAPREDMAXSPLITSIZE.varname, 5000); + conf.setLong(HiveConf.ConfVars.MAPRED_MIN_SPLIT_SIZE.varname, 1000); + conf.setLong(HiveConf.ConfVars.MAPRED_MAX_SPLIT_SIZE.varname, 5000); conf.setBoolean(ConfVars.HIVE_ORC_MS_FOOTER_CACHE_PPD.varname, isPpd); - conf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, isPpd); + conf.setBoolean(ConfVars.HIVE_OPT_INDEX_FILTER.varname, isPpd); } private ObjectInspector createIO() { diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java index b192da437ea7..2c80d3ee19ee 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedOrcAcidRowBatchReader.java @@ -383,13 +383,13 @@ public void testDeleteEventFilteringOn2() throws Exception { @Test public void testDeleteEventFilteringOnWithoutIdx2() throws Exception { HiveConf.setBoolVar(conf, HiveConf.ConfVars.FILTER_DELETE_EVENTS, true); - HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVETESTMODEACIDKEYIDXSKIP, true); + HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_TEST_MODE_ACID_KEY_IDX_SKIP, true); testDeleteEventFiltering2(); } @Test public void testDeleteEventFilteringOnWithoutIdx3() throws Exception { HiveConf.setBoolVar(conf, HiveConf.ConfVars.FILTER_DELETE_EVENTS, true); - HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVETESTMODEACIDKEYIDXSKIP, true); + HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_TEST_MODE_ACID_KEY_IDX_SKIP, true); conf.set("orc.stripe.size", "1000"); testDeleteEventFiltering(); } @@ -398,7 +398,7 @@ private void testDeleteEventFiltering2() throws Exception { boolean filterOn = HiveConf.getBoolVar(conf, HiveConf.ConfVars.FILTER_DELETE_EVENTS); boolean skipKeyIdx = - HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVETESTMODEACIDKEYIDXSKIP); + HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_TEST_MODE_ACID_KEY_IDX_SKIP); int bucket = 1; AcidOutputFormat.Options options = new AcidOutputFormat.Options(conf) .filesystem(fs) diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/ITestDbTxnManager.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/ITestDbTxnManager.java index e6806bdf7332..17328b1281e3 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/ITestDbTxnManager.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/ITestDbTxnManager.java @@ -50,7 +50,7 @@ public static void setupDb() throws Exception { .toLowerCase(); rule = getDatabaseRule(metastoreType).setVerbose(false); - conf.setVar(HiveConf.ConfVars.METASTOREDBTYPE, metastoreType.toUpperCase()); + conf.setVar(HiveConf.ConfVars.METASTORE_DB_TYPE, metastoreType.toUpperCase()); MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY, rule.getJdbcUrl()); MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECTION_DRIVER, rule.getJdbcDriver()); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java index dc20b552c1af..f7a02cc76746 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java @@ -502,7 +502,7 @@ public void testMetastoreTablesCleanup() throws Exception { Assert.assertEquals(5, count); // Fail some inserts, so that we have records in TXN_COMPONENTS - conf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true); + conf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, true); driver.run("insert into temp.T10 values (9, 9)"); driver.run("insert into temp.T11 values (10, 10)"); driver.run("insert into temp.T12p partition (ds='today', hour='1') values (11, 11)"); @@ -510,7 +510,7 @@ public void testMetastoreTablesCleanup() throws Exception { count = TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"TXN_COMPONENTS\" " + "where \"TC_DATABASE\"='temp' and \"TC_TABLE\" in ('t10', 't11', 't12p', 't13p')"); Assert.assertEquals(4, count); - conf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false); + conf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_ROLLBACK_TXN, false); // Drop a table/partition; corresponding records in TXN_COMPONENTS and COMPLETED_TXN_COMPONENTS should disappear count = TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"TXN_COMPONENTS\" " + @@ -580,7 +580,7 @@ public void testMetastoreTablesCleanup() throws Exception { // Tables need at least 2 delta files to compact, and minor compaction was just run, so insert driver.run("insert into temp.T11 values (14, 14)"); driver.run("insert into temp.T12p partition (ds='tomorrow', hour='2') values (15, 15)"); - conf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, true); + conf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, true); driver.run("alter table temp.T11 compact 'major'"); count = TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"COMPACTION_QUEUE\" " + "where \"CQ_DATABASE\"='temp' and \"CQ_TABLE\"='t11' and \"CQ_STATE\"='i' and \"CQ_TYPE\"='a'"); @@ -606,7 +606,7 @@ public void testMetastoreTablesCleanup() throws Exception { count = TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"COMPLETED_COMPACTIONS\" " + "where \"CC_DATABASE\"='temp' and \"CC_TABLE\"='t12p' and \"CC_STATE\"='f' and \"CC_TYPE\"='a'"); Assert.assertEquals(1, count); - conf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION, false); + conf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_MODE_FAIL_COMPACTION, false); // Put 2 records into COMPACTION_QUEUE and do nothing driver.run("alter table temp.T11 compact 'major'"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java index 33171ebb0cb2..f823b324b0c5 100755 --- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java @@ -943,8 +943,8 @@ public void testHiveRefreshOnConfChange() throws Throwable{ prevHiveObj.getDatabaseCurrent(); //change value of a metavar config param in new hive conf newHconf = new HiveConf(hiveConf); - newHconf.setIntVar(ConfVars.METASTORETHRIFTCONNECTIONRETRIES, - newHconf.getIntVar(ConfVars.METASTORETHRIFTCONNECTIONRETRIES) + 1); + newHconf.setIntVar(ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES, + newHconf.getIntVar(ConfVars.METASTORE_THRIFT_CONNECTION_RETRIES) + 1); newHiveObj = Hive.get(newHconf); assertTrue(prevHiveObj != newHiveObj); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientListPartitionsTempTable.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientListPartitionsTempTable.java index 1c26899ce21e..09d9dc2f5327 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientListPartitionsTempTable.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSessionHiveMetastoreClientListPartitionsTempTable.java @@ -260,7 +260,7 @@ private void checkPartitionNames(List expected, short numParts, String o @Test public void testListPartitionNames() throws Exception { Table t = createTable4PartColsParts(getClient()).table; - String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULT_PARTITION_NAME); List> testValues = Lists.newArrayList( Lists.newArrayList("1999", defaultPartitionName, "02"), Lists.newArrayList(defaultPartitionName, "02", "10"), diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestNullScanTaskDispatcher.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestNullScanTaskDispatcher.java index c9fc2a54edd6..07eabd171f55 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestNullScanTaskDispatcher.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestNullScanTaskDispatcher.java @@ -85,7 +85,7 @@ public class TestNullScanTaskDispatcher { public void setup() { hiveConf = new HiveConf(); hiveConf.set("fs.mock.impl", MockFileSystem.class.getName()); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES, true); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_METADATA_ONLY_QUERIES, true); sessionState = SessionState.start(hiveConf); parseContext = spy(new ParseContext()); context = new Context(hiveConf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java index ac5795295158..211c80aa662b 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestDMLSemanticAnalyzer.java @@ -232,7 +232,7 @@ public void setup() throws Exception { conf .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); - conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict"); + conf.setVar(HiveConf.ConfVars.HIVE_MAPRED_MODE, "nonstrict"); conf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"); conf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, true); conf.set(ValidTxnList.VALID_TXNS_KEY, diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestConditionalResolverCommonJoin.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestConditionalResolverCommonJoin.java index 780fb2a58e22..56c30c103bb2 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestConditionalResolverCommonJoin.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestConditionalResolverCommonJoin.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.ql.exec.Task; import org.junit.Test; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -70,19 +69,19 @@ public void testResolvingDriverAlias() throws Exception { ctx.setAliasToKnownSize(aliasToKnownSize); HiveConf conf = new HiveConf(); - conf.setLongVar(HiveConf.ConfVars.HIVESMALLTABLESFILESIZE, 4096); + conf.setLongVar(HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE, 4096); // alias3 only can be selected Task resolved = resolver.resolveMapJoinTask(ctx, conf); Assert.assertEquals("alias3", resolved.getId()); - conf.setLongVar(HiveConf.ConfVars.HIVESMALLTABLESFILESIZE, 65536); + conf.setLongVar(HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE, 65536); // alias1, alias2, alias3 all can be selected but overriden by biggest one (alias3) resolved = resolver.resolveMapJoinTask(ctx, conf); Assert.assertEquals("alias3", resolved.getId()); - conf.setLongVar(HiveConf.ConfVars.HIVESMALLTABLESFILESIZE, 2048); + conf.setLongVar(HiveConf.ConfVars.HIVE_SMALL_TABLES_FILESIZE, 2048); // not selected resolved = resolver.resolveMapJoinTask(ctx, conf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java index 3a50356cf8f0..66f508e215ac 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java @@ -148,7 +148,7 @@ public void testInConversion() throws ParseException, CommandProcessorException "explain select sum(id_uv) from tu where u in (1,2) group by u"; HiveConf conf = env_setup.getTestCtx().hiveConf; - conf.setIntVar(ConfVars.HIVEPOINTLOOKUPOPTIMIZERMIN, 10); + conf.setIntVar(ConfVars.HIVE_POINT_LOOKUP_OPTIMIZER_MIN, 10); IDriver driver = createDriver(); PlanMapper pm = getMapperForQuery(driver, query); @@ -229,7 +229,7 @@ private static IDriver createDriver() { "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); // conf.setVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, CheckInputReadEntityDirect.class.getName()); HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - HiveConf.setVar(conf, HiveConf.ConfVars.POSTEXECHOOKS, OperatorStatsReaderHook.class.getName()); + HiveConf.setVar(conf, HiveConf.ConfVars.POST_EXEC_HOOKS, OperatorStatsReaderHook.class.getName()); SessionState.start(conf); IDriver driver = DriverFactory.newDriver(conf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java index e5fcc3a0d762..dbe7e967f374 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java @@ -190,7 +190,7 @@ private boolean compareOperators(Operator opL, Operator opR) { private static IDriver createDriver() { HiveConf conf = env_setup.getTestCtx().hiveConf; - conf.setBoolVar(ConfVars.HIVEOPTPPD, false); + conf.setBoolVar(ConfVars.HIVE_OPT_PPD, false); conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, true); conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, false); conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS, true); @@ -202,7 +202,7 @@ private static IDriver createDriver() { conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - HiveConf.setVar(conf, HiveConf.ConfVars.POSTEXECHOOKS, OperatorStatsReaderHook.class.getName()); + HiveConf.setVar(conf, HiveConf.ConfVars.POST_EXEC_HOOKS, OperatorStatsReaderHook.class.getName()); SessionState.start(conf); IDriver driver = DriverFactory.newDriver(conf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java index b67385737ef1..43e6a820f020 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java @@ -303,7 +303,7 @@ private static IDriver createDriver(String strategies) { conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - HiveConf.setVar(conf, HiveConf.ConfVars.POSTEXECHOOKS, OperatorStatsReaderHook.class.getName()); + HiveConf.setVar(conf, HiveConf.ConfVars.POST_EXEC_HOOKS, OperatorStatsReaderHook.class.getName()); SessionState.start(conf); IDriver driver = DriverFactory.newDriver(conf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java index ddbbef0b0134..8a993686a690 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java @@ -65,13 +65,13 @@ public void setupTest() throws Exception { public void testHiddenConfig() throws Exception { runSetProcessor(""); String output = baos.toString(); - Assert.assertFalse(output.contains(HiveConf.ConfVars.METASTOREPWD.varname + "=")); + Assert.assertFalse(output.contains(HiveConf.ConfVars.METASTORE_PWD.varname + "=")); Assert.assertFalse(output.contains(HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname + "=")); } @Test public void testHiddenConfigSetVarName() throws CommandProcessorException { - runSetProcessor(HiveConf.ConfVars.METASTOREPWD.varname); + runSetProcessor(HiveConf.ConfVars.METASTORE_PWD.varname); String output = baos.toString(); Assert.assertTrue(output.contains("hidden")); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java index 410d9b7d9184..0f762da0abf0 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHivePrivilegeObjectOwnerNameAndType.java @@ -77,8 +77,8 @@ public static void beforeTest() throws Exception { conf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false); conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, true); conf.setVar(ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName()); - conf.setVar(ConfVars.HIVEMAPREDMODE, "nonstrict"); - conf.setVar(ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + conf.setVar(ConfVars.HIVE_MAPRED_MODE, "nonstrict"); + conf.setVar(ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); TestTxnDbUtil.prepDb(conf); SessionState.start(conf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java index 3a59ad54c4a0..c91622611954 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java @@ -53,7 +53,7 @@ processedConf, new HadoopDefaultAuthenticator(), getCLISessionCtx() // check that hook to disable transforms has not been added assertFalse("Check for transform query disabling hook", - processedConf.getVar(ConfVars.PREEXECHOOKS).contains(DisallowTransformHook.class.getName())); + processedConf.getVar(ConfVars.PRE_EXEC_HOOKS).contains(DisallowTransformHook.class.getName())); // verify that some dummy param can be set processedConf.verifyAndSet("dummy.param", "dummy.val"); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java index b087d3beab26..1fc3663e75bd 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerHS2.java @@ -62,7 +62,7 @@ public void testConfigProcessing() throws HiveAuthzPluginException, SecurityExce // check that hook to disable transforms has been added assertTrue("Check for transform query disabling hook", - processedConf.getVar(ConfVars.PREEXECHOOKS).contains(DisallowTransformHook.class.getName())); + processedConf.getVar(ConfVars.PRE_EXEC_HOOKS).contains(DisallowTransformHook.class.getName())); List settableParams = getSettableParams(); verifyParamSettability(settableParams, processedConf); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java b/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java index 4c374e8d418a..9f93f096d382 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java @@ -24,7 +24,6 @@ import java.io.File; import java.io.IOException; -import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.util.Arrays; @@ -205,7 +204,7 @@ private void generateRefreshJarFiles(String version) throws IOException, Interru @Test public void testReloadAuxJars2() { HiveConf conf = new HiveConf(); - HiveConf.setVar(conf, ConfVars.HIVERELOADABLEJARS, hiveReloadPath); + HiveConf.setVar(conf, ConfVars.HIVE_RELOADABLE_JARS, hiveReloadPath); SessionState ss = new SessionState(conf); SessionState.start(ss); @@ -275,7 +274,7 @@ public void testReflectionCleanup() throws Exception { @Test public void testReloadExistingAuxJars2() { HiveConf conf = new HiveConf(); - HiveConf.setVar(conf, ConfVars.HIVERELOADABLEJARS, hiveReloadPath); + HiveConf.setVar(conf, ConfVars.HIVE_RELOADABLE_JARS, hiveReloadPath); SessionState ss = new SessionState(conf); SessionState.start(ss); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java index add1b0b82bca..e2bfa9763b7f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java @@ -74,10 +74,10 @@ String getTestDataDir() { @Before public void setUp() throws Exception { this.hiveConf = new HiveConf(TestStatsUpdaterThread.class); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, getTestDataDir()); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); + hiveConf.set(HiveConf.ConfVars.PRE_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POST_EXEC_HOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, getTestDataDir()); + hiveConf.setVar(HiveConf.ConfVars.HIVE_INPUT_FORMAT, HiveInputFormat.class.getName()); hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); // hiveConf.setBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK, true); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java b/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java index 386a90fd9445..a10a6aa30f7a 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java @@ -84,7 +84,7 @@ public void testUpgrade() throws Exception { "--oldWarehouseRoot", oldWarehouse}; HiveConf newConf = new HiveConf(hiveConf); File newWarehouseDir = new File(getTestDataDir(), "newWarehouse"); - newConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, newWarehouseDir.getAbsolutePath()); + newConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, newWarehouseDir.getAbsolutePath()); newConf.set("strict.managed.tables.migration.owner", System.getProperty("user.name")); runMigrationTool(newConf, args); @@ -124,7 +124,7 @@ public void testExternalMove() throws Exception { HiveConf newConf = new HiveConf(hiveConf); File newManagedWarehouseDir = new File(getTestDataDir(), "newManaged"); File newExtWarehouseDir = new File(getTestDataDir(), "newExternal"); - newConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, newManagedWarehouseDir.getAbsolutePath()); + newConf.set(HiveConf.ConfVars.METASTORE_WAREHOUSE.varname, newManagedWarehouseDir.getAbsolutePath()); newConf.set(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname, newExtWarehouseDir.getAbsolutePath()); runMigrationTool(newConf, args); Assert.assertTrue(newExtWarehouseDir.exists()); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/util/TestUpgradeTool.java b/ql/src/test/org/apache/hadoop/hive/ql/util/TestUpgradeTool.java index 7974da839123..4f24454056b7 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/util/TestUpgradeTool.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/util/TestUpgradeTool.java @@ -47,7 +47,7 @@ protected String getTestDataDir() { public void testPostUpgrade() throws Exception { int[][] data = {{1, 2}, {3, 4}, {5, 6}}; int[][] dataPart = {{1, 2, 10}, {3, 4, 11}, {5, 6, 12}}; - hiveConf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "dynamic"); + hiveConf.setVar(HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "dynamic"); runStatementOnDriver("drop table if exists TAcid"); runStatementOnDriver("drop table if exists TAcidPart"); runStatementOnDriver("drop table if exists TFlat"); diff --git a/ql/src/test/results/clientnegative/groupby_cube_multi_gby.q.out b/ql/src/test/results/clientnegative/groupby_cube_multi_gby.q.out index 88b87bb25846..e2491900dd38 100644 --- a/ql/src/test/results/clientnegative/groupby_cube_multi_gby.q.out +++ b/ql/src/test/results/clientnegative/groupby_cube_multi_gby.q.out @@ -14,4 +14,4 @@ POSTHOOK: query: create table t2 like src POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t2 -FAILED: SemanticException [Error 10315]: Grouping sets aggregations (with rollups or cubes) are not allowed when HIVEMULTIGROUPBYSINGLEREDUCER is turned on. Set hive.multigroupby.singlereducer=false if you want to use grouping sets +FAILED: SemanticException [Error 10315]: Grouping sets aggregations (with rollups or cubes) are not allowed when HIVE_MULTI_GROUPBY_SINGLE_REDUCER is turned on. Set hive.multigroupby.singlereducer=false if you want to use grouping sets diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java b/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java index 1512a0eea0e6..bd0e9f5ed27a 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java @@ -43,7 +43,7 @@ public void initialize(Configuration hconf, Properties props) throws SerDeExcept } private AbstractSerDe initializeSerde(Configuration conf, Properties props) throws SerDeException { - String serdeName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE); + String serdeName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_FETCH_OUTPUT_SERDE); Class serdeClass; try { serdeClass = diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java index c14a041ba013..e4151a617cdd 100644 --- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -139,7 +139,7 @@ public HiveSessionImpl(SessionHandle sessionHandle, TProtocolVersion protocol, this.operationLock = serverConf.getBoolVar( ConfVars.HIVE_SERVER2_PARALLEL_OPS_IN_SESSION) ? null : new Semaphore(1); // Set an explicit session name to control the download directory name - sessionConf.set(ConfVars.HIVESESSIONID.varname, + sessionConf.set(ConfVars.HIVE_SESSION_ID.varname, this.sessionHandle.getHandleIdentifier().toString()); // Use thrift transportable formatter sessionConf.set(SerDeUtils.LIST_SINK_OUTPUT_FORMATTER, ThriftFormatter.class.getName()); @@ -468,7 +468,7 @@ public String getPassword() { @Override public HiveConf getHiveConf() { - sessionConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); + sessionConf.setVar(HiveConf.ConfVars.HIVE_FETCH_OUTPUT_SERDE, FETCH_WORK_SERDE_CLASS); return sessionConf; } diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java index e8eaab550aef..e1650e86f6be 100644 --- a/service/src/java/org/apache/hive/service/server/HiveServer2.java +++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java @@ -52,7 +52,6 @@ import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.recipes.leader.LeaderLatch; import org.apache.curator.framework.recipes.leader.LeaderLatchListener; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.JvmPauseMonitor; import org.apache.hadoop.hive.common.LogUtils; import org.apache.hadoop.hive.common.LogUtils.LogInitializationException; @@ -106,7 +105,6 @@ import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.AuthType; import org.apache.hive.service.auth.saml.HiveSaml2Client; -import org.apache.hive.service.auth.saml.HiveSamlUtils; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.session.HiveSession; @@ -1044,7 +1042,7 @@ public static void scheduleClearDanglingScratchDir(HiveConf hiveConf, int initia .daemon(true) .build()); executor.scheduleAtFixedRate(new ClearDanglingScratchDir(false, false, false, - HiveConf.getVar(hiveConf, HiveConf.ConfVars.SCRATCHDIR), hiveConf), initialWaitInSec, + HiveConf.getVar(hiveConf, HiveConf.ConfVars.SCRATCH_DIR), hiveConf), initialWaitInSec, HiveConf.getTimeVar(hiveConf, ConfVars.HIVE_SERVER2_CLEAR_DANGLING_SCRATCH_DIR_INTERVAL, TimeUnit.SECONDS), TimeUnit.SECONDS); } diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java index c19d97abe8f4..e1fbdfcdca3a 100644 --- a/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java +++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java @@ -105,7 +105,7 @@ public void testTempSessionFileCleanup() throws Exception { private String[] getPipeoutFiles(HiveConf hiveConf) { File localScratchDir = new File( - hiveConf.getVar(HiveConf.ConfVars.LOCALSCRATCHDIR)); + hiveConf.getVar(HiveConf.ConfVars.LOCAL_SCRATCH_DIR)); String[] pipeoutFiles = localScratchDir.list(new FilenameFilter() { @Override public boolean accept(File dir, String name) { diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java index a3356658c12e..7f31dce576fb 100644 --- a/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java +++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java @@ -26,7 +26,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.hadoop.hive.common.metrics.MetricsTestUtils; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; import org.apache.hadoop.hive.common.metrics.common.MetricsFactory; @@ -44,7 +43,6 @@ import org.apache.hive.service.cli.operation.MetadataOperation; import org.apache.hive.service.cli.operation.OperationManager; import org.apache.hive.service.rpc.thrift.TProtocolVersion; -import org.apache.hive.service.server.HiveServer2; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -76,7 +74,7 @@ public void setup() throws Exception { conf.setBoolVar(HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true); conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); conf.setVar(HiveConf.ConfVars.HIVE_METRICS_REPORTER, MetricsReporting.JSON_FILE.name() + "," + MetricsReporting.JMX.name()); - conf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false); + conf.setBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_QUERIES, false); //NOTES: If we enable operation log, SessionManager will delete operation logs directory on exit, //it maybe impact TestSessionCleanup, because they use the same location ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION, // when we run testing in parallel on local machine with -DforkCount=x, it happen. diff --git a/service/src/test/org/apache/hive/service/server/TestHS2HttpServer.java b/service/src/test/org/apache/hive/service/server/TestHS2HttpServer.java index 6c50e8170901..a1e30d3bb5e9 100644 --- a/service/src/test/org/apache/hive/service/server/TestHS2HttpServer.java +++ b/service/src/test/org/apache/hive/service/server/TestHS2HttpServer.java @@ -76,7 +76,7 @@ public static void beforeTests() throws Exception { Integer.valueOf(ConfVars.HIVE_SERVER2_WEBUI_PORT.getDefaultValue())); apiBaseURL = "http://localhost:" + webUIPort + "/api/v1"; hiveConf = new HiveConf(); - hiveConf.set(ConfVars.METASTOREPWD.varname, metastorePasswd); + hiveConf.set(ConfVars.METASTORE_PWD.varname, metastorePasswd); hiveConf.set(ConfVars.HIVE_SERVER2_WEBUI_PORT.varname, webUIPort.toString()); hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); @@ -298,7 +298,7 @@ public void testConfStrippedFromWebUI() throws Exception { if (line.contains(metastorePasswd)) { pwdValFound = line; } - if (line.contains(ConfVars.METASTOREPWD.varname)) { + if (line.contains(ConfVars.METASTORE_PWD.varname)) { pwdKeyFound = line; } } diff --git a/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java index 04f66b4e5d73..92706af7033a 100644 --- a/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java +++ b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java @@ -35,19 +35,8 @@ import org.junit.Test; import javax.security.sasl.AuthenticationException; -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; -import java.security.KeyStore; -import java.security.KeyStoreException; -import java.security.NoSuchAlgorithmException; -import java.security.cert.Certificate; -import java.security.cert.CertificateException; -import java.security.cert.CertificateFactory; import java.util.HashMap; import java.util.Map; @@ -68,7 +57,7 @@ public static void beforeTests() throws Exception { MetaStoreTestUtils.findFreePortExcepting(Integer.valueOf(ConfVars.HIVE_SERVER2_WEBUI_PORT.getDefaultValue())); hiveConf = new HiveConf(); hiveConf.setBoolVar(ConfVars.HIVE_IN_TEST, true); - hiveConf.set(ConfVars.METASTOREPWD.varname, metastorePasswd); + hiveConf.set(ConfVars.METASTORE_PWD.varname, metastorePasswd); hiveConf.set(ConfVars.HIVE_SERVER2_WEBUI_PORT.varname, webUIPort.toString()); hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java index 3e2ad22bc15a..cf1fc236ce86 100644 --- a/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java +++ b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java @@ -66,7 +66,7 @@ public static void beforeTests() throws Exception { hiveConf = new HiveConf(); hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_PAM, true); hiveConf.setBoolVar(ConfVars.HIVE_IN_TEST, false); - hiveConf.set(ConfVars.METASTOREPWD.varname, metastorePasswd); + hiveConf.set(ConfVars.METASTORE_PWD.varname, metastorePasswd); hiveConf.set(ConfVars.HIVE_SERVER2_WEBUI_PORT.varname, webUIPort.toString()); hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java index a12b989c7307..53889e2b6f07 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java @@ -141,7 +141,7 @@ * * An implementation of RawStore that verifies the DummyJdoConnectionUrlHook has already been * applied when this class's setConf method is called, by checking that the value of the - * METASTORECONNECTURLKEY ConfVar has been updated. + * METASTORE_CONNECT_URL_KEY ConfVar has been updated. * * All non-void methods return default values. */ diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java index a6e510e5c4db..380863b716c6 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java @@ -1181,10 +1181,10 @@ public void testConcurrentAddNotifications() throws ExecutionException, Interrup */ // conf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, "com.mysql.jdbc.Driver"); -// conf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY, +// conf.setVar(HiveConf.ConfVars.METASTORE_CONNECT_URL_KEY, // "jdbc:mysql://localhost:3306/metastore_db"); // conf.setVar(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME, ""); -// conf.setVar(HiveConf.ConfVars.METASTOREPWD, ""); +// conf.setVar(HiveConf.ConfVars.METASTORE_PWD, ""); /* we have to add this one manually as for tests the db is initialized via the metastoreDiretSQL diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java index 4c4905deb0aa..658677751c7d 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRetriesInRetryingHMSHandler.java @@ -85,7 +85,7 @@ public void testNoRetryInit() throws MetaException { /* * If the init method in HMSHandler throws exception all the times it should be retried until - * HiveConf.ConfVars.HMSHANDLERATTEMPTS is reached before giving up + * HiveConf.ConfVars.HMS_HANDLER_ATTEMPTS is reached before giving up */ @Test public void testRetriesLimit() throws MetaException { diff --git a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java index 6d78e70cffbb..73accd90b034 100644 --- a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java +++ b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java @@ -147,7 +147,7 @@ public void init(StreamingConnection conn, long minWriteId, long maxWriteId, this.curBatchMaxWriteId = maxWriteId; this.statementId = statementId; this.conf = conn.getHiveConf(); - this.defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME); + this.defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); this.table = conn.getTable(); String location = table.getSd().getLocation(); try { diff --git a/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java b/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java index a61beb55b8fd..41e356b93da3 100644 --- a/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java +++ b/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java @@ -827,7 +827,7 @@ private void overrideConfSettings(HiveConf conf) { setHiveConf(conf, HiveConf.ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName()); setHiveConf(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); setHiveConf(conf, MetastoreConf.ConfVars.EXECUTE_SET_UGI.getHiveName()); - setHiveConf(conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict"); + setHiveConf(conf, HiveConf.ConfVars.DYNAMIC_PARTITIONING_MODE, "nonstrict"); if (streamingOptimizations) { setHiveConf(conf, HiveConf.ConfVars.HIVE_ORC_DELTA_STREAMING_OPTIMIZATIONS_ENABLED, true); } diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java b/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java index c548ea7388a6..c8c7a8e26db0 100644 --- a/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java +++ b/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java @@ -513,7 +513,7 @@ public void testDPTwoLevelNonStringPartitionColumns() throws Exception { assertEquals("7\tfoo\t" + defaultPartitionName + "\t" + defaultPartitionName, res.get(6)); assertEquals("8\tbar\t" + defaultPartitionName + "\t12", res.get(7)); - defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME); + defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); res = queryTable(driver, "show partitions " + (dbName + "." + tblName)); assertEquals(5, res.size()); assertTrue(res.contains("year=2018/month=2")); @@ -573,7 +573,7 @@ public void testWriteBeforeBegin() throws Exception { assertTrue(exception.getMessage().equals("Transaction state is not OPEN. Missing beginTransaction?")); connection.close(); - String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); List res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id"); assertEquals(2, res.size()); assertEquals("1\tfoo\tAsia\t" + defaultPartitionName, res.get(0)); @@ -707,7 +707,7 @@ public void testWriteAfterClose() throws Exception { assertNotNull(exception); assertTrue(exception.getMessage().endsWith("Streaming connection is closed already.")); - String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); List res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id"); assertEquals(2, res.size()); assertEquals("1\tfoo\tAsia\t" + defaultPartitionName, res.get(0)); @@ -758,7 +758,7 @@ public void testWriteAfterAbort() throws Exception { } assertNotNull(exception); assertTrue(exception.getMessage().equals("Streaming connection is closed already.")); - String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME); + String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); List res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id"); assertEquals(3, res.size()); assertEquals("1\tfoo\tAsia\t" + defaultPartitionName, res.get(0)); From 96d46dc36cfd3a68c73f8c77e1f97c1c78507b24 Mon Sep 17 00:00:00 2001 From: yigress <104102129+yigress@users.noreply.github.com> Date: Wed, 3 Jan 2024 17:03:59 -0800 Subject: [PATCH 119/179] HIVE-27951: hcatalog dynamic partitioning fails with partition already exist error when exist parent partitions path (#4937) --- .../FileOutputCommitterContainer.java | 37 ++++++++++++------- .../mapreduce/TestHCatDynamicPartitioned.java | 17 ++++++--- .../TestHCatExternalDynamicPartitioned.java | 4 +- 3 files changed, 37 insertions(+), 21 deletions(-) diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java index de9ad252ff24..2ad306165d1b 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java @@ -488,7 +488,7 @@ private void updateTableSchema(IMetaStoreClient client, Table table, } /** - * Move all of the files from the temp directory to the final location + * Move task output from the temp directory to the final location * @param srcf the file to move * @param srcDir the source directory * @param destDir the target directory @@ -538,17 +538,17 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, final Path finalOutputPath = getFinalPath(destFs, srcF, srcDir, destDir, immutable); if (immutable && destFs.exists(finalOutputPath) && !org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(destFs, finalOutputPath)) { - throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, - "Data already exists in " + finalOutputPath - + ", duplicate publish not possible."); - } - if (srcStatus.isDirectory()) { + if (partitionsDiscoveredByPath.containsKey(srcF.toString())) { + throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, + "Data already exists in " + finalOutputPath + + ", duplicate publish not possible."); + } + // parent directory may exist for multi-partitions, check lower level partitions + Collections.addAll(srcQ, srcFs.listStatus(srcF,HIDDEN_FILES_PATH_FILTER)); + } else if (srcStatus.isDirectory()) { if (canRename && dynamicPartitioningUsed) { // If it is partition, move the partition directory instead of each file. - // If custom dynamic location provided, need to rename to final output path - final Path parentDir = finalOutputPath.getParent(); - Path dstPath = !customDynamicLocationUsed ? parentDir : finalOutputPath; - moves.add(Pair.of(srcF, dstPath)); + moves.add(Pair.of(srcF, finalOutputPath)); } else { Collections.addAll(srcQ, srcFs.listStatus(srcF, HIDDEN_FILES_PATH_FILTER)); } @@ -558,16 +558,27 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, } } - if (moves.isEmpty()) { + bulkMoveFiles(conf, srcFs, destFs, moves); + } + + /** + * Bulk move files from source to destination. + * @param srcFs the source filesystem where the source files are + * @param destFs the destionation filesystem where the destionation files are + * @param pairs list of pairs of , move source_path to destination_path + * @throws java.io.IOException + */ + private void bulkMoveFiles(final Configuration conf, final FileSystem srcFs, final FileSystem destFs, List> pairs) throws IOException{ + if (pairs.isEmpty()) { return; } - + final boolean canRename = srcFs.getUri().equals(destFs.getUri()); final List>> futures = new LinkedList<>(); final ExecutorService pool = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ? Executors.newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25), new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Move-Thread-%d").build()) : null; - for (final Pair pair: moves){ + for (final Pair pair: pairs){ Path srcP = pair.getLeft(); Path dstP = pair.getRight(); final String msg = "Unable to move source " + srcP + " to destination " + dstP; diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java index a97162de993a..5ee3a6348d1e 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java @@ -52,13 +52,13 @@ public class TestHCatDynamicPartitioned extends HCatMapReduceTest { private static List dataColumns; private static final Logger LOG = LoggerFactory.getLogger(TestHCatDynamicPartitioned.class); protected static final int NUM_RECORDS = 20; - protected static final int NUM_PARTITIONS = 5; + protected static final int NUM_TOP_PARTITIONS = 5; public TestHCatDynamicPartitioned(String formatName, String serdeClass, String inputFormatClass, String outputFormatClass) throws Exception { super(formatName, serdeClass, inputFormatClass, outputFormatClass); tableName = "testHCatDynamicPartitionedTable_" + formatName; - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); generateDataColumns(); } @@ -67,6 +67,8 @@ protected static void generateDataColumns() throws HCatException { dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""))); + dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, ""))); + } protected static void generateWriteRecords(int max, int mod, int offset) { @@ -78,6 +80,7 @@ protected static void generateWriteRecords(int max, int mod, int offset) { objList.add(i); objList.add("strvalue" + i); objList.add(String.valueOf((i % mod) + offset)); + objList.add(String.valueOf((i / (max/2)) + offset)); writeRecords.add(new DefaultHCatRecord(objList)); } } @@ -86,6 +89,7 @@ protected static void generateWriteRecords(int max, int mod, int offset) { protected List getPartitionKeys() { List fields = new ArrayList(); fields.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, "")); + fields.add(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, "")); return fields; } @@ -117,8 +121,9 @@ public void testHCatDynamicPartitionedTableMultipleTask() throws Exception { protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, String customDynamicPathPattern) throws Exception { - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); - runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, true, asSingleMapTask, customDynamicPathPattern); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); + runMRCreate(null, dataColumns, writeRecords.subList(0,NUM_RECORDS/2), NUM_RECORDS/2, true, asSingleMapTask, customDynamicPathPattern); + runMRCreate(null, dataColumns, writeRecords.subList(NUM_RECORDS/2,NUM_RECORDS), NUM_RECORDS/2, true, asSingleMapTask, customDynamicPathPattern); runMRRead(NUM_RECORDS); @@ -140,7 +145,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, //Test for duplicate publish IOException exc = null; try { - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); Job job = runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, false, true, customDynamicPathPattern); @@ -167,7 +172,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, driver.run(query); res = new ArrayList(); driver.getResults(res); - assertEquals(NUM_PARTITIONS, res.size()); + assertEquals(NUM_TOP_PARTITIONS*2, res.size()); query = "select * from " + tableName; driver.run(query); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java index 18fcfdbdd2a8..9698f178a8ed 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java @@ -28,7 +28,7 @@ public TestHCatExternalDynamicPartitioned(String formatName, String serdeClass, throws Exception { super(formatName, serdeClass, inputFormatClass, outputFormatClass); tableName = "testHCatExternalDynamicPartitionedTable_" + formatName; - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); generateDataColumns(); } @@ -43,7 +43,7 @@ protected Boolean isTableExternal() { */ @Test public void testHCatExternalDynamicCustomLocation() throws Exception { - runHCatDynamicPartitionedTable(true, "mapred/externalDynamicOutput/${p1}"); + runHCatDynamicPartitionedTable(true, "mapred/externalDynamicOutput/${p1}/{p2}"); } } From 1f2495ead58c8cf03fb55af4ba0c114eee7d3acc Mon Sep 17 00:00:00 2001 From: Akshat Mathur Date: Thu, 4 Jan 2024 18:33:48 +0530 Subject: [PATCH 120/179] HIVE-27969: Add verbose logging for schema initialisation and metastore service (#4972) (Akshat Mathur, reviewed by Zsolt Miskolczi, Zhihua Deng, Attila Turoczy, Kokila N) --- packaging/src/docker/README.md | 8 ++++++++ packaging/src/docker/entrypoint.sh | 7 ++++--- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/packaging/src/docker/README.md b/packaging/src/docker/README.md index c206914ef4d2..ebcbfcaf617d 100644 --- a/packaging/src/docker/README.md +++ b/packaging/src/docker/README.md @@ -88,9 +88,17 @@ Launch the HiveServer2 with an embedded Metastore, docker run -d -p 10000:10000 -p 10002:10002 --env SERVICE_NAME=hiveserver2 \ --env SERVICE_OPTS="-Dhive.metastore.uris=thrift://metastore:9083" \ --env IS_RESUME="true" \ + --env VERBOSE="true" \ --name hiveserver2-standalone apache/hive:${HIVE_VERSION} ``` + +NOTE: + +To skip schematool initialisation or upgrade for metastore use `IS_RESUME="true"`, and for verbose logging set `VERBOSE="true"` + + To save the data between container restarts, you can start the HiveServer2 with a Volume, + ```shell docker run -d -p 10000:10000 -p 10002:10002 --env SERVICE_NAME=hiveserver2 \ --env SERVICE_OPTS="-Dhive.metastore.uris=thrift://metastore:9083" \ diff --git a/packaging/src/docker/entrypoint.sh b/packaging/src/docker/entrypoint.sh index a19b50d8daad..f22652c2afb6 100644 --- a/packaging/src/docker/entrypoint.sh +++ b/packaging/src/docker/entrypoint.sh @@ -22,13 +22,14 @@ set -x : ${DB_DRIVER:=derby} SKIP_SCHEMA_INIT="${IS_RESUME:-false}" +[[ $VERBOSE = "true" ]] && VERBOSE_MODE="--verbose" || VERBOSE_MODE="" function initialize_hive { COMMAND="-initOrUpgradeSchema" if [ "$(echo "$HIVE_VER" | cut -d '.' -f1)" -lt "4" ]; then COMMAND="-${SCHEMA_COMMAND:-initSchema}" fi - $HIVE_HOME/bin/schematool -dbType $DB_DRIVER $COMMAND + $HIVE_HOME/bin/schematool -dbType $DB_DRIVER $COMMAND $VERBOSE_MODE if [ $? -eq 0 ]; then echo "Initialized schema successfully.." else @@ -53,8 +54,8 @@ fi if [ "${SERVICE_NAME}" == "hiveserver2" ]; then export HADOOP_CLASSPATH=$TEZ_HOME/*:$TEZ_HOME/lib/*:$HADOOP_CLASSPATH + exec $HIVE_HOME/bin/hive --skiphadoopversion --skiphbasecp --service $SERVICE_NAME elif [ "${SERVICE_NAME}" == "metastore" ]; then export METASTORE_PORT=${METASTORE_PORT:-9083} + exec $HIVE_HOME/bin/hive --skiphadoopversion --skiphbasecp $VERBOSE_MODE --service $SERVICE_NAME fi - -exec $HIVE_HOME/bin/hive --skiphadoopversion --skiphbasecp --service $SERVICE_NAME From 3ba23a022f9e57b9a03afb48c8c2032698006f96 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Fri, 5 Jan 2024 03:48:51 +0100 Subject: [PATCH 121/179] Revert "HIVE-27951: hcatalog dynamic partitioning fails with partition already exist error when exist parent partitions path (#4937)" This reverts commit 96d46dc36cfd3a68c73f8c77e1f97c1c78507b24. --- .../FileOutputCommitterContainer.java | 37 +++++++------------ .../mapreduce/TestHCatDynamicPartitioned.java | 17 +++------ .../TestHCatExternalDynamicPartitioned.java | 4 +- 3 files changed, 21 insertions(+), 37 deletions(-) diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java index 2ad306165d1b..de9ad252ff24 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java @@ -488,7 +488,7 @@ private void updateTableSchema(IMetaStoreClient client, Table table, } /** - * Move task output from the temp directory to the final location + * Move all of the files from the temp directory to the final location * @param srcf the file to move * @param srcDir the source directory * @param destDir the target directory @@ -538,17 +538,17 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, final Path finalOutputPath = getFinalPath(destFs, srcF, srcDir, destDir, immutable); if (immutable && destFs.exists(finalOutputPath) && !org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(destFs, finalOutputPath)) { - if (partitionsDiscoveredByPath.containsKey(srcF.toString())) { - throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, - "Data already exists in " + finalOutputPath - + ", duplicate publish not possible."); - } - // parent directory may exist for multi-partitions, check lower level partitions - Collections.addAll(srcQ, srcFs.listStatus(srcF,HIDDEN_FILES_PATH_FILTER)); - } else if (srcStatus.isDirectory()) { + throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, + "Data already exists in " + finalOutputPath + + ", duplicate publish not possible."); + } + if (srcStatus.isDirectory()) { if (canRename && dynamicPartitioningUsed) { // If it is partition, move the partition directory instead of each file. - moves.add(Pair.of(srcF, finalOutputPath)); + // If custom dynamic location provided, need to rename to final output path + final Path parentDir = finalOutputPath.getParent(); + Path dstPath = !customDynamicLocationUsed ? parentDir : finalOutputPath; + moves.add(Pair.of(srcF, dstPath)); } else { Collections.addAll(srcQ, srcFs.listStatus(srcF, HIDDEN_FILES_PATH_FILTER)); } @@ -558,27 +558,16 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, } } - bulkMoveFiles(conf, srcFs, destFs, moves); - } - - /** - * Bulk move files from source to destination. - * @param srcFs the source filesystem where the source files are - * @param destFs the destionation filesystem where the destionation files are - * @param pairs list of pairs of , move source_path to destination_path - * @throws java.io.IOException - */ - private void bulkMoveFiles(final Configuration conf, final FileSystem srcFs, final FileSystem destFs, List> pairs) throws IOException{ - if (pairs.isEmpty()) { + if (moves.isEmpty()) { return; } - final boolean canRename = srcFs.getUri().equals(destFs.getUri()); + final List>> futures = new LinkedList<>(); final ExecutorService pool = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ? Executors.newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25), new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Move-Thread-%d").build()) : null; - for (final Pair pair: pairs){ + for (final Pair pair: moves){ Path srcP = pair.getLeft(); Path dstP = pair.getRight(); final String msg = "Unable to move source " + srcP + " to destination " + dstP; diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java index 5ee3a6348d1e..a97162de993a 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java @@ -52,13 +52,13 @@ public class TestHCatDynamicPartitioned extends HCatMapReduceTest { private static List dataColumns; private static final Logger LOG = LoggerFactory.getLogger(TestHCatDynamicPartitioned.class); protected static final int NUM_RECORDS = 20; - protected static final int NUM_TOP_PARTITIONS = 5; + protected static final int NUM_PARTITIONS = 5; public TestHCatDynamicPartitioned(String formatName, String serdeClass, String inputFormatClass, String outputFormatClass) throws Exception { super(formatName, serdeClass, inputFormatClass, outputFormatClass); tableName = "testHCatDynamicPartitionedTable_" + formatName; - generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); generateDataColumns(); } @@ -67,8 +67,6 @@ protected static void generateDataColumns() throws HCatException { dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""))); - dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, ""))); - } protected static void generateWriteRecords(int max, int mod, int offset) { @@ -80,7 +78,6 @@ protected static void generateWriteRecords(int max, int mod, int offset) { objList.add(i); objList.add("strvalue" + i); objList.add(String.valueOf((i % mod) + offset)); - objList.add(String.valueOf((i / (max/2)) + offset)); writeRecords.add(new DefaultHCatRecord(objList)); } } @@ -89,7 +86,6 @@ protected static void generateWriteRecords(int max, int mod, int offset) { protected List getPartitionKeys() { List fields = new ArrayList(); fields.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, "")); - fields.add(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, "")); return fields; } @@ -121,9 +117,8 @@ public void testHCatDynamicPartitionedTableMultipleTask() throws Exception { protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, String customDynamicPathPattern) throws Exception { - generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); - runMRCreate(null, dataColumns, writeRecords.subList(0,NUM_RECORDS/2), NUM_RECORDS/2, true, asSingleMapTask, customDynamicPathPattern); - runMRCreate(null, dataColumns, writeRecords.subList(NUM_RECORDS/2,NUM_RECORDS), NUM_RECORDS/2, true, asSingleMapTask, customDynamicPathPattern); + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, true, asSingleMapTask, customDynamicPathPattern); runMRRead(NUM_RECORDS); @@ -145,7 +140,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, //Test for duplicate publish IOException exc = null; try { - generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); Job job = runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, false, true, customDynamicPathPattern); @@ -172,7 +167,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, driver.run(query); res = new ArrayList(); driver.getResults(res); - assertEquals(NUM_TOP_PARTITIONS*2, res.size()); + assertEquals(NUM_PARTITIONS, res.size()); query = "select * from " + tableName; driver.run(query); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java index 9698f178a8ed..18fcfdbdd2a8 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java @@ -28,7 +28,7 @@ public TestHCatExternalDynamicPartitioned(String formatName, String serdeClass, throws Exception { super(formatName, serdeClass, inputFormatClass, outputFormatClass); tableName = "testHCatExternalDynamicPartitionedTable_" + formatName; - generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); generateDataColumns(); } @@ -43,7 +43,7 @@ protected Boolean isTableExternal() { */ @Test public void testHCatExternalDynamicCustomLocation() throws Exception { - runHCatDynamicPartitionedTable(true, "mapred/externalDynamicOutput/${p1}/{p2}"); + runHCatDynamicPartitionedTable(true, "mapred/externalDynamicOutput/${p1}"); } } From b7cf4ffb41a02b326cf51e3b3d3dc2cbe201e620 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 5 Jan 2024 10:01:45 +0100 Subject: [PATCH 122/179] HIVE-27916: Increase tez.am.resource.memory.mb for TestIcebergCliDrver (#4907) (Laszlo Bodor reviewed by Ayush Saxena) --- data/conf/iceberg/tez/tez-site.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data/conf/iceberg/tez/tez-site.xml b/data/conf/iceberg/tez/tez-site.xml index fff6c8756203..3c2a96f7edbe 100644 --- a/data/conf/iceberg/tez/tez-site.xml +++ b/data/conf/iceberg/tez/tez-site.xml @@ -5,7 +5,7 @@ tez.am.resource.memory.mb - 256 + 512 tez.runtime.io.sort.mb From bc10f65b112301dba51082d50c02b6f4d1ea14fd Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Fri, 5 Jan 2024 19:55:28 +0100 Subject: [PATCH 123/179] HIVE-27982: TestConcurrentDppInserts fails on master - disable test --- .../apache/hadoop/hive/ql/exec/TestConcurrentDppInserts.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestConcurrentDppInserts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestConcurrentDppInserts.java index 1288c10d32ec..8595b25bc670 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestConcurrentDppInserts.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestConcurrentDppInserts.java @@ -34,6 +34,7 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestRule; @@ -42,6 +43,7 @@ import com.google.common.base.Joiner; +@Ignore public class TestConcurrentDppInserts { static final private Logger LOG = LoggerFactory.getLogger(TestConcurrentDppInserts.class.getName()); From 30495dd6d9d453ee3ae92b436d45b0c29c39b30d Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sat, 6 Jan 2024 05:25:47 +0100 Subject: [PATCH 124/179] HIVE-27978: Tests in hive-unit module are not running again (#4977) (Laszlo Bodor reviewed by Ayush Saxena, Stamatis Zampetakis, Zsolt Miskolczi, Dayakar M) --- itests/hive-unit/pom.xml | 7 ------- .../ql/parse/TestTimedOutTxnNotificationLogging.java | 10 +++++----- 2 files changed, 5 insertions(+), 12 deletions(-) diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index b89c3f05dd46..7c3b2333b9eb 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -27,7 +27,6 @@ ../.. 1.15.2 2.70.0 - 5.6.2 @@ -495,12 +494,6 @@ postgresql test - - org.junit.jupiter - junit-jupiter - ${junit.jupiter.version} - test - diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java index 888b8ee0f40b..559699cf3c36 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java @@ -44,6 +44,7 @@ import org.apache.hive.hcatalog.listener.DbNotificationListener; import org.apache.thrift.TException; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,7 +53,6 @@ import java.util.*; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.jupiter.api.Assertions.assertEquals; @RunWith(Parameterized.class) @@ -126,13 +126,13 @@ public void tearDown() throws Exception { public void testTxnNotificationLogging() throws Exception { try { List txnIds = openTxns(numberOfTxns, txnType); - assertEquals(txnIds.size(), getNumberOfTxnsWithTxnState(txnIds, TxnState.OPEN)); - assertEquals(expectedNotifications, getNumberOfNotificationsWithEventType(txnIds, MessageBuilder.OPEN_TXN_EVENT)); + Assert.assertEquals(txnIds.size(), getNumberOfTxnsWithTxnState(txnIds, TxnState.OPEN)); + Assert.assertEquals(expectedNotifications, getNumberOfNotificationsWithEventType(txnIds, MessageBuilder.OPEN_TXN_EVENT)); Thread.sleep(1000); acidHouseKeeperService.run(); //this will abort timed-out txns if (txnType != TxnType.REPL_CREATED) { - assertEquals(txnIds.size(), getNumberOfTxnsWithTxnState(txnIds, TxnState.ABORTED)); - assertEquals(expectedNotifications, getNumberOfNotificationsWithEventType(txnIds, MessageBuilder.ABORT_TXN_EVENT)); + Assert.assertEquals(txnIds.size(), getNumberOfTxnsWithTxnState(txnIds, TxnState.ABORTED)); + Assert.assertEquals(expectedNotifications, getNumberOfNotificationsWithEventType(txnIds, MessageBuilder.ABORT_TXN_EVENT)); } } finally { runCleanerServices(); From 0eea2a36b440e7c017e0a1a565ebdfb64c723abc Mon Sep 17 00:00:00 2001 From: rkirtir <111496669+rkirtir@users.noreply.github.com> Date: Mon, 8 Jan 2024 12:48:50 +0530 Subject: [PATCH 125/179] HIVE-27911 : Drop database query failing with Invalid ACL Exception (#4901) (Kirti Ruge reviewed by Laszlo Bodor) --- .../impl/LlapZookeeperRegistryImpl.java | 10 ++++++ .../hive/registry/impl/ZkRegistryBase.java | 31 ++++++++++++++++++- .../impl/TestLlapZookeeperRegistryImpl.java | 12 +++++++ 3 files changed, 52 insertions(+), 1 deletion(-) diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java index 25cb89a37567..1c4df16b7643 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java @@ -168,6 +168,16 @@ public String register() throws IOException { String uniqueId = UNIQUE_ID.toString(); long znodeCreationTimeout = 120; + /* When no LLAP executors are running, the getInstances()/ llapRegistryService.getInstances() method results in + InvalidACL exception when Hive tries to evict the cache in DROP DATABASE/TABLE + + As ContainerManager of zookeeper makes sure to cleanup znodes periodically, once query-coordinator zookeeper + client session is terminated,entire path will get deleted sooner or later. This results in InvalidACL exception + + PersistentNode created on server will be preserved through restarts of query executor and makes sure + proactive eviction call of DROP DATABASE/ DROP TABLE go through + successfully. */ + ensurePersistentNodePath(daemonZkRecord); initializeWithoutRegisteringInternal(); // Create a znode under the rootNamespace parent for this instance of the server try { diff --git a/llap-client/src/java/org/apache/hadoop/hive/registry/impl/ZkRegistryBase.java b/llap-client/src/java/org/apache/hadoop/hive/registry/impl/ZkRegistryBase.java index 9da200f2e928..f6a52cd3b2d0 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/registry/impl/ZkRegistryBase.java +++ b/llap-client/src/java/org/apache/hadoop/hive/registry/impl/ZkRegistryBase.java @@ -29,6 +29,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.ACLProvider; import org.apache.curator.framework.imps.CuratorFrameworkState; @@ -55,6 +57,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.InvalidACLException; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.ZooDefs; @@ -114,6 +117,8 @@ public abstract class ZkRegistryBase { private PersistentNode znode; private String znodePath; // unique identity for this instance + final String namespace; + private PathChildrenCache instancesCache; // Created on demand. /** Local hostname. */ @@ -160,7 +165,7 @@ public ZkRegistryBase(String instanceName, Configuration conf, String rootNs, St this.stateChangeListeners = new HashSet<>(); this.pathToInstanceCache = new ConcurrentHashMap<>(); this.nodeToInstanceCache = new ConcurrentHashMap<>(); - final String namespace = getRootNamespace(conf, rootNs, nsPrefix); + this.namespace = getRootNamespace(conf, rootNs, nsPrefix); ACLProvider aclProvider; // get acl provider for most outer path that is non-null if (userPathPrefix == null) { @@ -353,6 +358,30 @@ protected final void updateServiceRecord( } } + @VisibleForTesting + public String getPersistentNodePath() { + return "/" + PATH_JOINER.join(namespace, StringUtils.substringBetween(workersPath, "/", "/"), "pnode0"); + } + + protected void ensurePersistentNodePath(ServiceRecord srv) throws IOException { + String pNodePath = getPersistentNodePath(); + try { + LOG.info("Check if persistent node path {} exists, create if not", pNodePath); + if (zooKeeperClient.checkExists().forPath(pNodePath) == null) { + zooKeeperClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(pNodePath, encoder.toBytes(srv)); + LOG.info("Created persistent path at: {}", pNodePath); + } + } catch (Exception e) { + // throw exception if it is other than NODEEXISTS. + if (!(e instanceof KeeperException) || ((KeeperException) e).code() != KeeperException.Code.NODEEXISTS) { + LOG.error("Unable to create a persistent znode for this server instance", e); + throw new IOException(e); + } else { + LOG.debug("Ignoring KeeperException while ensuring path as the parent node {} already exists.", pNodePath); + } + } + } final protected void initializeWithoutRegisteringInternal() throws IOException { // Create a znode under the rootNamespace parent for this instance of the server diff --git a/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestLlapZookeeperRegistryImpl.java b/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestLlapZookeeperRegistryImpl.java index 50799f2f5cab..7b227d4356bc 100644 --- a/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestLlapZookeeperRegistryImpl.java +++ b/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestLlapZookeeperRegistryImpl.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.llap.registry.LlapServiceInstance; import org.apache.hadoop.hive.registry.ServiceInstanceSet; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -47,6 +48,7 @@ public class TestLlapZookeeperRegistryImpl { private CuratorFramework curatorFramework; private TestingServer server; + private final static String NAMESPACE_PREFIX = "llap-"; @Before public void setUp() throws Exception { @@ -124,6 +126,16 @@ public void testUpdate() throws Exception { attributes.get(LlapRegistryService.LLAP_DAEMON_NUM_ENABLED_EXECUTORS)); } + @Test + public void testPersistentNodePath() { + String llapRootNameSpace = "/" + LlapZookeeperRegistryImpl.getRootNamespace(hiveConf, + HiveConf.getVar(hiveConf, HiveConf.ConfVars.LLAP_ZK_REGISTRY_NAMESPACE), NAMESPACE_PREFIX); + String persistentNodeName = "/pnode0"; + + Assert.assertEquals(llapRootNameSpace + "/user-" + System.getProperty("user.name") + persistentNodeName, + registry.getPersistentNodePath()); + } + static void trySetMock(Object o, String field, T value) { try { Field fieldToChange = Arrays.stream(FieldUtils.getAllFields(o.getClass())) From 9a9e9a3f277b14d856b2ebc00da9b89604e74130 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 8 Jan 2024 10:02:23 +0100 Subject: [PATCH 126/179] HIVE-27937: Clarifying comments around tez container size (#4920) (Laszlo Bodor reviewed by Stamatis Zampetakis, Denys Kuzmenko) --- .../org/apache/hadoop/hive/conf/HiveConf.java | 15 ++++++++++++++- data/conf/iceberg/llap/hive-site.xml | 8 +++++--- data/conf/iceberg/llap/tez-site.xml | 4 ---- data/conf/iceberg/tez/hive-site.xml | 3 +-- data/conf/iceberg/tez/tez-site.xml | 4 ---- data/conf/llap/hive-site.xml | 4 +++- data/conf/llap/tez-site.xml | 4 ---- data/conf/tez/tez-site.xml | 4 ---- .../apache/hadoop/hive/ql/exec/tez/DagUtils.java | 16 +++++++++------- .../ql/exec/tez/TezAvailableSlotsCalculator.java | 9 ++++++++- 10 files changed, 40 insertions(+), 31 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index cbe91a509ff2..9a1433dc21da 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2461,7 +2461,20 @@ public static enum ConfVars { "The default input format for tez. Tez groups splits in the AM."), HIVE_TEZ_CONTAINER_SIZE("hive.tez.container.size", -1, - "By default Tez will spawn containers of the size of a mapper. This can be used to overwrite."), + "The memory in MB that's used by a Tez task container (TezChild) in Tez container mode. Hive uses this \n" + + "property to create a Resource object which is accepted by Yarn (and used in TezAM to ask for TezChild \n" + + "containers). This should be distinguished from the Tez AM's (DAGAppMaster) memory, \n" + + "which is driven by tez.am.resource.memory.mb! \n" + + "Also, as Hive takes care of TezChild memory by setting this option, there is no need \n " + + "to set tez.task.resource.memory.mb differently. \n" + + "The final -Xmx arg for TezChild process is not equal to this setting, \n " + + "because Tez considers a heap fraction (80%), so by default: \n" + + "Xmx = hive.tez.container.size * tez.container.max.java.heap.fraction. \n" + + "In case of values <= 0, container size falls back to mapreduce.map.memory.mb. \n" + + "LLAP notes: while generating splits, the needed per-task resource is derived from this option \n" + + "(refer to HiveSplitGenerator, TezAvailableSlotsCalculator), so even if its value doesn't change the \n" + + "LLAP daemons' total physical size, it has to be configured properly. In this context \n" + + "4096 implies that you assume a single task will consume 4096MB from a daemon's shared heap."), HIVE_TEZ_CPU_VCORES("hive.tez.cpu.vcores", -1, "By default Tez will ask for however many cpus map-reduce is configured to use per container.\n" + "This can be used to overwrite."), diff --git a/data/conf/iceberg/llap/hive-site.xml b/data/conf/iceberg/llap/hive-site.xml index 57982980abf3..fb941d991c49 100644 --- a/data/conf/iceberg/llap/hive-site.xml +++ b/data/conf/iceberg/llap/hive-site.xml @@ -38,10 +38,12 @@ A base for other temporary directories. + + + - hive.tez.container.size - 128 - + hive.tez.container.size + 128 diff --git a/data/conf/iceberg/llap/tez-site.xml b/data/conf/iceberg/llap/tez-site.xml index fff6c8756203..b96019bb37fa 100644 --- a/data/conf/iceberg/llap/tez-site.xml +++ b/data/conf/iceberg/llap/tez-site.xml @@ -11,10 +11,6 @@ tez.runtime.io.sort.mb 24 - - hive.tez.container.size - 512 - tez.counters.max 1024 diff --git a/data/conf/iceberg/tez/hive-site.xml b/data/conf/iceberg/tez/hive-site.xml index 0dc31fed5edd..272859dd915d 100644 --- a/data/conf/iceberg/tez/hive-site.xml +++ b/data/conf/iceberg/tez/hive-site.xml @@ -40,8 +40,7 @@ hive.tez.container.size - 128 - + 512 diff --git a/data/conf/iceberg/tez/tez-site.xml b/data/conf/iceberg/tez/tez-site.xml index 3c2a96f7edbe..7b26d6aad3e8 100644 --- a/data/conf/iceberg/tez/tez-site.xml +++ b/data/conf/iceberg/tez/tez-site.xml @@ -11,10 +11,6 @@ tez.runtime.io.sort.mb 24 - - hive.tez.container.size - 512 - tez.counters.max 1024 diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml index 27609b7a2be3..fe69d81de7e3 100644 --- a/data/conf/llap/hive-site.xml +++ b/data/conf/llap/hive-site.xml @@ -44,10 +44,12 @@ A base for other temporary directories. + + + hive.tez.container.size 128 - diff --git a/data/conf/llap/tez-site.xml b/data/conf/llap/tez-site.xml index 6118e6edf8eb..cc4177e3cbd2 100644 --- a/data/conf/llap/tez-site.xml +++ b/data/conf/llap/tez-site.xml @@ -9,10 +9,6 @@ tez.am.resource.memory.mb 128 - - tez.task.resource.memory.mb - 128 - tez.runtime.io.sort.mb 24 diff --git a/data/conf/tez/tez-site.xml b/data/conf/tez/tez-site.xml index 88adb6a57e80..ff3b468fca33 100644 --- a/data/conf/tez/tez-site.xml +++ b/data/conf/tez/tez-site.xml @@ -3,10 +3,6 @@ tez.am.resource.memory.mb 512 - - tez.task.resource.memory.mb - 128 - tez.runtime.io.sort.mb 24 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java index 896be0018d79..09d2ff719333 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; +import com.google.common.base.Strings; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; @@ -686,7 +687,9 @@ public static Resource getContainerResource(Configuration conf) { cpuCores = MRJobConfig.DEFAULT_MAP_CPU_VCORES; } } - return Resource.newInstance(memorySizeMb, cpuCores); + Resource resource = Resource.newInstance(memorySizeMb, cpuCores); + LOG.debug("Tez container resource: {}", resource); + return resource; } /* @@ -716,19 +719,18 @@ private static String getContainerJavaOpts(Configuration conf) { } logLevel = sb.toString(); + String finalOpts = null; if (HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TEZ_CONTAINER_SIZE) > 0) { - if (javaOpts != null) { - return javaOpts + " " + logLevel; - } else { - return logLevel; - } + finalOpts = Strings.nullToEmpty(javaOpts) + " " + logLevel; } else { if (javaOpts != null && !javaOpts.isEmpty()) { LOG.warn(HiveConf.ConfVars.HIVE_TEZ_JAVA_OPTS + " will be ignored because " + HiveConf.ConfVars.HIVE_TEZ_CONTAINER_SIZE + " is not set!"); } - return logLevel + " " + MRHelpers.getJavaOptsForMRMapper(conf); + finalOpts = logLevel + " " + MRHelpers.getJavaOptsForMRMapper(conf); } + LOG.debug("Tez container final opts: {}", finalOpts); + return finalOpts; } private Vertex createVertexFromMergeWork(JobConf conf, MergeJoinWork mergeJoinWork, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezAvailableSlotsCalculator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezAvailableSlotsCalculator.java index 731ad0942f81..e0117a08ba6d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezAvailableSlotsCalculator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezAvailableSlotsCalculator.java @@ -20,11 +20,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.tez.runtime.api.InputInitializerContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Default implementation of AvailableSlotsCalculator which relies on available capacity of the cluster */ public class TezAvailableSlotsCalculator implements AvailableSlotsCalculator { + private static final Logger LOG = LoggerFactory.getLogger(TezAvailableSlotsCalculator.class); + private InputInitializerContext inputInitializerContext; @Override public void initialize(Configuration conf, HiveSplitGenerator splitGenerator) { @@ -39,6 +43,9 @@ public int getAvailableSlots() { } int totalResource = inputInitializerContext.getTotalAvailableResource().getMemory(); int taskResource = inputInitializerContext.getVertexTaskResource().getMemory(); - return totalResource / taskResource; + int availableSlots = totalResource / taskResource;; + LOG.debug("totalResource: {}mb / taskResource: {}mb = availableSlots: {}", totalResource, taskResource, + availableSlots); + return availableSlots; } } From 24fffdc508f9402ad7145b59b50de738b27c92b4 Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Mon, 8 Jan 2024 14:07:57 +0100 Subject: [PATCH 127/179] HIVE-27948: Wrong results when using materialized views with non-deterministic/dynamic functions (Krisztian Kasa, reviewed by Stamatis Zampetakis) --- ...AlterMaterializedViewRewriteOperation.java | 13 +++- .../show/ShowMaterializedViewsFormatter.java | 2 +- .../apache/hadoop/hive/ql/metadata/Hive.java | 6 +- .../HiveMaterializedViewsRegistry.java | 14 ++--- .../metadata/HiveRelOptMaterialization.java | 31 ++-------- .../MaterializationValidationResult.java | 41 +++++++++++++ .../hive/ql/metadata/RewriteAlgorithm.java | 44 +++++++++++++ ...rializedViewASTSubQueryRewriteShuttle.java | 6 +- .../HiveRelOptMaterializationValidator.java | 61 ++++++++++--------- .../apache/hadoop/hive/ql/parse/CBOPlan.java | 13 ++-- .../hadoop/hive/ql/parse/CalcitePlanner.java | 6 +- .../hadoop/hive/ql/parse/ParseUtils.java | 3 +- .../hive/ql/parse/SemanticAnalyzer.java | 28 ++++----- .../metadata/TestMaterializedViewsCache.java | 2 +- .../materialized_view_no_cbo_rewrite.q} | 0 .../materialized_view_rewrite_by_text_10.q | 11 ++++ .../materialized_view_rewrite_by_text_11.q} | 0 .../materialized_view_no_cbo_rewrite.q.out} | 6 +- ...materialized_view_rewrite_by_text_10.q.out | 40 ++++++++++++ ...aterialized_view_rewrite_by_text_11.q.out} | 6 +- .../materialized_view_rewrite_by_text_8.q.out | 4 +- 21 files changed, 235 insertions(+), 102 deletions(-) create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializationValidationResult.java create mode 100644 ql/src/java/org/apache/hadoop/hive/ql/metadata/RewriteAlgorithm.java rename ql/src/test/queries/{clientnegative/materialized_view_no_cbo_rewrite_2.q => clientpositive/materialized_view_no_cbo_rewrite.q} (100%) create mode 100644 ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_10.q rename ql/src/test/queries/{clientnegative/materialized_view_no_supported_op_rewrite_2.q => clientpositive/materialized_view_rewrite_by_text_11.q} (100%) rename ql/src/test/results/{clientnegative/materialized_view_no_cbo_rewrite_2.q.out => clientpositive/llap/materialized_view_no_cbo_rewrite.q.out} (87%) create mode 100644 ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_10.q.out rename ql/src/test/results/{clientnegative/materialized_view_no_supported_op_rewrite_2.q.out => clientpositive/llap/materialized_view_rewrite_by_text_11.q.out} (87%) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rewrite/AlterMaterializedViewRewriteOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rewrite/AlterMaterializedViewRewriteOperation.java index 4f2b6cccc6eb..f4ada77ba3c4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rewrite/AlterMaterializedViewRewriteOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rewrite/AlterMaterializedViewRewriteOperation.java @@ -25,11 +25,15 @@ import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.ddl.DDLOperation; import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; +import org.apache.hadoop.hive.ql.metadata.MaterializationValidationResult; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.CalcitePlanner; import org.apache.hadoop.hive.ql.parse.ParseUtils; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.hadoop.hive.ql.processors.CompileProcessor.console; + /** * Operation process of enabling/disabling materialized view rewrite. */ @@ -64,9 +68,12 @@ public int execute() throws HiveException { } throw new HiveException(msg); } - if (!planner.isValidAutomaticRewritingMaterialization()) { - throw new HiveException("Cannot enable rewriting for materialized view. " + - planner.getInvalidAutomaticRewritingMaterializationReason()); + MaterializationValidationResult validationResult = planner.getMaterializationValidationResult(); + String validationErrorMessage = validationResult.getErrorMessage(); + if (validationResult.getSupportedRewriteAlgorithms().isEmpty()) { + throw new HiveException(validationErrorMessage); + } else if (isNotBlank(validationErrorMessage)) { + console.printError(validationErrorMessage); } } catch (Exception e) { throw new HiveException(e); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsFormatter.java index 7423f9067c06..c8c43af74c43 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsFormatter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/show/ShowMaterializedViewsFormatter.java @@ -41,7 +41,7 @@ import static org.apache.hadoop.hive.conf.Constants.MATERIALIZED_VIEW_REWRITING_TIME_WINDOW; import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.IncrementalRebuildMode.UNKNOWN; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.ALL; +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.ALL; /** * Formats SHOW MATERIALIZED VIEWS results. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index b3ef272bbfe5..11b52c51a730 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -40,8 +40,8 @@ import static org.apache.hadoop.hive.ql.ddl.DDLUtils.isIcebergStatsSource; import static org.apache.hadoop.hive.ql.ddl.DDLUtils.isIcebergTable; import static org.apache.hadoop.hive.ql.io.AcidUtils.getFullTableName; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.CALCITE; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.ALL; +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.CALCITE; +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.ALL; import static org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils.extractTable; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT; import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME; @@ -2236,7 +2236,7 @@ public HiveRelOptMaterialization getMaterializedViewForRebuild(String dbName, St private List getValidMaterializedViews(List
materializedViewTables, Set tablesUsed, boolean forceMVContentsUpToDate, boolean expandGroupingSets, - HiveTxnManager txnMgr, EnumSet scope) + HiveTxnManager txnMgr, EnumSet scope) throws HiveException { final String validTxnsList = conf.get(ValidTxnList.VALID_TXNS_KEY); final boolean tryIncrementalRewriting = diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java index ca11fcccffaa..9c5bdfe18af9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java @@ -83,9 +83,7 @@ import com.google.common.collect.ImmutableList; import static java.util.stream.Collectors.toList; -import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.ALL; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.TEXT; +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.ALL; /** * Registry for materialized views. The goal of this cache is to avoid parsing and creating @@ -236,9 +234,7 @@ public HiveRelOptMaterialization createMaterialization(HiveConf conf, Table mate } return new HiveRelOptMaterialization(viewScan, plan.getPlan(), - null, viewScan.getTable().getQualifiedName(), - isBlank(plan.getInvalidAutomaticRewritingMaterializationReason()) ? - EnumSet.allOf(HiveRelOptMaterialization.RewriteAlgorithm.class) : EnumSet.of(TEXT), + null, viewScan.getTable().getQualifiedName(), plan.getSupportedRewriteAlgorithms(), determineIncrementalRebuildMode(plan.getPlan()), plan.getAst()); } @@ -273,7 +269,7 @@ public void createMaterializedView(HiveConf conf, Table materializedViewTable) { } HiveRelOptMaterialization materialization = createMaterialization(conf, materializedViewTable); - if (materialization == null) { + if (materialization == null || materialization.getScope().isEmpty()) { return; } @@ -348,7 +344,7 @@ public void dropMaterializedView(String dbName, String tableName) { */ List getRewritingMaterializedViews() { return materializedViewsCache.values().stream() - .filter(materialization -> materialization.getScope().contains(HiveRelOptMaterialization.RewriteAlgorithm.CALCITE)) + .filter(materialization -> materialization.getScope().contains(RewriteAlgorithm.CALCITE)) .collect(toList()); } @@ -358,7 +354,7 @@ List getRewritingMaterializedViews() { * @return the collection of materialized views, or the empty collection if none */ public HiveRelOptMaterialization getRewritingMaterializedView(String dbName, String viewName, - EnumSet scope) { + EnumSet scope) { HiveRelOptMaterialization materialization = materializedViewsCache.get(dbName, viewName); if (materialization == null) { return null; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java index b4a20cea0fb1..04548bfb801a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java @@ -28,7 +28,7 @@ import java.util.EnumSet; import java.util.List; -import java.util.function.Predicate; +import java.util.Set; import static org.apache.commons.collections.CollectionUtils.intersection; @@ -37,27 +37,6 @@ */ public class HiveRelOptMaterialization extends RelOptMaterialization { - /** - * Enumeration of Materialized view query rewrite algorithms. - */ - public enum RewriteAlgorithm { - /** - * Query sql text is compared to stored materialized view definition sql texts. - */ - TEXT, - /** - * Use rewriting algorithm provided by Calcite. - */ - CALCITE; - - public static final EnumSet ALL = EnumSet.allOf(RewriteAlgorithm.class); - - public static final Predicate> ANY = - rewriteAlgorithms -> true; - public static final Predicate> NON_CALCITE = - rewriteAlgorithms -> !rewriteAlgorithms.contains(HiveRelOptMaterialization.RewriteAlgorithm.CALCITE); - } - public enum IncrementalRebuildMode { AVAILABLE, INSERT_ONLY, @@ -65,7 +44,7 @@ public enum IncrementalRebuildMode { UNKNOWN } - private final EnumSet scope; + private final Set scope; private final boolean sourceTablesUpdateDeleteModified; private final boolean sourceTablesCompacted; private final IncrementalRebuildMode rebuildMode; @@ -73,13 +52,13 @@ public enum IncrementalRebuildMode { public HiveRelOptMaterialization(RelNode tableRel, RelNode queryRel, RelOptTable starRelOptTable, List qualifiedTableName, - EnumSet scope, IncrementalRebuildMode rebuildMode, ASTNode ast) { + Set scope, IncrementalRebuildMode rebuildMode, ASTNode ast) { this(tableRel, queryRel, starRelOptTable, qualifiedTableName, scope, false, false, rebuildMode, ast); } private HiveRelOptMaterialization(RelNode tableRel, RelNode queryRel, RelOptTable starRelOptTable, List qualifiedTableName, - EnumSet scope, + Set scope, boolean sourceTablesUpdateDeleteModified, boolean sourceTablesCompacted, IncrementalRebuildMode rebuildMode, ASTNode ast) { super(tableRel, queryRel, starRelOptTable, qualifiedTableName); this.scope = scope; @@ -89,7 +68,7 @@ private HiveRelOptMaterialization(RelNode tableRel, RelNode queryRel, this.ast = ast; } - public EnumSet getScope() { + public Set getScope() { return scope; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializationValidationResult.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializationValidationResult.java new file mode 100644 index 000000000000..3f3372e9c6df --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializationValidationResult.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.metadata; + +import java.util.EnumSet; +import java.util.Set; + +public class MaterializationValidationResult { + private final EnumSet supportedRewriteAlgorithms; + private final String errorMessage; + + public MaterializationValidationResult( + EnumSet supportedRewriteAlgorithms, String errorMessage) { + this.supportedRewriteAlgorithms = supportedRewriteAlgorithms; + this.errorMessage = errorMessage; + } + + public Set getSupportedRewriteAlgorithms() { + return supportedRewriteAlgorithms; + } + + public String getErrorMessage() { + return errorMessage; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/RewriteAlgorithm.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/RewriteAlgorithm.java new file mode 100644 index 000000000000..308df7299373 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/RewriteAlgorithm.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.metadata; + +import java.util.EnumSet; +import java.util.Set; +import java.util.function.Predicate; + +/** + * Enumeration of Materialized view query rewrite algorithms. + */ +public enum RewriteAlgorithm { + /** + * Query sql text is compared to stored materialized view definition sql texts. + */ + TEXT, + /** + * Use rewriting algorithm provided by Calcite. + */ + CALCITE; + + public static final EnumSet ALL = EnumSet.allOf(RewriteAlgorithm.class); + + public static final Predicate> ANY = + rewriteAlgorithms -> true; + public static final Predicate> NON_CALCITE = + rewriteAlgorithms -> !rewriteAlgorithms.contains(CALCITE); +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java index 151a047c18e6..d24ad12331b6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization; +import org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject; @@ -36,7 +37,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayDeque; -import java.util.EnumSet; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -46,7 +46,7 @@ import static java.util.Collections.singletonList; import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableSet; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.NON_CALCITE; +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.NON_CALCITE; import static org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils.extractTable; /** @@ -140,7 +140,7 @@ public RelNode visit(HiveFilter filter) { public static RelNode getMaterializedViewByAST( ASTNode expandedAST, RelOptCluster optCluster, - Predicate> filter, + Predicate> filter, Hive db, Set tablesUsedByOriginalPlan, HiveTxnManager txnManager) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java index cf419b170c13..4f20609cff94 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java @@ -38,7 +38,8 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.util.Util; import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.metadata.MaterializationValidationResult; +import org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin; @@ -52,8 +53,10 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import java.util.EnumSet; + +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.TEXT; /** * Checks the query plan for conditions that would make the plan unsuitable for @@ -62,13 +65,16 @@ * - References to non-deterministic functions. */ public class HiveRelOptMaterializationValidator extends HiveRelShuttleImpl { - static final Logger LOG = LoggerFactory.getLogger(HiveRelOptMaterializationValidator.class); + private static final String UNSUPPORTED_BY_CALCITE_FORMAT = + "Only query text based automatic rewriting is available for materialized view. " + + "Statement has unsupported %s: %s."; protected String resultCacheInvalidReason; - protected String automaticRewritingInvalidReason; + protected MaterializationValidationResult materializationValidationResult; public void validate(RelNode relNode) { try { + materializationValidationResult = new MaterializationValidationResult(RewriteAlgorithm.ALL, ""); relNode.accept(this); } catch (Util.FoundOne e) { // Can ignore - the check failed. @@ -113,7 +119,7 @@ public RelNode visit(HiveFilter filter) { @Override public RelNode visit(HiveJoin join) { if (join.getJoinType() != JoinRelType.INNER) { - setAutomaticRewritingInvalidReason(join.getJoinType() + " join type is not supported by rewriting algorithm."); + unsupportedByCalciteRewrite("join type", join.getJoinType().toString()); } checkExpr(join.getCondition()); return super.visit(join); @@ -152,7 +158,7 @@ public RelNode visit(RelNode node) { @Override public RelNode visit(TableFunctionScan scan) { - setAutomaticRewritingInvalidReason(scan); + unsupportedByCalciteRewrite("expression", "window function"); checkExpr(scan.getCall()); return super.visit(scan); } @@ -231,13 +237,13 @@ public RelNode visit(LogicalExchange exchange) { // Note: Not currently part of the HiveRelNode interface private RelNode visit(HiveUnion union) { - setAutomaticRewritingInvalidReason("Statement has unsupported operator: union."); + unsupportedByCalciteRewrite("operator", "union"); return visitChildren(union); } @Override public RelNode visit(HiveSortLimit sort) { - setAutomaticRewritingInvalidReason("Statement has unsupported clause: order by."); + unsupportedByCalciteRewrite("clause","order by"); checkExpr(sort.getFetchExpr()); checkExpr(sort.getOffsetExpr()); return visitChildren(sort); @@ -245,20 +251,20 @@ public RelNode visit(HiveSortLimit sort) { // Note: Not currently part of the HiveRelNode interface private RelNode visit(HiveSortExchange sort) { - setAutomaticRewritingInvalidReason("Statement has unsupported clause: sort by."); + unsupportedByCalciteRewrite("clause", "sort by"); return visitChildren(sort); } // Note: Not currently part of the HiveRelNode interface private RelNode visit(HiveSemiJoin semiJoin) { - setAutomaticRewritingInvalidReason("Statement has unsupported join type: semi join."); + unsupportedByCalciteRewrite("join type", "semi join"); checkExpr(semiJoin.getCondition()); checkExpr(semiJoin.getJoinFilter()); return visitChildren(semiJoin); } private RelNode visit(HiveAntiJoin antiJoin) { - setAutomaticRewritingInvalidReason("Statement has unsupported join type: anti join."); + unsupportedByCalciteRewrite("join type", "anti join"); checkExpr(antiJoin.getCondition()); checkExpr(antiJoin.getJoinFilter()); return visitChildren(antiJoin); @@ -266,26 +272,30 @@ private RelNode visit(HiveAntiJoin antiJoin) { // Note: Not currently part of the HiveRelNode interface private RelNode visit(HiveExcept except) { - setAutomaticRewritingInvalidReason("Statement has unsupported operator: except."); + unsupportedByCalciteRewrite("operator", "except"); return visitChildren(except); } // Note: Not currently part of the HiveRelNode interface private RelNode visit(HiveIntersect intersect) { - setAutomaticRewritingInvalidReason("Statement has unsupported operator: intersect."); + unsupportedByCalciteRewrite("operator", "intersect"); return visitChildren(intersect); } private void fail(String reason) { setResultCacheInvalidReason(reason); - setAutomaticRewritingInvalidReason(reason); + this.materializationValidationResult = new MaterializationValidationResult( + EnumSet.noneOf(RewriteAlgorithm.class), "Cannot enable automatic rewriting for materialized view. " + reason); throw Util.FoundOne.NULL; } private RelNode fail(RelNode node) { setResultCacheInvalidReason("Unsupported RelNode type " + node.getRelTypeName() + " encountered in the query plan"); - setAutomaticRewritingInvalidReason(node); + this.materializationValidationResult = + new MaterializationValidationResult(EnumSet.noneOf(RewriteAlgorithm.class), + String.format("Cannot enable automatic rewriting for materialized view. " + + "Unsupported RelNode type %s encountered in the query plan", node.getRelTypeName())); throw Util.FoundOne.NULL; } @@ -308,24 +318,19 @@ public boolean isValidForQueryCaching() { return resultCacheInvalidReason == null; } - public String getAutomaticRewritingInvalidReason() { - return automaticRewritingInvalidReason; - } - - public void setAutomaticRewritingInvalidReason(String automaticRewritingInvalidReason) { - if (isValidForAutomaticRewriting()) { - this.automaticRewritingInvalidReason = automaticRewritingInvalidReason; - } + public MaterializationValidationResult getAutomaticRewritingValidationResult() { + return materializationValidationResult; } - public void setAutomaticRewritingInvalidReason(RelNode node) { + public void unsupportedByCalciteRewrite(String sqlPartType, String sqlPart) { if (isValidForAutomaticRewriting()) { - this.automaticRewritingInvalidReason = "Unsupported RelNode type " + node.getRelTypeName() + - " encountered in the query plan"; + String errorMessage = String.format(UNSUPPORTED_BY_CALCITE_FORMAT, sqlPartType, sqlPart); + this.materializationValidationResult = + new MaterializationValidationResult(EnumSet.of(TEXT), errorMessage); } } public boolean isValidForAutomaticRewriting() { - return automaticRewritingInvalidReason == null; + return RewriteAlgorithm.ALL.equals(materializationValidationResult.getSupportedRewriteAlgorithms()); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java index 4230ef07b4a6..53beeeb6df72 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java @@ -19,6 +19,9 @@ package org.apache.hadoop.hive.ql.parse; import org.apache.calcite.rel.RelNode; +import org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm; + +import java.util.Set; /** * Wrapper of Calcite plan. @@ -26,12 +29,12 @@ public class CBOPlan { private final ASTNode ast; private final RelNode plan; - private final String invalidAutomaticRewritingMaterializationReason; + private final Set supportedRewriteAlgorithms; - public CBOPlan(ASTNode ast, RelNode plan, String invalidAutomaticRewritingMaterializationReason) { + public CBOPlan(ASTNode ast, RelNode plan, Set supportedRewriteAlgorithms) { this.ast = ast; this.plan = plan; - this.invalidAutomaticRewritingMaterializationReason = invalidAutomaticRewritingMaterializationReason; + this.supportedRewriteAlgorithms = supportedRewriteAlgorithms; } public ASTNode getAst() { @@ -52,7 +55,7 @@ public RelNode getPlan() { * Null or empty string otherwise. * @return String contains error message or null. */ - public String getInvalidAutomaticRewritingMaterializationReason() { - return invalidAutomaticRewritingMaterializationReason; + public Set getSupportedRewriteAlgorithms() { + return supportedRewriteAlgorithms; } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java index c23e94e6e165..6bb6b0938436 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java @@ -355,7 +355,7 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.hadoop.hive.ql.optimizer.calcite.HiveMaterializedViewASTSubQueryRewriteShuttle.getMaterializedViewByAST; -import static org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.ANY; +import static org.apache.hadoop.hive.ql.metadata.RewriteAlgorithm.ANY; public class CalcitePlanner extends SemanticAnalyzer { @@ -1681,8 +1681,8 @@ public RelNode apply(RelOptCluster cluster, RelOptSchema relOptSchema, SchemaPlu materializationValidator.validate(calcitePlan); setInvalidResultCacheReason( materializationValidator.getResultCacheInvalidReason()); - setInvalidAutomaticRewritingMaterializationReason( - materializationValidator.getAutomaticRewritingInvalidReason()); + setMaterializationValidationResult( + materializationValidator.getAutomaticRewritingValidationResult()); // 2. Apply pre-join order optimizations calcitePlan = applyPreJoinOrderingTransforms(calcitePlan, mdProvider.getMetadataProvider(), executorProvider); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java index 972d7997673d..3feb18b9202a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java @@ -541,7 +541,8 @@ public static CBOPlan parseQuery(HiveConf conf, String viewQuery) final ASTNode ast = parse(viewQuery, ctx); final CalcitePlanner analyzer = getAnalyzer(conf, ctx); RelNode logicalPlan = analyzer.genLogicalPlan(ast); - return new CBOPlan(ast, logicalPlan, analyzer.getInvalidAutomaticRewritingMaterializationReason()); + return new CBOPlan( + ast, logicalPlan, analyzer.getMaterializationValidationResult().getSupportedRewriteAlgorithms()); } public static List parseQueryAndGetSchema(HiveConf conf, String viewQuery) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 3c4dd24e70b6..5b84437ecef4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -181,6 +181,7 @@ import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; import org.apache.hadoop.hive.ql.lockmgr.LockException; +import org.apache.hadoop.hive.ql.metadata.MaterializationValidationResult; import org.apache.hadoop.hive.ql.metadata.DefaultConstraint; import org.apache.hadoop.hive.ql.metadata.DummyPartition; import org.apache.hadoop.hive.ql.metadata.Hive; @@ -456,7 +457,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { HiveParser.TOK_DISTRIBUTEBY, HiveParser.TOK_SORTBY); private String invalidResultCacheReason; - private String invalidAutomaticRewritingMaterializationReason; + private MaterializationValidationResult materializationValidationResult; private final NullOrdering defaultNullOrder; @@ -14670,11 +14671,12 @@ private void validateCreateView() } throw new SemanticException(msg); } - if (!isValidAutomaticRewritingMaterialization()) { - String errorMessage = "Only query text based automatic rewriting is available for materialized view. " + - getInvalidAutomaticRewritingMaterializationReason(); + if (materializationValidationResult.getSupportedRewriteAlgorithms().isEmpty()) { + createVwDesc.setRewriteEnabled(false); + } + String errorMessage = materializationValidationResult.getErrorMessage(); + if (isNotBlank(errorMessage)) { console.printError(errorMessage); - LOG.warn(errorMessage); } } } catch (HiveException e) { @@ -15931,18 +15933,14 @@ public ColsAndTypes(String cols, String colTypes) { public String colTypes; } - public String getInvalidAutomaticRewritingMaterializationReason() { - return invalidAutomaticRewritingMaterializationReason; - } - - public void setInvalidAutomaticRewritingMaterializationReason( - String invalidAutomaticRewritingMaterializationReason) { - this.invalidAutomaticRewritingMaterializationReason = - invalidAutomaticRewritingMaterializationReason; + public MaterializationValidationResult getMaterializationValidationResult() { + return materializationValidationResult; } - public boolean isValidAutomaticRewritingMaterialization() { - return (invalidAutomaticRewritingMaterializationReason == null); + public void setMaterializationValidationResult( + MaterializationValidationResult materializationValidationResult) { + this.materializationValidationResult = + materializationValidationResult; } public String getInvalidResultCacheReason() { diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java index 34e85b471951..6978fd10004d 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java @@ -157,7 +157,7 @@ private Table getTable(String db, String tableName, String definition) { private static HiveRelOptMaterialization createMaterialization(Table table) throws ParseException { return new HiveRelOptMaterialization( new DummyRel(table), new DummyRel(table), null, asList(table.getDbName(), table.getTableName()), - EnumSet.allOf(HiveRelOptMaterialization.RewriteAlgorithm.class), + RewriteAlgorithm.ALL, HiveRelOptMaterialization.IncrementalRebuildMode.AVAILABLE, ParseUtils.parse(table.getViewExpandedText(), null)); } diff --git a/ql/src/test/queries/clientnegative/materialized_view_no_cbo_rewrite_2.q b/ql/src/test/queries/clientpositive/materialized_view_no_cbo_rewrite.q similarity index 100% rename from ql/src/test/queries/clientnegative/materialized_view_no_cbo_rewrite_2.q rename to ql/src/test/queries/clientpositive/materialized_view_no_cbo_rewrite.q diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_10.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_10.q new file mode 100644 index 000000000000..43deec4a8142 --- /dev/null +++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_10.q @@ -0,0 +1,11 @@ +-- Materialzed view definition has non-deterministic function +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +CREATE TABLE EMPS (ENAME STRING, BIRTH_EPOCH_SECS INT) STORED AS ORC TBLPROPERTIES ('transactional'='true'); + +CREATE MATERIALIZED VIEW v_emp AS SELECT * FROM EMPS WHERE BIRTH_EPOCH_SECS <= UNIX_TIMESTAMP(); + +-- View can not be used +explain cbo +SELECT * FROM EMPS WHERE BIRTH_EPOCH_SECS <= UNIX_TIMESTAMP(); diff --git a/ql/src/test/queries/clientnegative/materialized_view_no_supported_op_rewrite_2.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_11.q similarity index 100% rename from ql/src/test/queries/clientnegative/materialized_view_no_supported_op_rewrite_2.q rename to ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_11.q diff --git a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_no_cbo_rewrite.q.out similarity index 87% rename from ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out rename to ql/src/test/results/clientpositive/llap/materialized_view_no_cbo_rewrite.q.out index 9083c736d640..4cba8933ab55 100644 --- a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_no_cbo_rewrite.q.out @@ -36,4 +36,8 @@ PREHOOK: query: alter materialized view cmv_mat_view enable rewrite PREHOOK: type: ALTER_MATERIALIZED_VIEW_REWRITE PREHOOK: Input: default@cmv_mat_view PREHOOK: Output: default@cmv_mat_view -FAILED: Execution Error, return code 40000 from org.apache.hadoop.hive.ql.ddl.DDLTask. org.apache.hadoop.hive.ql.metadata.HiveException: Cannot enable rewriting for materialized view. Statement has unsupported clause: sort by. +Only query text based automatic rewriting is available for materialized view. Statement has unsupported clause: sort by. +POSTHOOK: query: alter materialized view cmv_mat_view enable rewrite +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REWRITE +POSTHOOK: Input: default@cmv_mat_view +POSTHOOK: Output: default@cmv_mat_view diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_10.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_10.q.out new file mode 100644 index 000000000000..3455a1fe9ee6 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_10.q.out @@ -0,0 +1,40 @@ +PREHOOK: query: CREATE TABLE EMPS (ENAME STRING, BIRTH_EPOCH_SECS INT) STORED AS ORC TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@EMPS +POSTHOOK: query: CREATE TABLE EMPS (ENAME STRING, BIRTH_EPOCH_SECS INT) STORED AS ORC TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@EMPS +unix_timestamp(void) is deprecated. Use current_timestamp instead. +unix_timestamp(void) is deprecated. Use current_timestamp instead. +Cannot enable automatic rewriting for materialized view. UNIX_TIMESTAMP is not a deterministic function +PREHOOK: query: CREATE MATERIALIZED VIEW v_emp AS SELECT * FROM EMPS WHERE BIRTH_EPOCH_SECS <= UNIX_TIMESTAMP() +PREHOOK: type: CREATE_MATERIALIZED_VIEW +PREHOOK: Input: default@emps +PREHOOK: Output: database:default +PREHOOK: Output: default@v_emp +POSTHOOK: query: CREATE MATERIALIZED VIEW v_emp AS SELECT * FROM EMPS WHERE BIRTH_EPOCH_SECS <= UNIX_TIMESTAMP() +POSTHOOK: type: CREATE_MATERIALIZED_VIEW +POSTHOOK: Input: default@emps +POSTHOOK: Output: database:default +POSTHOOK: Output: default@v_emp +POSTHOOK: Lineage: v_emp.birth_epoch_secs SIMPLE [(emps)emps.FieldSchema(name:birth_epoch_secs, type:int, comment:null), ] +POSTHOOK: Lineage: v_emp.ename SIMPLE [(emps)emps.FieldSchema(name:ename, type:string, comment:null), ] +unix_timestamp(void) is deprecated. Use current_timestamp instead. +unix_timestamp(void) is deprecated. Use current_timestamp instead. +PREHOOK: query: explain cbo +SELECT * FROM EMPS WHERE BIRTH_EPOCH_SECS <= UNIX_TIMESTAMP() +PREHOOK: type: QUERY +PREHOOK: Input: default@emps +#### A masked pattern was here #### +POSTHOOK: query: explain cbo +SELECT * FROM EMPS WHERE BIRTH_EPOCH_SECS <= UNIX_TIMESTAMP() +POSTHOOK: type: QUERY +POSTHOOK: Input: default@emps +#### A masked pattern was here #### +CBO PLAN: +HiveProject(ename=[$0], birth_epoch_secs=[$1]) + HiveFilter(condition=[<=(CAST($1):BIGINT, UNIX_TIMESTAMP())]) + HiveTableScan(table=[[default, emps]], table:alias=[emps]) + diff --git a/ql/src/test/results/clientnegative/materialized_view_no_supported_op_rewrite_2.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_11.q.out similarity index 87% rename from ql/src/test/results/clientnegative/materialized_view_no_supported_op_rewrite_2.q.out rename to ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_11.q.out index bb771614eec0..bd12d44dc3b9 100644 --- a/ql/src/test/results/clientnegative/materialized_view_no_supported_op_rewrite_2.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_11.q.out @@ -35,4 +35,8 @@ PREHOOK: query: alter materialized view cmv_mat_view enable rewrite PREHOOK: type: ALTER_MATERIALIZED_VIEW_REWRITE PREHOOK: Input: default@cmv_mat_view PREHOOK: Output: default@cmv_mat_view -FAILED: Execution Error, return code 40000 from org.apache.hadoop.hive.ql.ddl.DDLTask. org.apache.hadoop.hive.ql.metadata.HiveException: Cannot enable rewriting for materialized view. LEFT join type is not supported by rewriting algorithm. +Only query text based automatic rewriting is available for materialized view. Statement has unsupported join type: LEFT. +POSTHOOK: query: alter materialized view cmv_mat_view enable rewrite +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REWRITE +POSTHOOK: Input: default@cmv_mat_view +POSTHOOK: Output: default@cmv_mat_view diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out index fa31a16a2650..e003990db64a 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out @@ -8,7 +8,7 @@ POSTHOOK: query: create table t1(col0 int) STORED AS ORC POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t1 -Only query text based automatic rewriting is available for materialized view. LEFT join type is not supported by rewriting algorithm. +Only query text based automatic rewriting is available for materialized view. Statement has unsupported join type: LEFT. Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: create materialized view mat1 as select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where l.col0 = 20 @@ -23,7 +23,7 @@ POSTHOOK: Input: default@t1 POSTHOOK: Output: database:default POSTHOOK: Output: default@mat1 POSTHOOK: Lineage: mat1.col0 SIMPLE [] -Only query text based automatic rewriting is available for materialized view. LEFT join type is not supported by rewriting algorithm. +Only query text based automatic rewriting is available for materialized view. Statement has unsupported join type: LEFT. Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: create materialized view mat2 as select col0 from From 0c3b8223564cef7d70d915c855fb8b969517e262 Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Tue, 9 Jan 2024 09:37:42 +0800 Subject: [PATCH 128/179] HIVE-27857: Do not check write permission while dropping external table or partition (#4860) (Wechar Yu, Reviewed by Ayush Saxena, Sai Hemanth Gantasala) --- .../metadata/SessionHiveMetaStoreClient.java | 2 +- .../hadoop/hive/metastore/HMSHandler.java | 30 +++++++------- .../TestTablesCreateDropAlterTruncate.java | 40 +++++++++++++++++++ 3 files changed, 56 insertions(+), 16 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java index 31efa27abec1..0e3dfb281b44 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java @@ -823,7 +823,7 @@ private void dropTempTable(org.apache.hadoop.hive.metastore.api.Table table, boo if (pathStr != null) { try { tablePath = new Path(table.getSd().getLocation()); - if (!getWh().isWritable(tablePath.getParent())) { + if (deleteData && !isExternalTable(table) && !getWh().isWritable(tablePath.getParent())) { throw new MetaException("Table metadata not deleted since " + tablePath.getParent() + " is not writable by " + SecurityUtils.getUser()); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index 7100bf93ae15..54eb8b3c2560 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -2969,7 +2969,7 @@ private boolean drop_table_core(final RawStore ms, final String catName, final S firePreEvent(new PreDropTableEvent(tbl, deleteData, this)); tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(tbl, deleteData); - if (tbl.getSd().getLocation() != null) { + if (tableDataShouldBeDeleted && tbl.getSd().getLocation() != null) { tblPath = new Path(tbl.getSd().getLocation()); if (!wh.isWritable(tblPath.getParent())) { String target = indexName == null ? "Table" : "Index table"; @@ -4971,6 +4971,7 @@ private boolean drop_partition_common(RawStore ms, String catName, String db_nam Table tbl = null; Partition part = null; boolean mustPurge = false; + boolean tableDataShouldBeDeleted = false; long writeId = 0; Map transactionalListenerResponses = Collections.emptyMap(); @@ -4994,7 +4995,8 @@ private boolean drop_partition_common(RawStore ms, String catName, String db_nam request.setCatName(catName); tbl = get_table_core(request); firePreEvent(new PreDropPartitionEvent(tbl, part, deleteData, this)); - + + tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(tbl, deleteData); mustPurge = isMustPurge(envContext, tbl); writeId = getWriteId(envContext); @@ -5002,12 +5004,12 @@ private boolean drop_partition_common(RawStore ms, String catName, String db_nam throw new NoSuchObjectException("Partition doesn't exist. " + part_vals); } isArchived = MetaStoreUtils.isArchived(part); - if (isArchived) { + if (tableDataShouldBeDeleted && isArchived) { archiveParentDir = MetaStoreUtils.getOriginalLocation(part); verifyIsWritablePath(archiveParentDir); } - if ((part.getSd() != null) && (part.getSd().getLocation() != null)) { + if (tableDataShouldBeDeleted && (part.getSd() != null) && (part.getSd().getLocation() != null)) { partPath = new Path(part.getSd().getLocation()); verifyIsWritablePath(partPath); } @@ -5027,9 +5029,7 @@ private boolean drop_partition_common(RawStore ms, String catName, String db_nam } finally { if (!success) { ms.rollbackTransaction(); - } else if (checkTableDataShouldBeDeleted(tbl, deleteData) && - (partPath != null || archiveParentDir != null)) { - + } else if (tableDataShouldBeDeleted && (partPath != null || archiveParentDir != null)) { LOG.info(mustPurge ? "dropPartition() will purge " + partPath + " directly, skipping trash." : "dropPartition() will move " + partPath + " to trash-directory."); @@ -5159,7 +5159,7 @@ public DropPartitionsResult drop_partitions_req( boolean deleteData = request.isSetDeleteData() && request.isDeleteData(); boolean ignoreProtection = request.isSetIgnoreProtection() && request.isIgnoreProtection(); boolean needResult = !request.isSetNeedResult() || request.isNeedResult(); - + List dirsToDelete = new ArrayList<>(); List archToDelete = new ArrayList<>(); EnvironmentContext envContext = @@ -5169,19 +5169,21 @@ public DropPartitionsResult drop_partitions_req( Table tbl = null; List parts = null; boolean mustPurge = false; + boolean tableDataShouldBeDeleted = false; long writeId = 0; Map transactionalListenerResponses = null; boolean needsCm = false; - + try { ms.openTransaction(); // We need Partition-s for firing events and for result; DN needs MPartition-s to drop. // Great... Maybe we could bypass fetching MPartitions by issuing direct SQL deletes. tbl = get_table_core(catName, dbName, tblName); mustPurge = isMustPurge(envContext, tbl); + tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(tbl, deleteData); writeId = getWriteId(envContext); - + int minCount = 0; RequestPartsSpec spec = request.getParts(); List partNames = null; @@ -5245,14 +5247,12 @@ public DropPartitionsResult drop_partitions_req( if (colNames != null) { partNames.add(FileUtils.makePartName(colNames, part.getValues())); } - // Preserve the old behavior of failing when we cannot write, even w/o deleteData, - // and even if the table is external. That might not make any sense. - if (MetaStoreUtils.isArchived(part)) { + if (tableDataShouldBeDeleted && MetaStoreUtils.isArchived(part)) { Path archiveParentDir = MetaStoreUtils.getOriginalLocation(part); verifyIsWritablePath(archiveParentDir); archToDelete.add(archiveParentDir); } - if ((part.getSd() != null) && (part.getSd().getLocation() != null)) { + if (tableDataShouldBeDeleted && (part.getSd() != null) && (part.getSd().getLocation() != null)) { Path partPath = new Path(part.getSd().getLocation()); verifyIsWritablePath(partPath); dirsToDelete.add(new PathAndDepth(partPath, part.getValues().size())); @@ -5276,7 +5276,7 @@ public DropPartitionsResult drop_partitions_req( } finally { if (!success) { ms.rollbackTransaction(); - } else if (checkTableDataShouldBeDeleted(tbl, deleteData)) { + } else if (tableDataShouldBeDeleted) { LOG.info(mustPurge ? "dropPartition() will purge partition-directories directly, skipping trash." : "dropPartition() will move partition-directories to trash-directory."); diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java index 0d38b628abb7..cf4fc482808e 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java @@ -18,7 +18,10 @@ package org.apache.hadoop.hive.metastore.client; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.metastore.ColumnType; @@ -68,6 +71,7 @@ import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; @@ -118,6 +122,7 @@ public static void startMetaStores() { extraConf.put("fs.trash.interval", "30"); // FS_TRASH_INTERVAL_KEY (hadoop-2) extraConf.put(ConfVars.HIVE_IN_TEST.getVarname(), "true"); extraConf.put(ConfVars.METASTORE_METADATA_TRANSFORMER_CLASS.getVarname(), " "); + extraConf.put(ConfVars.AUTHORIZATION_STORAGE_AUTH_CHECKS.getVarname(), "true"); startMetaStores(msConf, extraConf); } @@ -1563,6 +1568,41 @@ public void dropTableBogusCatalog() throws TException { client.dropTable("nosuch", testTables[0].getDbName(), testTables[0].getTableName(), true, false); } + @Test(expected = MetaException.class) + public void testDropManagedTableWithoutStoragePermission() throws TException, IOException { + String dbName = testTables[0].getDbName(); + String tblName = testTables[0].getTableName(); + Table table = client.getTable(dbName, tblName); + Path tablePath = new Path(table.getSd().getLocation()); + FileSystem fs = Warehouse.getFs(tablePath, new Configuration()); + fs.setPermission(tablePath.getParent(), new FsPermission((short) 0555)); + + try { + client.dropTable(dbName, tblName); + } finally { + // recover write permission so that file can be cleaned. + fs.setPermission(tablePath.getParent(), new FsPermission((short) 0755)); + } + } + + @Test + public void testDropExternalTableWithoutStoragePermission() throws TException, IOException { + // external table + String dbName = testTables[4].getDbName(); + String tblName = testTables[4].getTableName(); + Table table = client.getTable(dbName, tblName); + Path tablePath = new Path(table.getSd().getLocation()); + FileSystem fs = Warehouse.getFs(tablePath, new Configuration()); + fs.setPermission(tablePath.getParent(), new FsPermission((short) 0555)); + + try { + client.dropTable(dbName, tblName); + } finally { + // recover write permission so that file can be cleaned. + fs.setPermission(tablePath.getParent(), new FsPermission((short) 0755)); + } + } + /** * Creates a Table with all of the parameters set. The temporary table is available only on HS2 * server, so do not use it. From 2c775f88e637e051926b859209f4a3d1df73741a Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 11 Jan 2024 10:41:20 +0100 Subject: [PATCH 129/179] HIVE-27977: Fix ordering flakiness in TestHplSqlViaBeeLine (#4994) (Laszlo Bodor reviewed by Butao Zhang, Ayush Saxena) --- .../apache/hive/beeline/TestHplSqlViaBeeLine.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java index 2e3a3c945a80..a7a547adb8d5 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java @@ -184,8 +184,8 @@ public void testUdfBoolean() throws Throwable { "BEGIN\n" + " RETURN 'This is ' || b;\n" + "END;\n" + - "SELECT check(col_b) FROM result;\n"; - testScriptFile(SCRIPT_TEXT, args(), "This is true.*This is false"); + "SELECT check(col_b) FROM result ORDER BY col_b ASC;\n"; + testScriptFile(SCRIPT_TEXT, args(), "This is false.*This is true"); } @Test @@ -280,7 +280,7 @@ public void testUdfString() throws Throwable { "BEGIN\n" + " RETURN 'Hello ' || s || '!';\n" + "END;\n" + - "SELECT hello(col_s) FROM result;\n"; + "SELECT hello(col_s) FROM result ORDER BY col_s ASC;\n"; testScriptFile(SCRIPT_TEXT, args(), "Hello Alice!.*Hello Smith!"); } @@ -344,8 +344,8 @@ public void testUdfVarchar() throws Throwable { "BEGIN\n" + " RETURN 'Hello ' || v || '!';\n" + "END;\n" + - "SELECT hello(col_v) FROM result;\n"; - testScriptFile(SCRIPT_TEXT, args(), "Hello Smith!.*Hello Sachin!"); + "SELECT hello(col_v) FROM result ORDER BY col_v ASC;\n"; + testScriptFile(SCRIPT_TEXT, args(), "Hello Sachin!.*Hello Smith!"); } @Test @@ -360,8 +360,8 @@ public void testUdfChar() throws Throwable { "BEGIN\n" + " RETURN 'Hello ' || c || '!';\n" + "END;\n" + - "SELECT hello(col_c) FROM result;\n"; - testScriptFile(SCRIPT_TEXT, args(), "Hello Daya!.*Hello Alice!"); + "SELECT hello(col_c) FROM result ORDER BY col_c ASC;\n"; + testScriptFile(SCRIPT_TEXT, args(), "Hello Alice!.*Hello Daya!"); } @Test From 3b064f45fb909b68e3ceb3ac259b8b1b37c8c0de Mon Sep 17 00:00:00 2001 From: yigress <104102129+yigress@users.noreply.github.com> Date: Fri, 12 Jan 2024 13:04:11 -0800 Subject: [PATCH 130/179] HIVE-27951: hcatalog dynamic partitioning fails with partition already exist error when exist parent partitions path (#4979) (yigress reviewed by Laszlo Bodor) --- .../FileOutputCommitterContainer.java | 112 ++++++++++-------- .../mapreduce/TestHCatDynamicPartitioned.java | 19 ++- .../TestHCatExternalDynamicPartitioned.java | 4 +- 3 files changed, 80 insertions(+), 55 deletions(-) diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java index de9ad252ff24..e585b5b08c5c 100644 --- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java +++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java @@ -488,7 +488,7 @@ private void updateTableSchema(IMetaStoreClient client, Table table, } /** - * Move all of the files from the temp directory to the final location + * Move task output from the temp directory to the final location * @param srcf the file to move * @param srcDir the source directory * @param destDir the target directory @@ -497,7 +497,7 @@ private void updateTableSchema(IMetaStoreClient client, Table table, */ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, Path destDir, boolean immutable) throws IOException { - if(LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { LOG.debug("moveTaskOutputs " + srcf + " from: " + srcDir + " to: " + destDir + " immutable: " + immutable); } @@ -516,8 +516,8 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, LinkedList> moves = new LinkedList<>(); if (customDynamicLocationUsed) { - if (immutable && destFs.exists(destDir) && - !org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(destFs, destDir)) { + if (immutable && destFs.exists(destDir) + && !org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(destFs, destDir)) { throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, "Data already exists in " + destDir + ", duplicate publish not possible."); @@ -536,19 +536,18 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, FileStatus srcStatus = srcQ.remove(); Path srcF = srcStatus.getPath(); final Path finalOutputPath = getFinalPath(destFs, srcF, srcDir, destDir, immutable); - if (immutable && destFs.exists(finalOutputPath) && - !org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(destFs, finalOutputPath)) { - throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, - "Data already exists in " + finalOutputPath - + ", duplicate publish not possible."); - } - if (srcStatus.isDirectory()) { + if (immutable && destFs.exists(finalOutputPath) + && !org.apache.hadoop.hive.metastore.utils.FileUtils.isDirEmpty(destFs, finalOutputPath)) { + if (partitionsDiscoveredByPath.containsKey(srcF.toString())) { + throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION, + "Data already exists in " + finalOutputPath + ", duplicate publish not possible."); + } + // parent directory may exist for multi-partitions, check lower level partitions + Collections.addAll(srcQ, srcFs.listStatus(srcF, HIDDEN_FILES_PATH_FILTER)); + } else if (srcStatus.isDirectory()) { if (canRename && dynamicPartitioningUsed) { // If it is partition, move the partition directory instead of each file. - // If custom dynamic location provided, need to rename to final output path - final Path parentDir = finalOutputPath.getParent(); - Path dstPath = !customDynamicLocationUsed ? parentDir : finalOutputPath; - moves.add(Pair.of(srcF, dstPath)); + moves.add(Pair.of(srcF, finalOutputPath)); } else { Collections.addAll(srcQ, srcFs.listStatus(srcF, HIDDEN_FILES_PATH_FILTER)); } @@ -558,50 +557,69 @@ private void moveTaskOutputs(final Configuration conf, Path srcf, Path srcDir, } } - if (moves.isEmpty()) { + bulkMoveFiles(conf, srcFs, destFs, moves); + } + + /** + * Bulk move files from source to destination. + * @param srcFs the source filesystem where the source files are + * @param destFs the destionation filesystem where the destionation files are + * @param pairs list of pairs of , move source_path to destination_path + * @throws java.io.IOException + */ + private void bulkMoveFiles(final Configuration conf, final FileSystem srcFs, final FileSystem destFs, + final List> pairs) throws IOException { + if (pairs.isEmpty()) { return; } - + final boolean canRename = srcFs.getUri().equals(destFs.getUri()); final List>> futures = new LinkedList<>(); - final ExecutorService pool = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ? - Executors.newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25), - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Move-Thread-%d").build()) : null; + final int moveThreadsCount = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25); - for (final Pair pair: moves){ + if (moveThreadsCount <= 0) { + for (final Pair pair: pairs) { + Path srcP = pair.getLeft(); + Path dstP = pair.getRight(); + if (!moveFile(srcFs, srcP, destFs, dstP, conf, canRename)) { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, + "Unable to move from " + srcP + " to " + dstP); + } + } + return; + } + + final ExecutorService pool = Executors.newFixedThreadPool(moveThreadsCount, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Move-Thread-%d").build()); + + for (final Pair pair: pairs) { Path srcP = pair.getLeft(); Path dstP = pair.getRight(); - final String msg = "Unable to move source " + srcP + " to destination " + dstP; - if (null==pool) { - moveFile(srcFs, srcP, destFs, dstP, conf, canRename); - } else { - futures.add(pool.submit(new Callable>() { - @Override - public Pair call() throws IOException { - if (moveFile(srcFs, srcP, destFs, dstP, conf, canRename)) { - return pair; - } else { - throw new HCatException(ErrorType.ERROR_MOVE_FAILED, msg); - } + futures.add(pool.submit(new Callable>() { + @Override + public Pair call() throws IOException { + if (moveFile(srcFs, srcP, destFs, dstP, conf, canRename)) { + return pair; + } else { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, + "Unable to move from " + srcP + " to " + dstP); } - })); - } - } - if (null != pool) { - pool.shutdown(); - for (Future> future : futures) { - try { - Pair pair = future.get(); - LOG.debug("Moved src: {}, to dest: {}", pair.getLeft().toString(), pair.getRight().toString()); - } catch (Exception e) { - LOG.error("Failed to move {}", e.getMessage()); - pool.shutdownNow(); - throw new HCatException(ErrorType.ERROR_MOVE_FAILED, e.getMessage()); } + })); + } + pool.shutdown(); + for (Future> future : futures) { + try { + future.get(); + } catch (Exception e) { + pool.shutdownNow(); + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, e.getMessage()); } } } - private boolean moveFile(FileSystem srcFs, Path srcf, FileSystem destFs, Path destf, Configuration conf, boolean canRename) throws IOException { + private boolean moveFile(final FileSystem srcFs, final Path srcf, final FileSystem destFs, final Path destf, + final Configuration conf, final boolean canRename) throws IOException { + LOG.debug("Moving src: {}, to dest: {}", srcf, destf); boolean moved; if (canRename) { destFs.mkdirs(destf.getParent()); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java index a97162de993a..9ee887b933bd 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatDynamicPartitioned.java @@ -52,13 +52,13 @@ public class TestHCatDynamicPartitioned extends HCatMapReduceTest { private static List dataColumns; private static final Logger LOG = LoggerFactory.getLogger(TestHCatDynamicPartitioned.class); protected static final int NUM_RECORDS = 20; - protected static final int NUM_PARTITIONS = 5; + protected static final int NUM_TOP_PARTITIONS = 5; public TestHCatDynamicPartitioned(String formatName, String serdeClass, String inputFormatClass, String outputFormatClass) throws Exception { super(formatName, serdeClass, inputFormatClass, outputFormatClass); tableName = "testHCatDynamicPartitionedTable_" + formatName; - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); generateDataColumns(); } @@ -67,6 +67,8 @@ protected static void generateDataColumns() throws HCatException { dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""))); + dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, ""))); + } protected static void generateWriteRecords(int max, int mod, int offset) { @@ -78,6 +80,7 @@ protected static void generateWriteRecords(int max, int mod, int offset) { objList.add(i); objList.add("strvalue" + i); objList.add(String.valueOf((i % mod) + offset)); + objList.add(String.valueOf((i / (max/2)) + offset)); writeRecords.add(new DefaultHCatRecord(objList)); } } @@ -86,6 +89,7 @@ protected static void generateWriteRecords(int max, int mod, int offset) { protected List getPartitionKeys() { List fields = new ArrayList(); fields.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, "")); + fields.add(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, "")); return fields; } @@ -117,8 +121,11 @@ public void testHCatDynamicPartitionedTableMultipleTask() throws Exception { protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, String customDynamicPathPattern) throws Exception { - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); - runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, true, asSingleMapTask, customDynamicPathPattern); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); + runMRCreate(null, dataColumns, writeRecords.subList(0,NUM_RECORDS/2), NUM_RECORDS/2, + true, asSingleMapTask, customDynamicPathPattern); + runMRCreate(null, dataColumns, writeRecords.subList(NUM_RECORDS/2,NUM_RECORDS), NUM_RECORDS/2, + true, asSingleMapTask, customDynamicPathPattern); runMRRead(NUM_RECORDS); @@ -140,7 +147,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, //Test for duplicate publish IOException exc = null; try { - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); Job job = runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, false, true, customDynamicPathPattern); @@ -167,7 +174,7 @@ protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask, driver.run(query); res = new ArrayList(); driver.getResults(res); - assertEquals(NUM_PARTITIONS, res.size()); + assertEquals(NUM_TOP_PARTITIONS*2, res.size()); query = "select * from " + tableName; driver.run(query); diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java index 18fcfdbdd2a8..f142f3d488fa 100644 --- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java +++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatExternalDynamicPartitioned.java @@ -28,7 +28,7 @@ public TestHCatExternalDynamicPartitioned(String formatName, String serdeClass, throws Exception { super(formatName, serdeClass, inputFormatClass, outputFormatClass); tableName = "testHCatExternalDynamicPartitionedTable_" + formatName; - generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateWriteRecords(NUM_RECORDS, NUM_TOP_PARTITIONS, 0); generateDataColumns(); } @@ -43,7 +43,7 @@ protected Boolean isTableExternal() { */ @Test public void testHCatExternalDynamicCustomLocation() throws Exception { - runHCatDynamicPartitionedTable(true, "mapred/externalDynamicOutput/${p1}"); + runHCatDynamicPartitionedTable(true, "mapred/externalDynamicOutput/${p1}/${p2}"); } } From bd18b431ed5326076d1388ae296ae8861744eb76 Mon Sep 17 00:00:00 2001 From: InvisibleProgrammer Date: Fri, 12 Jan 2024 22:18:43 +0100 Subject: [PATCH 131/179] HIVE-27974: Fix flaky test - TestReplicationMetricCollector.testSuccessStageFailure (#4976). (Zsolt Miskolczi, reviewed by Ayush Saxena) --- .../parse/repl/metric/TestReplicationMetricCollector.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/metric/TestReplicationMetricCollector.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/metric/TestReplicationMetricCollector.java index 842eb1dd78bb..ce4c774c7385 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/metric/TestReplicationMetricCollector.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/metric/TestReplicationMetricCollector.java @@ -49,7 +49,6 @@ import org.junit.runner.RunWith; import org.mockito.MockedStatic; import org.mockito.Mockito; -import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; import java.util.Map; @@ -70,16 +69,17 @@ public class TestReplicationMetricCollector { HiveConf conf; - @Mock private FailoverMetaData fmd; - @Mock private MetricSink metricSinkInstance; - static MockedStatic metricSinkMockedStatic; + MockedStatic metricSinkMockedStatic; @Before public void setup() throws Exception { + fmd = Mockito.mock(FailoverMetaData.class); + metricSinkInstance = Mockito.mock(MetricSink.class); + conf = new HiveConf(); conf.set(Constants.SCHEDULED_QUERY_SCHEDULENAME, "repl"); conf.set(Constants.SCHEDULED_QUERY_EXECUTIONID, "1"); From b4682cfff26b6d0bec8d2922fe965d0323aa8a15 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Sat, 13 Jan 2024 05:20:34 +0800 Subject: [PATCH 132/179] HIVE-27989: Wrong database name in MetaException from MetastoreDefaultTransformer.java (#4989). (Butao Zhang, reviewed by Ayush Saxena) --- .../hadoop/hive/metastore/MetastoreDefaultTransformer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java index ff92ab86d420..89c78aa0af33 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDefaultTransformer.java @@ -772,7 +772,7 @@ private Database getDbForTable(Table oldTable) throws MetaException { return hmsHandler.get_database_core(oldTable.getCatName(), oldTable.getDbName()); } catch (NoSuchObjectException e) { throw new MetaException( - "Database " + oldTable.getTableName() + " for table " + oldTable.getTableName() + " could not be found"); + "Database " + oldTable.getDbName() + " for table " + oldTable.getTableName() + " could not be found"); } } @@ -895,7 +895,7 @@ private Table validateTablePaths(Table table) throws MetaException { try { db = hmsHandler.get_database_core(table.getCatName(), table.getDbName()); } catch (NoSuchObjectException e) { - throw new MetaException("Database " + table.getTableName() + " for table " + table.getTableName() + " could not be found"); + throw new MetaException("Database " + table.getDbName() + " for table " + table.getTableName() + " could not be found"); } if (TableType.MANAGED_TABLE.name().equals(table.getTableType())) { From 5d0f2502afaf6a220bc14a2cad5f643441707a47 Mon Sep 17 00:00:00 2001 From: Alagappan Maruthappan Date: Fri, 12 Jan 2024 23:40:19 -0800 Subject: [PATCH 133/179] HIVE-27023: Add setting to prevent tez session from being opened during startup (#4015) (Alagappan Maruthappan reviewed by Laszlo Bodor) --- cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java | 1 + common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 6 +++++- .../org/apache/hadoop/hive/ql/session/SessionState.java | 4 +++- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java index 6eb679ddebc1..93e06208e1ee 100644 --- a/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java +++ b/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java @@ -809,6 +809,7 @@ public Map getHiveVariable() { } catch (CommandProcessorException e) { return e.getResponseCode(); } finally { + SessionState.endStart(ss); ss.resetThreadName(); ss.close(); } diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 9a1433dc21da..3052349b5492 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -3693,8 +3693,12 @@ public static enum ConfVars { HIVE_CLI_PRINT_ESCAPE_CRLF("hive.cli.print.escape.crlf", false, "Whether to print carriage returns and line feeds in row output as escaped \\r and \\n"), + HIVE_CLI_TEZ_INITIALIZE_SESSION("hive.cli.tez.initialize.session", true, + "When enabled, CLI running with Tez will preemptively open a tez session during start up."), + HIVE_CLI_TEZ_SESSION_ASYNC("hive.cli.tez.session.async", true, "Whether to start Tez\n" + - "session in background when running CLI with Tez, allowing CLI to be available earlier."), + "session in background when running CLI with Tez, allowing CLI to be available earlier. " + + "If hive.cli.tez.initialize.session is set to false, this value is ignored."), HIVE_DISABLE_UNSAFE_EXTERNALTABLE_OPERATIONS("hive.disable.unsafe.external.table.operations", true, "Whether to disable certain optimizations and operations on external tables," + diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 3ba5fb3361f1..533462fe5074 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -733,7 +733,9 @@ private static void start(SessionState startSs, boolean isAsync, LogHelper conso } String engine = HiveConf.getVar(startSs.getConf(), HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); - if (!engine.equals("tez") || startSs.isHiveServerQuery) { + + if (!engine.equals("tez") || startSs.isHiveServerQuery + || !HiveConf.getBoolVar(startSs.getConf(), ConfVars.HIVE_CLI_TEZ_INITIALIZE_SESSION)) { return; } From c5c8fe4ed6c4294298d75caeaf671ba632b2d4c2 Mon Sep 17 00:00:00 2001 From: Dayakar M <59791497+mdayakar@users.noreply.github.com> Date: Mon, 15 Jan 2024 07:52:29 +0530 Subject: [PATCH 134/179] HIVE-27492: HPL/SQL built-in functions like sysdate not working (Dayakar M, reviewed by Krisztian Kasa, Aman Sinha, Attila Turoczy) --- .../antlr4/org/apache/hive/hplsql/Hplsql.g4 | 6 +- .../java/org/apache/hive/hplsql/Exec.java | 2 +- .../main/java/org/apache/hive/hplsql/Var.java | 4 +- .../hplsql/functions/BuiltinFunctions.java | 7 +- .../hplsql/functions/FunctionDatetime.java | 47 +--- .../hive/hplsql/functions/FunctionMisc.java | 60 +--- .../hive/hplsql/functions/FunctionString.java | 162 ++--------- .../apache/hive/hplsql/TestHplsqlLocal.java | 49 ---- hplsql/src/test/queries/local/cast.sql | 4 - hplsql/src/test/queries/local/cast2.sql | 10 - hplsql/src/test/queries/local/coalesce.sql | 4 - hplsql/src/test/queries/local/concat.sql | 2 +- .../test/queries/local/create_function3.sql | 22 +- .../test/queries/local/create_function4.sql | 4 +- hplsql/src/test/queries/local/date.sql | 3 +- hplsql/src/test/queries/local/expr.sql | 5 +- hplsql/src/test/queries/local/if2.sql | 2 +- hplsql/src/test/queries/local/instr.sql | 49 ---- hplsql/src/test/queries/local/interval.sql | 8 +- hplsql/src/test/queries/local/length.sql | 1 - hplsql/src/test/queries/local/lower.sql | 1 - hplsql/src/test/queries/local/mult_div.sql | 4 +- hplsql/src/test/queries/local/nvl.sql | 4 - hplsql/src/test/queries/local/replace.sql | 1 - hplsql/src/test/queries/local/timestamp.sql | 4 - .../src/test/queries/local/timestamp_iso.sql | 3 +- hplsql/src/test/queries/local/trim.sql | 1 - hplsql/src/test/results/local/add.out.txt | 4 +- hplsql/src/test/results/local/cast.out.txt | 8 - hplsql/src/test/results/local/cast2.out.txt | 15 - .../src/test/results/local/coalesce.out.txt | 4 - hplsql/src/test/results/local/date.out.txt | 7 +- hplsql/src/test/results/local/declare.out.txt | 4 +- hplsql/src/test/results/local/expr.out.txt | 4 - hplsql/src/test/results/local/instr.out.txt | 33 --- .../src/test/results/local/interval.out.txt | 24 +- .../test/results/local/invalid_syntax.out.txt | 2 +- hplsql/src/test/results/local/length.out.txt | 1 - hplsql/src/test/results/local/lower.out.txt | 1 - hplsql/src/test/results/local/nvl.out.txt | 4 - hplsql/src/test/results/local/replace.out.txt | 1 - hplsql/src/test/results/local/sub.out.txt | 2 +- .../src/test/results/local/timestamp.out.txt | 4 - .../test/results/local/timestamp_iso.out.txt | 3 +- hplsql/src/test/results/local/to_char.out.txt | 2 +- .../test/results/local/to_timestamp.out.txt | 8 +- hplsql/src/test/results/local/trim.out.txt | 1 - .../hive/beeline/TestHplSqlViaBeeLine.java | 260 +++++++++++++++++- 48 files changed, 368 insertions(+), 493 deletions(-) delete mode 100644 hplsql/src/test/queries/local/cast.sql delete mode 100644 hplsql/src/test/queries/local/cast2.sql delete mode 100644 hplsql/src/test/queries/local/coalesce.sql delete mode 100644 hplsql/src/test/queries/local/instr.sql delete mode 100644 hplsql/src/test/queries/local/length.sql delete mode 100644 hplsql/src/test/queries/local/lower.sql delete mode 100644 hplsql/src/test/queries/local/nvl.sql delete mode 100644 hplsql/src/test/queries/local/replace.sql delete mode 100644 hplsql/src/test/queries/local/timestamp.sql delete mode 100644 hplsql/src/test/queries/local/trim.sql delete mode 100644 hplsql/src/test/results/local/cast.out.txt delete mode 100644 hplsql/src/test/results/local/cast2.out.txt delete mode 100644 hplsql/src/test/results/local/coalesce.out.txt delete mode 100644 hplsql/src/test/results/local/instr.out.txt delete mode 100644 hplsql/src/test/results/local/length.out.txt delete mode 100644 hplsql/src/test/results/local/lower.out.txt delete mode 100644 hplsql/src/test/results/local/nvl.out.txt delete mode 100644 hplsql/src/test/results/local/replace.out.txt delete mode 100644 hplsql/src/test/results/local/timestamp.out.txt delete mode 100644 hplsql/src/test/results/local/trim.out.txt diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 index 8bec8cd78c38..14cf034ad017 100644 --- a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 +++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 @@ -1049,8 +1049,8 @@ expr : | expr_case | expr_cursor_attribute | expr_agg_window_func - | expr_spec_func | expr_func + | expr_spec_func | expr_atom ; @@ -1324,6 +1324,7 @@ non_reserved_words : // Tokens that are not reserved words | T_CURRENT | T_CURRENT_DATE | T_CURRENT_SCHEMA + | T_CURRENT_TIME_MILLIS | T_CURRENT_TIMESTAMP | T_CURRENT_USER | T_CURSOR @@ -1770,6 +1771,7 @@ T_MERGE : M E R G E ; T_MESSAGE_TEXT : M E S S A G E '_' T E X T ; T_MICROSECOND : M I C R O S E C O N D ; T_MICROSECONDS : M I C R O S E C O N D S; +T_MILLIS : M I L L I S ; T_MIN : M I N ; T_MULTISET : M U L T I S E T ; T_NCHAR : N C H A R ; @@ -1877,6 +1879,7 @@ T_TEMPORARY : T E M P O R A R Y ; T_TERMINATED : T E R M I N A T E D ; T_TEXTIMAGE_ON : T E X T I M A G E '_' O N ; T_THEN : T H E N ; +T_TIME : T I M E ; T_TIMESTAMP : T I M E S T A M P ; T_TINYINT : T I N Y I N T ; T_TITLE : T I T L E ; @@ -1913,6 +1916,7 @@ T_YES : Y E S ; T_ACTIVITY_COUNT : A C T I V I T Y '_' C O U N T ; T_CUME_DIST : C U M E '_' D I S T ; T_CURRENT_DATE : C U R R E N T '_' D A T E ; +T_CURRENT_TIME_MILLIS : C U R R E N T '_' T I M E '_' M I L L I S ; T_CURRENT_TIMESTAMP : C U R R E N T '_' T I M E S T A M P ; T_CURRENT_USER : C U R R E N T '_' U S E R ; T_DENSE_RANK : D E N S E '_' R A N K ; diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java index 8d5cab92e136..e423452e3da5 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java @@ -1801,7 +1801,7 @@ public Integer visitExpr_func(HplsqlParser.Expr_funcContext ctx) { private int functionCall(ParserRuleContext ctx, HplsqlParser.IdentContext ident, HplsqlParser.Expr_func_paramsContext params) { String name = ident.getText(); - if (exec.buildSql) { + if (exec.buildSql && !builtinFunctions.exists(name)) { exec.execSql(name, params); } else { name = name.toUpperCase(); diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java index 198a4d0b09a3..1c7bb41824cd 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java @@ -601,7 +601,7 @@ else if (type == Type.STRING) { return (String)value; } else if (type == Type.DATE) { - return ((Date)value).toString(); + return String.format("DATE '%s'", value); } else if (type == Type.TIMESTAMP) { int len = 19; @@ -612,7 +612,7 @@ else if (type == Type.TIMESTAMP) { if (t.length() > len) { t = t.substring(0, len); } - return t; + return String.format("TIMESTAMP '%s'", t); } return value.toString(); } diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/BuiltinFunctions.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/BuiltinFunctions.java index e5335854d054..100fa19c5349 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/BuiltinFunctions.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/BuiltinFunctions.java @@ -114,7 +114,10 @@ public void specExec(HplsqlParser.Expr_spec_funcContext ctx) { execMinPartDate(ctx); } else if (ctx.T_PART_LOC() != null) { execPartLoc(ctx); - } else { + } else if (exec.buildSql){ + exec.stackPush(Exec.getFormattedText(ctx)); + } + else { evalNull(); } } @@ -132,7 +135,7 @@ public void specExecSql(HplsqlParser.Expr_spec_funcContext ctx) { func.run(ctx); } else { - exec.stackPush(Exec.getFormattedText(ctx)); + specExec(ctx); } } diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java index 4870c14745a5..e7e562aca67f 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java @@ -21,7 +21,6 @@ import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.Calendar; -import java.util.Date; import java.util.TimeZone; import org.apache.commons.lang3.StringUtils; @@ -39,15 +38,11 @@ public FunctionDatetime(Exec e, QueryExecutor queryExecutor) { @Override public void register(BuiltinFunctions f) { f.map.put("DATE", this::date); - f.map.put("FROM_UNIXTIME", this::fromUnixtime); f.map.put("NOW", ctx -> now(ctx)); f.map.put("TIMESTAMP_ISO", this::timestampIso); f.map.put("TO_TIMESTAMP", this::toTimestamp); - f.map.put("UNIX_TIMESTAMP", this::unixTimestamp); f.map.put("CURRENT_TIME_MILLIS", this::currentTimeMillis); - f.specMap.put("CURRENT_DATE", this::currentDate); - f.specMap.put("CURRENT_TIMESTAMP", this::currentTimestamp); f.specMap.put("SYSDATE", this::currentTimestamp); f.specSqlMap.put("CURRENT_DATE", (FuncSpecCommand) this::currentDateSql); @@ -57,10 +52,6 @@ public void register(BuiltinFunctions f) { /** * CURRENT_DATE */ - public void currentDate(HplsqlParser.Expr_spec_funcContext ctx) { - evalVar(currentDate()); - } - public static Var currentDate() { SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd"); String s = f.format(Calendar.getInstance().getTime()); @@ -86,7 +77,7 @@ public void currentTimestamp(HplsqlParser.Expr_spec_funcContext ctx) { int precision = evalPop(ctx.expr(0), 3).intValue(); evalVar(currentTimestamp(precision)); } - + public static Var currentTimestamp(int precision) { String format = "yyyy-MM-dd HH:mm:ss"; if (precision > 0 && precision <= 3) { @@ -118,7 +109,9 @@ void date(HplsqlParser.Expr_func_paramsContext ctx) { return; } Var var = new Var(Var.Type.DATE); - var.cast(evalPop(ctx.func_param(0).expr())); + Var date = evalPop(ctx.func_param(0).expr()); + date.setValue(Utils.unquoteString(date.toString())); + var.cast(date); evalVar(var); } @@ -142,7 +135,9 @@ void timestampIso(HplsqlParser.Expr_func_paramsContext ctx) { return; } Var var = new Var(Var.Type.TIMESTAMP); - var.cast(evalPop(ctx.func_param(0).expr())); + Var val = evalPop(ctx.func_param(0).expr()); + val.setValue(Utils.unquoteString(val.toString())); + var.cast(val); evalVar(var); } @@ -154,8 +149,8 @@ void toTimestamp(HplsqlParser.Expr_func_paramsContext ctx) { evalNull(); return; } - String value = evalPop(ctx.func_param(0).expr()).toString(); - String sqlFormat = evalPop(ctx.func_param(1).expr()).toString(); + String value = Utils.unquoteString(evalPop(ctx.func_param(0).expr()).toString()); + String sqlFormat = Utils.unquoteString(evalPop(ctx.func_param(1).expr()).toString()); String format = Utils.convertSqlDatetimeFormat(sqlFormat); try { long timeInMs = new SimpleDateFormat(format).parse(value).getTime(); @@ -166,30 +161,6 @@ void toTimestamp(HplsqlParser.Expr_func_paramsContext ctx) { evalNull(); } } - - /** - * FROM_UNIXTIME() function (convert seconds since 1970-01-01 00:00:00 to timestamp) - */ - void fromUnixtime(HplsqlParser.Expr_func_paramsContext ctx) { - int cnt = BuiltinFunctions.getParamCount(ctx); - if (cnt == 0) { - evalNull(); - return; - } - long epoch = evalPop(ctx.func_param(0).expr()).longValue(); - String format = "yyyy-MM-dd HH:mm:ss"; - if (cnt > 1) { - format = evalPop(ctx.func_param(1).expr()).toString(); - } - evalString(new SimpleDateFormat(format).format(new Date(epoch * 1000))); - } - - /** - * UNIX_TIMESTAMP() function (current date and time in seconds since 1970-01-01 00:00:00) - */ - void unixTimestamp(HplsqlParser.Expr_func_paramsContext ctx) { - evalVar(new Var(System.currentTimeMillis()/1000)); - } public void currentTimeMillis(HplsqlParser.Expr_func_paramsContext ctx) { evalVar(new Var(System.currentTimeMillis())); diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java index dba5594d933e..18c2ded2e036 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java @@ -40,15 +40,11 @@ public FunctionMisc(Exec e, QueryExecutor queryExecutor) { */ @Override public void register(BuiltinFunctions f) { - f.map.put("COALESCE", this::nvl); f.map.put("DECODE", this::decode); - f.map.put("NVL", this::nvl); f.map.put("NVL2", this::nvl2); f.map.put("PART_COUNT_BY", this::partCountBy); - f.map.put("MOD", this::modulo); f.specMap.put("ACTIVITY_COUNT", this::activityCount); - f.specMap.put("CAST", this::cast); f.specMap.put("CURRENT", this::current); f.specMap.put("CURRENT_USER", this::currentUser); f.specMap.put("PART_COUNT", this::partCount); @@ -64,28 +60,6 @@ void activityCount(HplsqlParser.Expr_spec_funcContext ctx) { evalInt(Long.valueOf(exec.getRowCount())); } - /** - * CAST function - */ - void cast(HplsqlParser.Expr_spec_funcContext ctx) { - if (ctx.expr().size() != 1) { - evalNull(); - return; - } - String type = ctx.dtype().getText(); - String len = null; - String scale = null; - if (ctx.dtype_len() != null) { - len = ctx.dtype_len().L_INT(0).getText(); - if (ctx.dtype_len().L_INT(1) != null) { - scale = ctx.dtype_len().L_INT(1).getText(); - } - } - Var var = new Var(null, type, len, scale, null); - var.cast(evalPop(ctx.expr(0))); - evalVar(var); - } - /** * CURRENT function */ @@ -124,8 +98,9 @@ else if (ctx.T_TIMESTAMP() != null) { else { evalString("CURRENT_TIMESTAMP"); } - } - else { + } else if (ctx.T_USER() != null) { + evalString("CURRENT_USER()"); + } else { evalString(exec.getFormattedText(ctx)); } } @@ -138,7 +113,7 @@ void currentUser(HplsqlParser.Expr_spec_funcContext ctx) { } public static Var currentUser() { - return new Var(System.getProperty("user.name")); + return new Var("CURRENT_USER()"); } /** @@ -168,26 +143,13 @@ void decode(HplsqlParser.Expr_func_paramsContext ctx) { } } - /** - * NVL function - Return first non-NULL expression - */ - void nvl(HplsqlParser.Expr_func_paramsContext ctx) { - for (int i=0; i < ctx.func_param().size(); i++) { - Var v = evalPop(ctx.func_param(i).expr()); - if (v.type != Var.Type.NULL) { - exec.stackPush(v); - return; - } - } - evalNull(); - } - /** * NVL2 function - If expr1 is not NULL return expr2, otherwise expr3 */ void nvl2(HplsqlParser.Expr_func_paramsContext ctx) { if (ctx.func_param().size() == 3) { - if (!evalPop(ctx.func_param(0).expr()).isNull()) { + Var firstParam = evalPop(ctx.func_param(0).expr()); + if (!(firstParam.isNull() || "null".equalsIgnoreCase((String)firstParam.value))) { eval(ctx.func_param(1).expr()); } else { @@ -249,16 +211,6 @@ public void partCount(HplsqlParser.Expr_spec_funcContext ctx) { query.close(); } - public void modulo(HplsqlParser.Expr_func_paramsContext ctx) { - if (ctx.func_param().size() == 2) { - int a = evalPop(ctx.func_param(0).expr()).intValue(); - int b = evalPop(ctx.func_param(1).expr()).intValue(); - evalInt(a % b); - } else { - evalNull(); - } - } - /** * PART_COUNT_BY function */ diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java index 702da33908f9..0020189a7978 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java @@ -33,18 +33,12 @@ public FunctionString(Exec e, QueryExecutor queryExecutor) { public void register(BuiltinFunctions f) { f.map.put("CONCAT", this::concat); f.map.put("CHAR", this::char_); - f.map.put("INSTR", this::instr); f.map.put("LEN", this::len); - f.map.put("LENGTH", this::length); - f.map.put("LOWER", this::lower); - f.map.put("REPLACE", this::replace); f.map.put("SUBSTR", this::substr); f.map.put("SUBSTRING", this::substr); f.map.put("TO_CHAR", this::toChar); f.map.put("UPPER", this::upper); - f.specMap.put("SUBSTRING", this::substring); - f.specMap.put("TRIM", this::trim); } /** @@ -52,12 +46,13 @@ public void register(BuiltinFunctions f) { */ void concat(HplsqlParser.Expr_func_paramsContext ctx) { StringBuilder val = new StringBuilder(); + appendSingleQuote(val); int cnt = getParamCount(ctx); boolean nulls = true; for (int i = 0; i < cnt; i++) { Var c = evalPop(ctx.func_param(i).expr()); - if (!c.isNull()) { - val.append(c.toString()); + if (!c.isNull() && !"null".equalsIgnoreCase((String)c.value)) { + val.append(Utils.unquoteString(c.toString())); nulls = false; } } @@ -65,6 +60,7 @@ void concat(HplsqlParser.Expr_func_paramsContext ctx) { evalNull(); } else { + appendSingleQuote(val); evalString(val); } } @@ -82,67 +78,6 @@ void char_(HplsqlParser.Expr_func_paramsContext ctx) { evalString(str); } - /** - * INSTR function - */ - void instr(HplsqlParser.Expr_func_paramsContext ctx) { - int cnt = getParamCount(ctx); - if (cnt < 2) { - evalNull(); - return; - } - String str = evalPop(ctx.func_param(0).expr()).toString(); - if (str == null) { - evalNull(); - return; - } - else if(str.isEmpty()) { - evalInt(0); - return; - } - String substr = evalPop(ctx.func_param(1).expr()).toString(); - int pos = 1; - int occur = 1; - int idx = 0; - if (cnt >= 3) { - pos = evalPop(ctx.func_param(2).expr()).intValue(); - if (pos == 0) { - pos = 1; - } - } - if (cnt >= 4) { - occur = evalPop(ctx.func_param(3).expr()).intValue(); - if (occur < 0) { - occur = 1; - } - } - for (int i = occur; i > 0; i--) { - if (pos > 0) { - idx = str.indexOf(substr, pos - 1); - } - else { - str = str.substring(0, str.length() - pos*(-1)); - idx = str.lastIndexOf(substr); - } - if (idx == -1) { - idx = 0; - break; - } - else { - idx++; - } - if (i > 1) { - if (pos > 0) { - pos = idx + 1; - } - else { - pos = (str.length() - idx + 1) * (-1); - } - } - } - evalInt(idx); - } - /** * LEN function (excluding trailing spaces) */ @@ -151,49 +86,10 @@ void len(HplsqlParser.Expr_func_paramsContext ctx) { evalNull(); return; } - int len = evalPop(ctx.func_param(0).expr()).toString().trim().length(); + int len = Utils.unquoteString(evalPop(ctx.func_param(0).expr()).toString()).trim().length(); evalInt(len); } - - /** - * LENGTH function - */ - void length(HplsqlParser.Expr_func_paramsContext ctx) { - if (ctx.func_param().size() != 1) { - evalNull(); - return; - } - int len = evalPop(ctx.func_param(0).expr()).toString().length(); - evalInt(len); - } - - /** - * LOWER function - */ - void lower(HplsqlParser.Expr_func_paramsContext ctx) { - if (ctx.func_param().size() != 1) { - evalNull(); - return; - } - String str = evalPop(ctx.func_param(0).expr()).toString().toLowerCase(); - evalString(str); - } - - /** - * REPLACE function - */ - void replace(HplsqlParser.Expr_func_paramsContext ctx) { - int cnt = getParamCount(ctx); - if (cnt < 3) { - evalNull(); - return; - } - String str = evalPop(ctx.func_param(0).expr()).toString(); - String what = evalPop(ctx.func_param(1).expr()).toString(); - String with = evalPop(ctx.func_param(2).expr()).toString(); - evalString(str.replaceAll(what, with)); - } - + /** * SUBSTR and SUBSTRING function */ @@ -203,18 +99,18 @@ void substr(HplsqlParser.Expr_func_paramsContext ctx) { evalNull(); return; } - String str = evalPop(ctx.func_param(0).expr()).toString(); + String str = Utils.unquoteString(evalPop(ctx.func_param(0).expr()).toString()); int start = evalPop(ctx.func_param(1).expr()).intValue(); int len = -1; if (start == 0) { - start = 1; + start = 1; } if (cnt > 2) { len = evalPop(ctx.func_param(2).expr()).intValue(); } substr(str, start, len); } - + void substr(String str, int start, int len) { if (str == null) { evalNull(); @@ -225,27 +121,42 @@ else if (str.isEmpty()) { return; } if (start == 0) { - start = 1; + start = 1; } + StringBuilder resultStr = new StringBuilder(); if (len == -1) { if (start > 0) { - evalString(str.substring(start - 1)); + String substring = str.substring(start - 1); + appendSingleQuote(resultStr); + resultStr.append(substring); + appendSingleQuote(resultStr); + evalString(resultStr); } } else { - evalString(str.substring(start - 1, start - 1 + len)); + String substring = str.substring(start - 1, start - 1 + len); + appendSingleQuote(resultStr); + resultStr.append(substring); + appendSingleQuote(resultStr); + evalString(resultStr); } } - + + private void appendSingleQuote(StringBuilder resultStr) { + if (exec.buildSql) { + resultStr.append("'"); + } + } + /** * SUBSTRING FROM FOR function */ void substring(HplsqlParser.Expr_spec_funcContext ctx) { - String str = evalPop(ctx.expr(0)).toString(); + String str = evalPop(ctx.expr(0)).toString(); int start = evalPop(ctx.expr(1)).intValue(); int len = -1; if (start == 0) { - start = 1; + start = 1; } if (ctx.T_FOR() != null) { len = evalPop(ctx.expr(2)).intValue(); @@ -253,19 +164,6 @@ void substring(HplsqlParser.Expr_spec_funcContext ctx) { substr(str, start, len); } - /** - * TRIM function - */ - void trim(HplsqlParser.Expr_spec_funcContext ctx) { - int cnt = ctx.expr().size(); - if (cnt != 1) { - evalNull(); - return; - } - String str = evalPop(ctx.expr(0)).toString(); - evalString(str.trim()); - } - /** * TO_CHAR function */ diff --git a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java index 064ea9116e23..0af1f83be423 100644 --- a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java +++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java @@ -64,26 +64,11 @@ public void testCase() throws Exception { run("case"); } - @Test - public void testCast() throws Exception { - run("cast"); - } - - @Test - public void testCast2() throws Exception { - run("cast2"); - } - @Test public void testChar() throws Exception { run("char"); } - @Test - public void testCoalesce() throws Exception { - run("coalesce"); - } - @Test public void testConcat() throws Exception { run("concat"); @@ -273,10 +258,6 @@ public void testIf3Bteq() throws Exception { public void testInclude() throws Exception { run("include"); } - @Test - public void testInstr() throws Exception { - run("instr"); - } @Test public void testInterval() throws Exception { @@ -293,31 +274,16 @@ public void testLeave() throws Exception { run("leave"); } - @Test - public void testLength() throws Exception { - run("length"); - } - @Test public void testLen() throws Exception { run("len"); } - @Test - public void testLower() throws Exception { - run("lower"); - } - @Test public void testMultDiv() throws Exception { run("mult_div"); } - @Test - public void testNvl() throws Exception { - run("nvl"); - } - @Test public void testNvl2() throws Exception { run("nvl2"); @@ -327,11 +293,6 @@ public void testNvl2() throws Exception { public void testPrint() throws Exception { run("print"); } - - @Test - public void testReplace() throws Exception { - run("replace"); - } @Test public void testReturn() throws Exception { @@ -363,11 +324,6 @@ public void testTimestampIso() throws Exception { run("timestamp_iso"); } - @Test - public void testTimestamp() throws Exception { - run("timestamp"); - } - @Test public void testToChar() throws Exception { run("to_char"); @@ -378,11 +334,6 @@ public void testToTimestamp() throws Exception { run("to_timestamp"); } - @Test - public void testTrim() throws Exception { - run("trim"); - } - @Test public void testTwoPipes() throws Exception { run("twopipes"); diff --git a/hplsql/src/test/queries/local/cast.sql b/hplsql/src/test/queries/local/cast.sql deleted file mode 100644 index 3adab22823dd..000000000000 --- a/hplsql/src/test/queries/local/cast.sql +++ /dev/null @@ -1,4 +0,0 @@ -CAST('Abc' AS CHAR(1)); -CAST('Abc' AS VARCHAR(2)); -CAST('Abc' AS CHAR); -CAST(TIMESTAMP '2015-03-12 10:58:34.111' AS CHAR(10)) diff --git a/hplsql/src/test/queries/local/cast2.sql b/hplsql/src/test/queries/local/cast2.sql deleted file mode 100644 index d68db6d1b756..000000000000 --- a/hplsql/src/test/queries/local/cast2.sql +++ /dev/null @@ -1,10 +0,0 @@ -temp_int = CAST('1' AS int); -print temp_int -temp_float = CAST('1.2' AS float); -print temp_float -temp_double = CAST('1.2' AS double); -print temp_double -temp_decimal = CAST('1.2' AS decimal(10, 4)); -print temp_decimal -temp_string = CAST('1.2' AS string); -print temp_string \ No newline at end of file diff --git a/hplsql/src/test/queries/local/coalesce.sql b/hplsql/src/test/queries/local/coalesce.sql deleted file mode 100644 index 4b65d58b56fb..000000000000 --- a/hplsql/src/test/queries/local/coalesce.sql +++ /dev/null @@ -1,4 +0,0 @@ -COALESCE('First non-null', 1); -COALESCE(NULL, 'First non-null'); -COALESCE(NULL, 'First non-null', 1); -COALESCE(NULL, NULL, 'First non-null', 1); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/concat.sql b/hplsql/src/test/queries/local/concat.sql index b7769bb98247..37b7619ccb5d 100644 --- a/hplsql/src/test/queries/local/concat.sql +++ b/hplsql/src/test/queries/local/concat.sql @@ -1,2 +1,2 @@ CONCAT('a', 'b', NULL, 'c'); -NVL(CONCAT(NULL, NULL, NULL), 'NULL Value'); \ No newline at end of file +NVL2(CONCAT(NULL, NULL, NULL), NULL, 'NULL Value'); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/create_function3.sql b/hplsql/src/test/queries/local/create_function3.sql index 840c09573566..ec1de72fceb0 100644 --- a/hplsql/src/test/queries/local/create_function3.sql +++ b/hplsql/src/test/queries/local/create_function3.sql @@ -6,47 +6,47 @@ FUNCTION gettype(tag1 varchar2, srcvalue varchar2) return varchar2 as return '@I'; end if; - if trim(tag1) = 'WHMM' then + if (tag1) = 'WHMM' then return '002'; end if; - if trim(tag1) = 'TCPJ' and srcvalue = '010105' then + if (tag1) = 'TCPJ' and srcvalue = '010105' then return '010105'; end if; - if trim(tag1) = 'TCPJ' and srcvalue != '010105' then + if (tag1) = 'TCPJ' and srcvalue != '010105' then return '003'; end if; - if trim(tag1) = 'TCPJ' and srcvalue != '010105' then + if (tag1) = 'TCPJ' and srcvalue != '010105' then return '003_ticket'; end if; - if trim(tag1) = 'TCJY' and srcvalue != '010105' then + if (tag1) = 'TCJY' and srcvalue != '010105' then return '003_ticket'; end if; - if trim(tag1) = 'TCJY' and srcvalue != '010105' then + if (tag1) = 'TCJY' and srcvalue != '010105' then return '003_ticket'; end if; - if trim(tag1) = 'YHHPD' then + if (tag1) = 'YHHPD' then return '002_foreign'; end if; - if trim(tag1) = 'WHWZ' then + if (tag1) = 'WHWZ' then return '002_foreign'; end if; - if trim(tag1) = 'WHLZ' then + if (tag1) = 'WHLZ' then return '002_foreign'; end if; - if trim(tag1) = 'DEWZ' then + if (tag1) = 'DEWZ' then return '024_out'; end if; - if trim(tag1) = 'DELZ' then + if (tag1) = 'DELZ' then return '024_out'; end if; diff --git a/hplsql/src/test/queries/local/create_function4.sql b/hplsql/src/test/queries/local/create_function4.sql index 21986e081719..bc3733436499 100644 --- a/hplsql/src/test/queries/local/create_function4.sql +++ b/hplsql/src/test/queries/local/create_function4.sql @@ -4,10 +4,10 @@ FUNCTION get(CODE VARCHAR2) RETURN VARCHAR2 AS TMPVAR := ''; - IF TRIM(TMPVAR) = '' THEN + IF (TMPVAR) = '' THEN RETURN '00080000'; ELSE - RETURN TRIM(TMPVAR); + RETURN (TMPVAR); END IF; EXCEPTION WHEN NO_DATA_FOUND THEN diff --git a/hplsql/src/test/queries/local/date.sql b/hplsql/src/test/queries/local/date.sql index 2ef474329afa..981075ce33d6 100644 --- a/hplsql/src/test/queries/local/date.sql +++ b/hplsql/src/test/queries/local/date.sql @@ -1,5 +1,4 @@ DATE '2014-12-20' DATE('2015-03-12'); -DATE('2015' || '-03-' || '12'); -DATE(TIMESTAMP '2015-03-12 10:58:34.111'); \ No newline at end of file +DATE('2015' || '-03-' || '12'); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/expr.sql b/hplsql/src/test/queries/local/expr.sql index 33388a25b121..e65dff98b546 100644 --- a/hplsql/src/test/queries/local/expr.sql +++ b/hplsql/src/test/queries/local/expr.sql @@ -15,7 +15,4 @@ PRINT c; PRINT 'Integer decrement'; c := 3; c := c - 1; -PRINT c; - -PRINT NVL(null - 3, 'Correct'); -PRINT NVL(null + 3, 'Correct'); \ No newline at end of file +PRINT c; \ No newline at end of file diff --git a/hplsql/src/test/queries/local/if2.sql b/hplsql/src/test/queries/local/if2.sql index b645b86e72f2..058847939715 100644 --- a/hplsql/src/test/queries/local/if2.sql +++ b/hplsql/src/test/queries/local/if2.sql @@ -1,4 +1,4 @@ -if not (coalesce(1,0) between 3 and 5) then +if not (nvl2(1,0) between 3 and 5) then print 'correct'; else print 'failed'; diff --git a/hplsql/src/test/queries/local/instr.sql b/hplsql/src/test/queries/local/instr.sql deleted file mode 100644 index 9cd8dcacc3cd..000000000000 --- a/hplsql/src/test/queries/local/instr.sql +++ /dev/null @@ -1,49 +0,0 @@ -IF INSTR('abc', 'b') = 2 THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -IF INSTR('abcabc', 'b', 3) = 5 THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -IF INSTR('abcabcabc', 'b', 3, 2) = 8 THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -IF INSTR('abcabcabc', 'b', -3) = 5 THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -IF INSTR('abcabcabc', 'b', -3, 2) = 2 THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -DECLARE c STRING; - -IF INSTR(c, 'b') IS NULL THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -IF INSTR(NULL, 'b') IS NULL THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; - -IF INSTR('', 'b') = 0 THEN - PRINT 'Correct'; -ELSE - PRINT 'Failed'; -END IF; \ No newline at end of file diff --git a/hplsql/src/test/queries/local/interval.sql b/hplsql/src/test/queries/local/interval.sql index e40fcbd229c7..98cf4927c7ee 100644 --- a/hplsql/src/test/queries/local/interval.sql +++ b/hplsql/src/test/queries/local/interval.sql @@ -2,15 +2,15 @@ DATE '2015-03-12' + 1 DAY; TIMESTAMP '2015-03-12' + 1 DAY; TIMESTAMP '2015-03-12 10:10:10.000' + 1 MICROSECOND; -DATE '2015-03-12' + NVL(NULL, 3) DAYS; -TIMESTAMP '2015-03-12' + NVL(NULL, 3) DAYS; +DATE '2015-03-12' + NVL2(NULL, NULL, 3) DAYS; +TIMESTAMP '2015-03-12' + NVL2(NULL, NULL, 3) DAYS; DATE '2015-03-12' - 1 DAY; TIMESTAMP '2015-03-12' - 1 DAY; TIMESTAMP '2015-03-12 10:10:10.000' - 1 MICROSECOND; -DATE '2015-03-12' - NVL(NULL, 3) DAYS; -TIMESTAMP '2015-03-12' - NVL(NULL, 3) DAYS; +DATE '2015-03-12' - NVL2(NULL, NULL, 3) DAYS; +TIMESTAMP '2015-03-12' - NVL2(NULL, NULL, 3) DAYS; TIMESTAMP '2015-03-12' - 1 DAY - 1 MICROSECOND; diff --git a/hplsql/src/test/queries/local/length.sql b/hplsql/src/test/queries/local/length.sql deleted file mode 100644 index 42cf3ccf0a84..000000000000 --- a/hplsql/src/test/queries/local/length.sql +++ /dev/null @@ -1 +0,0 @@ -LENGTH('Abc '); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/lower.sql b/hplsql/src/test/queries/local/lower.sql deleted file mode 100644 index f29b0e970eae..000000000000 --- a/hplsql/src/test/queries/local/lower.sql +++ /dev/null @@ -1 +0,0 @@ -LOWER('ABC'); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/mult_div.sql b/hplsql/src/test/queries/local/mult_div.sql index ebad8f46a15a..23fe88f91d2f 100644 --- a/hplsql/src/test/queries/local/mult_div.sql +++ b/hplsql/src/test/queries/local/mult_div.sql @@ -4,5 +4,5 @@ declare c int default 2; print a/b/c; -set a = 4 * 2 / cast(4 as int) /2; -set b = 4 * 2 /cast(4 as int)/2; \ No newline at end of file +set a = 4 * 2 / 4 / 2; +set b = 4 * 2 / 4 / 2; \ No newline at end of file diff --git a/hplsql/src/test/queries/local/nvl.sql b/hplsql/src/test/queries/local/nvl.sql deleted file mode 100644 index 1a843bcb28a0..000000000000 --- a/hplsql/src/test/queries/local/nvl.sql +++ /dev/null @@ -1,4 +0,0 @@ -NVL('First non-null', 1); -NVL(NULL, 'First non-null'); -NVL(NULL, 'First non-null', 1); -NVL(NULL, NULL, 'First non-null', 1); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/replace.sql b/hplsql/src/test/queries/local/replace.sql deleted file mode 100644 index 820aa067870f..000000000000 --- a/hplsql/src/test/queries/local/replace.sql +++ /dev/null @@ -1 +0,0 @@ -replace('2016-03-03', '-', ''); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/timestamp.sql b/hplsql/src/test/queries/local/timestamp.sql deleted file mode 100644 index 2971ceac8883..000000000000 --- a/hplsql/src/test/queries/local/timestamp.sql +++ /dev/null @@ -1,4 +0,0 @@ -TIMESTAMP '2015-03-03 11:39:31.123456'; -TIMESTAMP '2015-03-03 11:39:31.123'; -TIMESTAMP '2015-03-03 11:39:31'; -TIMESTAMP '2015-03-03-11.39.31.123'; \ No newline at end of file diff --git a/hplsql/src/test/queries/local/timestamp_iso.sql b/hplsql/src/test/queries/local/timestamp_iso.sql index 9bcdfe089040..e6601801fe12 100644 --- a/hplsql/src/test/queries/local/timestamp_iso.sql +++ b/hplsql/src/test/queries/local/timestamp_iso.sql @@ -1,2 +1 @@ -TIMESTAMP_ISO('2015-03-12'); -TIMESTAMP_ISO(DATE '2015-03-12'); \ No newline at end of file +TIMESTAMP_ISO('2015-03-12'); \ No newline at end of file diff --git a/hplsql/src/test/queries/local/trim.sql b/hplsql/src/test/queries/local/trim.sql deleted file mode 100644 index f8a2978d6503..000000000000 --- a/hplsql/src/test/queries/local/trim.sql +++ /dev/null @@ -1 +0,0 @@ -'#' || TRIM(' Hello ') || '#'; \ No newline at end of file diff --git a/hplsql/src/test/results/local/add.out.txt b/hplsql/src/test/results/local/add.out.txt index 37a195bfe189..7cfd6d68a017 100644 --- a/hplsql/src/test/results/local/add.out.txt +++ b/hplsql/src/test/results/local/add.out.txt @@ -1,2 +1,2 @@ -2015-01-01 -2015-01-01 +DATE '2015-01-01' +DATE '2015-01-01' diff --git a/hplsql/src/test/results/local/cast.out.txt b/hplsql/src/test/results/local/cast.out.txt deleted file mode 100644 index f3de493e67c1..000000000000 --- a/hplsql/src/test/results/local/cast.out.txt +++ /dev/null @@ -1,8 +0,0 @@ -Ln:1 FUNC CAST -A -Ln:2 FUNC CAST -Ab -Ln:3 FUNC CAST -Abc -Ln:4 FUNC CAST -2015-03-12 diff --git a/hplsql/src/test/results/local/cast2.out.txt b/hplsql/src/test/results/local/cast2.out.txt deleted file mode 100644 index fc136b8f3f20..000000000000 --- a/hplsql/src/test/results/local/cast2.out.txt +++ /dev/null @@ -1,15 +0,0 @@ -Ln:1 SET temp_int = 1 -Ln:2 PRINT -1 -Ln:3 SET temp_float = 1.2 -Ln:4 PRINT -1.2 -Ln:5 SET temp_double = 1.2 -Ln:6 PRINT -1.2 -Ln:7 SET temp_decimal = 1.2 -Ln:8 PRINT -1.2 -Ln:9 SET temp_string = '1.2' -Ln:10 PRINT -1.2 diff --git a/hplsql/src/test/results/local/coalesce.out.txt b/hplsql/src/test/results/local/coalesce.out.txt deleted file mode 100644 index a111c8575fba..000000000000 --- a/hplsql/src/test/results/local/coalesce.out.txt +++ /dev/null @@ -1,4 +0,0 @@ -First non-null -First non-null -First non-null -First non-null diff --git a/hplsql/src/test/results/local/date.out.txt b/hplsql/src/test/results/local/date.out.txt index 118bd29a8c2a..1e4435a3a244 100644 --- a/hplsql/src/test/results/local/date.out.txt +++ b/hplsql/src/test/results/local/date.out.txt @@ -1,4 +1,3 @@ -2014-12-20 -2015-03-12 -2015-03-12 -2015-03-12 +DATE '2014-12-20' +DATE '2015-03-12' +DATE '2015-03-12' diff --git a/hplsql/src/test/results/local/declare.out.txt b/hplsql/src/test/results/local/declare.out.txt index 2b6a6d7d6718..cb970ea048a7 100644 --- a/hplsql/src/test/results/local/declare.out.txt +++ b/hplsql/src/test/results/local/declare.out.txt @@ -9,5 +9,5 @@ Ln:12 DECLARE status INT = 1 Ln:12 DECLARE status2 INT = 1 Ln:13 DECLARE count SMALLINT Ln:13 DECLARE limit INT = 100 -Ln:15 DECLARE dt DATE = 2015-05-13 -Ln:16 DECLARE ts TIMESTAMP = 2015-05-13 11:10:01 +Ln:15 DECLARE dt DATE = DATE '2015-05-13' +Ln:16 DECLARE ts TIMESTAMP = TIMESTAMP '2015-05-13 11:10:01' diff --git a/hplsql/src/test/results/local/expr.out.txt b/hplsql/src/test/results/local/expr.out.txt index 377f8d14f453..1ae8e373bcda 100644 --- a/hplsql/src/test/results/local/expr.out.txt +++ b/hplsql/src/test/results/local/expr.out.txt @@ -23,7 +23,3 @@ Ln:16 SET c = 3 Ln:17 SET c = 2 Ln:18 PRINT 2 -Ln:20 PRINT -Correct -Ln:21 PRINT -Correct diff --git a/hplsql/src/test/results/local/instr.out.txt b/hplsql/src/test/results/local/instr.out.txt deleted file mode 100644 index 9a23e12ae6b4..000000000000 --- a/hplsql/src/test/results/local/instr.out.txt +++ /dev/null @@ -1,33 +0,0 @@ -Ln:1 IF -Ln:1 IF TRUE executed -Ln:2 PRINT -Correct -Ln:7 IF -Ln:7 IF TRUE executed -Ln:8 PRINT -Correct -Ln:13 IF -Ln:13 IF TRUE executed -Ln:14 PRINT -Correct -Ln:19 IF -Ln:19 IF TRUE executed -Ln:20 PRINT -Correct -Ln:25 IF -Ln:25 IF TRUE executed -Ln:26 PRINT -Correct -Ln:31 DECLARE c STRING -Ln:33 IF -Ln:33 IF TRUE executed -Ln:34 PRINT -Correct -Ln:39 IF -Ln:39 IF TRUE executed -Ln:40 PRINT -Correct -Ln:45 IF -Ln:45 IF TRUE executed -Ln:46 PRINT -Correct diff --git a/hplsql/src/test/results/local/interval.out.txt b/hplsql/src/test/results/local/interval.out.txt index d73d95a49615..3e14a4b45245 100644 --- a/hplsql/src/test/results/local/interval.out.txt +++ b/hplsql/src/test/results/local/interval.out.txt @@ -1,12 +1,12 @@ -2015-03-13 -2015-03-13 00:00:00 -2015-03-12 10:10:10.001 -2015-03-15 -2015-03-15 00:00:00 -2015-03-11 -2015-03-11 00:00:00 -2015-03-12 10:10:09.999 -2015-03-09 -2015-03-09 00:00:00 -2015-03-10 23:59:59 -2016-01-24 +DATE '2015-03-13' +TIMESTAMP '2015-03-13 00:00:00' +TIMESTAMP '2015-03-12 10:10:10.001' +DATE '2015-03-15' +TIMESTAMP '2015-03-15 00:00:00' +DATE '2015-03-11' +TIMESTAMP '2015-03-11 00:00:00' +TIMESTAMP '2015-03-12 10:10:09.999' +DATE '2015-03-09' +TIMESTAMP '2015-03-09 00:00:00' +TIMESTAMP '2015-03-10 23:59:59' +DATE '2016-01-24' diff --git a/hplsql/src/test/results/local/invalid_syntax.out.txt b/hplsql/src/test/results/local/invalid_syntax.out.txt index 56206511ecb8..26eba40e305f 100644 --- a/hplsql/src/test/results/local/invalid_syntax.out.txt +++ b/hplsql/src/test/results/local/invalid_syntax.out.txt @@ -1,3 +1,3 @@ Syntax error at line 1:27 no viable alternative at input 'b,' -Syntax error at line 1:27 mismatched input ',' expecting {, '@', '#', '!', T_ACTION, T_ADD2, T_ALL, T_ALLOCATE, T_ALTER, T_AND, T_ANSI_NULLS, T_ANSI_PADDING, T_AS, T_ASC, T_ASSOCIATE, T_AT, T_AUTO_INCREMENT, T_AVG, T_BATCHSIZE, T_BEGIN, T_BETWEEN, T_BIGINT, T_BINARY_DOUBLE, T_BINARY_FLOAT, T_BIT, T_BODY, T_BREAK, T_BULK, T_BY, T_BYTE, T_CALL, T_CALLER, T_CASCADE, T_CASE, T_CASESPECIFIC, T_CAST, T_CHAR, T_CHARACTER, T_CHARSET, T_CLIENT, T_CLOSE, T_CLUSTERED, T_CMP, T_COLLECT, T_COLLECTION, T_COLUMN, T_COMMENT, T_CONSTANT, T_COMMIT, T_COMPRESS, T_CONCAT, T_CONDITION, T_CONSTRAINT, T_CONTINUE, T_COPY, T_COUNT, T_COUNT_BIG, T_CREATE, T_CREATION, T_CREATOR, T_CS, T_CURRENT, T_CURRENT_SCHEMA, T_CURSOR, T_DATABASE, T_DATA, T_DATE, T_DATETIME, T_DAY, T_DAYS, T_DEC, T_DECIMAL, T_DECLARE, T_DEFAULT, T_DEFERRED, T_DEFINED, T_DEFINER, T_DEFINITION, T_DELETE, T_DELIMITED, T_DELIMITER, T_DESC, T_DESCRIBE, T_DIAGNOSTICS, T_DIR, T_DIRECTORY, T_DISTINCT, T_DISTRIBUTE, T_DO, T_DOUBLE, T_DROP, T_DYNAMIC, T_ENABLE, T_END, T_ENGINE, T_ESCAPED, T_EXCEPT, T_EXEC, T_EXECUTE, T_EXCEPTION, T_EXCLUSIVE, T_EXISTS, T_EXIT, T_FALLBACK, T_FALSE, T_FETCH, T_FIELDS, T_FILE, T_FILES, T_FLOAT, T_FOR, T_FOREIGN, T_FORMAT, T_FOUND, T_FROM, T_FULL, T_FUNCTION, T_GET, T_GLOBAL, T_GO, T_GRANT, T_GROUP, T_HANDLER, T_HASH, T_HAVING, T_HDFS, T_HIVE, T_HOST, T_IDENTITY, T_IF, T_IGNORE, T_IMMEDIATE, T_IN, T_INCLUDE, T_INDEX, T_INITRANS, T_INNER, T_INOUT, T_INSERT, T_INT, T_INT2, T_INT4, T_INT8, T_INTEGER, T_INTERSECT, T_INTERVAL, T_INTO, T_INVOKER, T_IS, T_ISOPEN, T_ITEMS, T_JOIN, T_KEEP, T_KEY, T_KEYS, T_LANGUAGE, T_LEAVE, T_LEFT, T_LIKE, T_LIMIT, T_LINES, T_LOCAL, T_LOCATION, T_LOCATOR, T_LOCATORS, T_LOCKS, T_LOG, T_LOGGED, T_LOGGING, T_LOOP, T_MAP, T_MATCHED, T_MAX, T_MAXTRANS, T_MERGE, T_MESSAGE_TEXT, T_MICROSECOND, T_MICROSECONDS, T_MIN, T_MULTISET, T_NCHAR, T_NEW, T_NVARCHAR, T_NO, T_NOCOUNT, T_NOCOMPRESS, T_NOLOGGING, T_NONE, T_NOT, T_NOTFOUND, T_NULL, T_NUMERIC, T_NUMBER, T_OBJECT, T_OFF, T_ON, T_ONLY, T_OPEN, T_OR, T_ORDER, T_OUT, T_OUTER, T_OVER, T_OVERWRITE, T_OWNER, T_PACKAGE, T_PARTITION, T_PCTFREE, T_PCTUSED, T_PRECISION, T_PRESERVE, T_PRIMARY, T_PRINT, T_PROC, T_PROCEDURE, T_QUALIFY, T_QUERY_BAND, T_QUIT, T_QUOTED_IDENTIFIER, T_RAISE, T_REAL, T_REFERENCES, T_REGEXP, T_REPLACE, T_RESIGNAL, T_RESTRICT, T_RESULT, T_RESULT_SET_LOCATOR, T_RETURN, T_RETURNS, T_REVERSE, T_RIGHT, T_RLIKE, T_ROLE, T_ROLLBACK, T_ROW, T_ROWS, T_ROW_COUNT, T_RR, T_RS, T_PWD, T_TRIM, T_SCHEMA, T_SECOND, T_SECONDS, T_SECURITY, T_SEGMENT, T_SEL, T_SELECT, T_SET, T_SESSION, T_SESSIONS, T_SETS, T_SHARE, T_SIGNAL, T_SIMPLE_DOUBLE, T_SIMPLE_FLOAT, T_SMALLDATETIME, T_SMALLINT, T_SQL, T_SQLEXCEPTION, T_SQLINSERT, T_SQLSTATE, T_SQLWARNING, T_STATS, T_STATISTICS, T_STEP, T_STORAGE, T_STORED, T_STRING, T_SUBDIR, T_SUBSTRING, T_SUM, T_SUMMARY, T_SYS_REFCURSOR, T_TABLE, T_TABLESPACE, T_TEMPORARY, T_TERMINATED, T_TEXTIMAGE_ON, T_THEN, T_TIMESTAMP, T_TITLE, T_TO, T_TOP, T_TRANSACTION, T_TRUE, T_TRUNCATE, T_TYPE, T_UNIQUE, T_UPDATE, T_UR, T_USE, T_USING, T_VALUE, T_VALUES, T_VAR, T_VARCHAR, T_VARCHAR2, T_VARYING, T_VOLATILE, T_WHILE, T_WITH, T_WITHOUT, T_WORK, T_XACT_ABORT, T_XML, T_YES, T_ACTIVITY_COUNT, T_CUME_DIST, T_CURRENT_DATE, T_CURRENT_TIMESTAMP, T_CURRENT_USER, T_DENSE_RANK, T_FIRST_VALUE, T_LAG, T_LAST_VALUE, T_LEAD, T_MAX_PART_STRING, T_MIN_PART_STRING, T_MAX_PART_INT, T_MIN_PART_INT, T_MAX_PART_DATE, T_MIN_PART_DATE, T_PART_COUNT, T_PART_LOC, T_RANK, T_ROW_NUMBER, T_STDEV, T_SYSDATE, T_VARIANCE, T_USER, '+', '/', '.', '<', '*', '(', ';', '-', L_ID, L_S_STRING, L_D_STRING, L_INT, L_DEC, L_LABEL} +Syntax error at line 1:27 mismatched input ',' expecting {, '@', '#', '!', T_ACTION, T_ADD2, T_ALL, T_ALLOCATE, T_ALTER, T_AND, T_ANSI_NULLS, T_ANSI_PADDING, T_AS, T_ASC, T_ASSOCIATE, T_AT, T_AUTO_INCREMENT, T_AVG, T_BATCHSIZE, T_BEGIN, T_BETWEEN, T_BIGINT, T_BINARY_DOUBLE, T_BINARY_FLOAT, T_BIT, T_BODY, T_BREAK, T_BULK, T_BY, T_BYTE, T_CALL, T_CALLER, T_CASCADE, T_CASE, T_CASESPECIFIC, T_CAST, T_CHAR, T_CHARACTER, T_CHARSET, T_CLIENT, T_CLOSE, T_CLUSTERED, T_CMP, T_COLLECT, T_COLLECTION, T_COLUMN, T_COMMENT, T_CONSTANT, T_COMMIT, T_COMPRESS, T_CONCAT, T_CONDITION, T_CONSTRAINT, T_CONTINUE, T_COPY, T_COUNT, T_COUNT_BIG, T_CREATE, T_CREATION, T_CREATOR, T_CS, T_CURRENT, T_CURRENT_SCHEMA, T_CURSOR, T_DATABASE, T_DATA, T_DATE, T_DATETIME, T_DAY, T_DAYS, T_DEC, T_DECIMAL, T_DECLARE, T_DEFAULT, T_DEFERRED, T_DEFINED, T_DEFINER, T_DEFINITION, T_DELETE, T_DELIMITED, T_DELIMITER, T_DESC, T_DESCRIBE, T_DIAGNOSTICS, T_DIR, T_DIRECTORY, T_DISTINCT, T_DISTRIBUTE, T_DO, T_DOUBLE, T_DROP, T_DYNAMIC, T_ENABLE, T_END, T_ENGINE, T_ESCAPED, T_EXCEPT, T_EXEC, T_EXECUTE, T_EXCEPTION, T_EXCLUSIVE, T_EXISTS, T_EXIT, T_FALLBACK, T_FALSE, T_FETCH, T_FIELDS, T_FILE, T_FILES, T_FLOAT, T_FOR, T_FOREIGN, T_FORMAT, T_FOUND, T_FROM, T_FULL, T_FUNCTION, T_GET, T_GLOBAL, T_GO, T_GRANT, T_GROUP, T_HANDLER, T_HASH, T_HAVING, T_HDFS, T_HIVE, T_HOST, T_IDENTITY, T_IF, T_IGNORE, T_IMMEDIATE, T_IN, T_INCLUDE, T_INDEX, T_INITRANS, T_INNER, T_INOUT, T_INSERT, T_INT, T_INT2, T_INT4, T_INT8, T_INTEGER, T_INTERSECT, T_INTERVAL, T_INTO, T_INVOKER, T_IS, T_ISOPEN, T_ITEMS, T_JOIN, T_KEEP, T_KEY, T_KEYS, T_LANGUAGE, T_LEAVE, T_LEFT, T_LIKE, T_LIMIT, T_LINES, T_LOCAL, T_LOCATION, T_LOCATOR, T_LOCATORS, T_LOCKS, T_LOG, T_LOGGED, T_LOGGING, T_LOOP, T_MAP, T_MATCHED, T_MAX, T_MAXTRANS, T_MERGE, T_MESSAGE_TEXT, T_MICROSECOND, T_MICROSECONDS, T_MIN, T_MULTISET, T_NCHAR, T_NEW, T_NVARCHAR, T_NO, T_NOCOUNT, T_NOCOMPRESS, T_NOLOGGING, T_NONE, T_NOT, T_NOTFOUND, T_NULL, T_NUMERIC, T_NUMBER, T_OBJECT, T_OFF, T_ON, T_ONLY, T_OPEN, T_OR, T_ORDER, T_OUT, T_OUTER, T_OVER, T_OVERWRITE, T_OWNER, T_PACKAGE, T_PARTITION, T_PCTFREE, T_PCTUSED, T_PRECISION, T_PRESERVE, T_PRIMARY, T_PRINT, T_PROC, T_PROCEDURE, T_QUALIFY, T_QUERY_BAND, T_QUIT, T_QUOTED_IDENTIFIER, T_RAISE, T_REAL, T_REFERENCES, T_REGEXP, T_REPLACE, T_RESIGNAL, T_RESTRICT, T_RESULT, T_RESULT_SET_LOCATOR, T_RETURN, T_RETURNS, T_REVERSE, T_RIGHT, T_RLIKE, T_ROLE, T_ROLLBACK, T_ROW, T_ROWS, T_ROW_COUNT, T_RR, T_RS, T_PWD, T_TRIM, T_SCHEMA, T_SECOND, T_SECONDS, T_SECURITY, T_SEGMENT, T_SEL, T_SELECT, T_SET, T_SESSION, T_SESSIONS, T_SETS, T_SHARE, T_SIGNAL, T_SIMPLE_DOUBLE, T_SIMPLE_FLOAT, T_SMALLDATETIME, T_SMALLINT, T_SQL, T_SQLEXCEPTION, T_SQLINSERT, T_SQLSTATE, T_SQLWARNING, T_STATS, T_STATISTICS, T_STEP, T_STORAGE, T_STORED, T_STRING, T_SUBDIR, T_SUBSTRING, T_SUM, T_SUMMARY, T_SYS_REFCURSOR, T_TABLE, T_TABLESPACE, T_TEMPORARY, T_TERMINATED, T_TEXTIMAGE_ON, T_THEN, T_TIMESTAMP, T_TITLE, T_TO, T_TOP, T_TRANSACTION, T_TRUE, T_TRUNCATE, T_TYPE, T_UNIQUE, T_UPDATE, T_UR, T_USE, T_USING, T_VALUE, T_VALUES, T_VAR, T_VARCHAR, T_VARCHAR2, T_VARYING, T_VOLATILE, T_WHILE, T_WITH, T_WITHOUT, T_WORK, T_XACT_ABORT, T_XML, T_YES, T_ACTIVITY_COUNT, T_CUME_DIST, T_CURRENT_DATE, T_CURRENT_TIME_MILLIS, T_CURRENT_TIMESTAMP, T_CURRENT_USER, T_DENSE_RANK, T_FIRST_VALUE, T_LAG, T_LAST_VALUE, T_LEAD, T_MAX_PART_STRING, T_MIN_PART_STRING, T_MAX_PART_INT, T_MIN_PART_INT, T_MAX_PART_DATE, T_MIN_PART_DATE, T_PART_COUNT, T_PART_LOC, T_RANK, T_ROW_NUMBER, T_STDEV, T_SYSDATE, T_VARIANCE, T_USER, '+', '/', '.', '<', '*', '(', ';', '-', L_ID, L_S_STRING, L_D_STRING, L_INT, L_DEC, L_LABEL} Ln:1 identifier 'CREATE' must be declared. diff --git a/hplsql/src/test/results/local/length.out.txt b/hplsql/src/test/results/local/length.out.txt deleted file mode 100644 index b8626c4cff28..000000000000 --- a/hplsql/src/test/results/local/length.out.txt +++ /dev/null @@ -1 +0,0 @@ -4 diff --git a/hplsql/src/test/results/local/lower.out.txt b/hplsql/src/test/results/local/lower.out.txt deleted file mode 100644 index 8baef1b4abc4..000000000000 --- a/hplsql/src/test/results/local/lower.out.txt +++ /dev/null @@ -1 +0,0 @@ -abc diff --git a/hplsql/src/test/results/local/nvl.out.txt b/hplsql/src/test/results/local/nvl.out.txt deleted file mode 100644 index a111c8575fba..000000000000 --- a/hplsql/src/test/results/local/nvl.out.txt +++ /dev/null @@ -1,4 +0,0 @@ -First non-null -First non-null -First non-null -First non-null diff --git a/hplsql/src/test/results/local/replace.out.txt b/hplsql/src/test/results/local/replace.out.txt deleted file mode 100644 index 2cd3602ff952..000000000000 --- a/hplsql/src/test/results/local/replace.out.txt +++ /dev/null @@ -1 +0,0 @@ -20160303 diff --git a/hplsql/src/test/results/local/sub.out.txt b/hplsql/src/test/results/local/sub.out.txt index d883fcd7d786..8d3b91922fb1 100644 --- a/hplsql/src/test/results/local/sub.out.txt +++ b/hplsql/src/test/results/local/sub.out.txt @@ -1 +1 @@ -2014-12-31 +DATE '2014-12-31' diff --git a/hplsql/src/test/results/local/timestamp.out.txt b/hplsql/src/test/results/local/timestamp.out.txt deleted file mode 100644 index 223b5f492cb2..000000000000 --- a/hplsql/src/test/results/local/timestamp.out.txt +++ /dev/null @@ -1,4 +0,0 @@ -2015-03-03 11:39:31.123 -2015-03-03 11:39:31.123 -2015-03-03 11:39:31 -2015-03-03 11:39:31.123 diff --git a/hplsql/src/test/results/local/timestamp_iso.out.txt b/hplsql/src/test/results/local/timestamp_iso.out.txt index 997df7fffca7..dc697a127d64 100644 --- a/hplsql/src/test/results/local/timestamp_iso.out.txt +++ b/hplsql/src/test/results/local/timestamp_iso.out.txt @@ -1,2 +1 @@ -2015-03-12 00:00:00 -2015-03-12 00:00:00 +TIMESTAMP '2015-03-12 00:00:00' diff --git a/hplsql/src/test/results/local/to_char.out.txt b/hplsql/src/test/results/local/to_char.out.txt index 22e8ceff87b2..651eb14fee42 100644 --- a/hplsql/src/test/results/local/to_char.out.txt +++ b/hplsql/src/test/results/local/to_char.out.txt @@ -1 +1 @@ -2015-04-02 +DATE '2015-04-02' diff --git a/hplsql/src/test/results/local/to_timestamp.out.txt b/hplsql/src/test/results/local/to_timestamp.out.txt index 1ee727873aab..08884ae0cc53 100644 --- a/hplsql/src/test/results/local/to_timestamp.out.txt +++ b/hplsql/src/test/results/local/to_timestamp.out.txt @@ -1,4 +1,4 @@ -2015-04-02 00:00:00 -2015-04-02 00:00:00 -2015-04-02 00:00:00 -2015-04-02 13:51:31 +TIMESTAMP '2015-04-02 00:00:00' +TIMESTAMP '2015-04-02 00:00:00' +TIMESTAMP '2015-04-02 00:00:00' +TIMESTAMP '2015-04-02 13:51:31' diff --git a/hplsql/src/test/results/local/trim.out.txt b/hplsql/src/test/results/local/trim.out.txt deleted file mode 100644 index bbf851d14c6c..000000000000 --- a/hplsql/src/test/results/local/trim.out.txt +++ /dev/null @@ -1 +0,0 @@ -#Hello# diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java index a7a547adb8d5..4118013769fe 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java @@ -27,6 +27,8 @@ import java.io.File; import java.io.FileOutputStream; import java.io.PrintStream; +import java.sql.Date; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -610,12 +612,268 @@ public void testNullCast() throws Throwable { testScriptFile(SCRIPT_TEXT, args(), "^(.(?!(NullPointerException)))*$", OutStream.ERR); } + @Test + public void testACTIVITY_COUNTHplSqlFunction() throws Throwable { + String SCRIPT_TEXT = + "DROP TABLE IF EXISTS result;\n" + + "CREATE TABLE result (col1 string);\n" + + "INSERT INTO result VALUES('Alice');\n" + + "INSERT INTO result VALUES('Bob');\n" + + "SELECT * FROM result;\n" + + "SELECT ACTIVITY_COUNT;"; + testScriptFile(SCRIPT_TEXT, args(), "2"); + } + + @Test + public void testCASTHplSqlFunction1() throws Throwable { + String SCRIPT_TEXT = "SELECT CAST('Abc' AS CHAR(1));"; + testScriptFile(SCRIPT_TEXT, args(), "A"); + } + + @Test + public void testCASTHplSqlFunction2() throws Throwable { + String SCRIPT_TEXT = "SELECT CAST(TIMESTAMP '2015-03-12 10:58:34.111' AS CHAR(10));"; + testScriptFile(SCRIPT_TEXT, args(), "2015-03-12"); + } + + @Test + public void testCHARHplSqlFunction() throws Throwable { + String SCRIPT_TEXT = "select CHAR(2023)"; + testScriptFile(SCRIPT_TEXT, args(), "2023"); + } + + @Test + public void testCOALESCEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "select COALESCE(null,123,2023)"; + testScriptFile(SCRIPT_TEXT, args(), "123"); + } + + @Test + public void testCONCATHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "select CONCAT('a', 'b', NULL, 'c')"; + testScriptFile(SCRIPT_TEXT, args(), "abc"); + } + + @Test + public void testCURRENTHplSQLFunction1() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT DATE;"; + testCurrentDate(SCRIPT_TEXT); + } + + private void testCurrentDate(String SCRIPT_TEXT) throws Throwable { + Date today = new Date(System.currentTimeMillis()); + testScriptFile(SCRIPT_TEXT, args(), today.toString()); + } + + @Test + public void testCURRENTHplSQLFunction2() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT TIMESTAMP;"; + testCurrentTimestamp(SCRIPT_TEXT); + } + + private void testCurrentTimestamp(String SCRIPT_TEXT) throws Throwable { + Timestamp today = new Timestamp(System.currentTimeMillis()); + String timestamp = today.toString(); + testScriptFile(SCRIPT_TEXT, args(), timestamp.substring(0, timestamp.length() - 9)); + } + + @Test + public void testCURRENTHplSQLFunction3() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT USER;"; + testScriptFile(SCRIPT_TEXT, args(), System.getProperty("user.name")); + } + + @Test + public void testCURRENT_DATEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT_DATE;"; + testCurrentDate(SCRIPT_TEXT); + } + + @Test + public void testCURRENT_TIME_MILLISHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT_TIME_MILLIS();"; + testScriptFile(SCRIPT_TEXT, args(), String.valueOf(System.currentTimeMillis() / 100000)); + } + + @Test + public void testCURRENT_TIMESTAMPHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT_TIMESTAMP;"; + testCurrentTimestamp(SCRIPT_TEXT); + } + + @Test + public void testCURRENT_USERHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT CURRENT_USER;"; + testScriptFile(SCRIPT_TEXT, args(), System.getProperty("user.name")); + } + + @Test + public void testDATEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT DATE('2015-03-12');"; + testScriptFile(SCRIPT_TEXT, args(), "2015-03-12"); + } + + @Test + public void testDECODEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "DECLARE var1 INT DEFAULT 3;\n" + "SELECT DECODE (var1, 1, 'A', 2, 'B', 3, 'C');"; + testScriptFile(SCRIPT_TEXT, args(), "C"); + } + + @Test + public void testFROM_UNIXTIMEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT from_unixtime(1447141681, 'yyyy-MM-dd');"; + testScriptFile(SCRIPT_TEXT, args(), "2015-11-"); + } + + @Test + public void testINSTRHplSQLFunction1() throws Throwable { + String SCRIPT_TEXT = "SELECT INSTR('abc', 'b');"; + testScriptFile(SCRIPT_TEXT, args(), "2"); + } + + @Test + public void testINSTRHplSQLFunction2() throws Throwable { + String SCRIPT_TEXT = "SELECT INSTR('abcabcabc', 'b', 3, 2);"; + testScriptFile(SCRIPT_TEXT, args(), "8"); + } + + @Test + public void testLOWERHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT LOWER('ABC');"; + testScriptFile(SCRIPT_TEXT, args(), "abc"); + } + + @Test + public void testLENHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT LEN('Abc ');"; + testScriptFile(SCRIPT_TEXT, args(), "3"); + } + + @Test + public void testLENGTHHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT LENGTH('Abc ');"; + testScriptFile(SCRIPT_TEXT, args(), "4"); + } + + @Test + public void testMODHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT MOD(5,2);"; + testScriptFile(SCRIPT_TEXT, args(), "1"); + } + + @Test + public void testNOWHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT NOW();"; + testCurrentTimestamp(SCRIPT_TEXT); + } + + @Test + public void testNVLHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT NVL(NULL, 100);"; + testScriptFile(SCRIPT_TEXT, args(), "100"); + } + + @Test + public void testNVL2HplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT NVL2(NULL, 100, 200);"; + testScriptFile(SCRIPT_TEXT, args(), "200"); + } + + @Test + public void testREPLACEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT replace('2016-03-03', '-', '');"; + testScriptFile(SCRIPT_TEXT, args(), "20160303"); + } + + @Test + public void testSUBSTRHplSQLFunction1() throws Throwable { + String SCRIPT_TEXT = "SELECT SUBSTR('Remark', 3);"; + testScriptFile(SCRIPT_TEXT, args(), "mark"); + } + + @Test + public void testSUBSTRHplSQLFunction2() throws Throwable { + String SCRIPT_TEXT = "SELECT SUBSTR('Remark', 3, 3);"; + testScriptFile(SCRIPT_TEXT, args(), "mar"); + } + + @Test + public void testSUBSTRINGHplSQLFunction1() throws Throwable { + String SCRIPT_TEXT = "SELECT SUBSTRING('Remark', 3);"; + testScriptFile(SCRIPT_TEXT, args(), "mark"); + } + + @Test + public void testSUBSTRINGHplSQLFunction2() throws Throwable { + String SCRIPT_TEXT = "SELECT SUBSTRING('Remark', 3, 3);"; + testScriptFile(SCRIPT_TEXT, args(), "mar"); + } + + @Test + public void testSYSDATEHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT SYSDATE;"; + testCurrentTimestamp(SCRIPT_TEXT); + } + + @Test + public void testTIMESTAMP_ISOHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT TIMESTAMP_ISO('2015-03-12');"; + testScriptFile(SCRIPT_TEXT, args(), "2015-03-12 00:00:00"); + } + + @Test + public void testTO_CHARHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT TO_CHAR(CURRENT_DATE);"; + testCurrentDate(SCRIPT_TEXT); + } + + @Test + public void testTO_TIMESTAMPHplSQLFunction1() throws Throwable { + String SCRIPT_TEXT = "SELECT TO_TIMESTAMP('2015-04-02', 'YYYY-MM-DD');"; + testScriptFile(SCRIPT_TEXT, args(), "2015-04-02 00:00:00.0"); + } + + @Test + public void testTO_TIMESTAMPHplSQLFunction2() throws Throwable { + String SCRIPT_TEXT = "SELECT TO_TIMESTAMP('04/02/2015', 'mm/dd/yyyy');"; + testScriptFile(SCRIPT_TEXT, args(), "2015-04-02 00:00:00.0"); + } + + @Test + public void testTO_TIMESTAMPHplSQLFunction3() throws Throwable { + String SCRIPT_TEXT = "SELECT TO_TIMESTAMP('2015-04-02 13:51:31', 'YYYY-MM-DD HH24:MI:SS');"; + testScriptFile(SCRIPT_TEXT, args(), "2015-04-02 13:51:31.0"); + } + + @Test + public void testTRIMHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT '#' || TRIM(' Hello ') || '#';"; + testScriptFile(SCRIPT_TEXT, args(), "#Hello#"); + } + + @Test + public void testUNIX_TIMESTAMPHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT UNIX_TIMESTAMP()"; + testScriptFile(SCRIPT_TEXT, args(), String.valueOf(System.currentTimeMillis()/10000)); + } + + @Test + public void testUPPERHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT UPPER('abc');"; + testScriptFile(SCRIPT_TEXT, args(), "ABC"); + } + + @Test + public void testUSERHplSQLFunction() throws Throwable { + String SCRIPT_TEXT = "SELECT USER;"; + testScriptFile(SCRIPT_TEXT, args(), System.getProperty("user.name")); + } + private static List args() { return Arrays.asList("-d", BeeLine.BEELINE_DEFAULT_JDBC_DRIVER, "-u", miniHS2.getBaseJdbcURL() + ";mode=hplsql", "-n", userName); } - private void testScriptFile(String scriptText, List argList, String expectedPattern) throws Throwable { testScriptFile(scriptText, argList, expectedPattern, OutStream.OUT); From 06ef7c8231501c0314875d640f081adc35a70db6 Mon Sep 17 00:00:00 2001 From: dengzh Date: Tue, 16 Jan 2024 09:17:29 +0800 Subject: [PATCH 135/179] HIVE-27955: Missing Postgres driver when start services from Docker compose (#4948) --- packaging/src/docker/README.md | 36 ++++++++++++++++++------- packaging/src/docker/docker-compose.yml | 3 +++ 2 files changed, 30 insertions(+), 9 deletions(-) diff --git a/packaging/src/docker/README.md b/packaging/src/docker/README.md index ebcbfcaf617d..e452013aac52 100644 --- a/packaging/src/docker/README.md +++ b/packaging/src/docker/README.md @@ -57,18 +57,19 @@ For a quick start, launch the Metastore with Derby, Everything would be lost when the service is down. In order to save the Hive table's schema and data, start the container with an external Postgres and Volume to keep them, ```shell - docker run -d -p 9083:9083 --env SERVICE_NAME=metastore \ - --env DB_DRIVER=postgres \ + docker run -d -p 9083:9083 --env SERVICE_NAME=metastore --env DB_DRIVER=postgres \ --env SERVICE_OPTS="-Djavax.jdo.option.ConnectionDriverName=org.postgresql.Driver -Djavax.jdo.option.ConnectionURL=jdbc:postgresql://postgres:5432/metastore_db -Djavax.jdo.option.ConnectionUserName=hive -Djavax.jdo.option.ConnectionPassword=password" \ --mount source=warehouse,target=/opt/hive/data/warehouse \ + --mount type=bind,source=`mvn help:evaluate -Dexpression=settings.localRepository -q -DforceStdout`/org/postgresql/postgresql/42.5.1/postgresql-42.5.1.jar,target=/opt/hive/lib/postgres.jar \ --name metastore-standalone apache/hive:${HIVE_VERSION} ``` If you want to use your own `hdfs-site.xml` or `yarn-site.xml` for the service, you can provide the environment variable `HIVE_CUSTOM_CONF_DIR` for the command. For instance, put the custom configuration file under the directory `/opt/hive/conf`, then run, ```shell - docker run -d -p 9083:9083 --env SERVICE_NAME=metastore \ - --env DB_DRIVER=postgres -v /opt/hive/conf:/hive_custom_conf --env HIVE_CUSTOM_CONF_DIR=/hive_custom_conf \ + docker run -d -p 9083:9083 --env SERVICE_NAME=metastore --env DB_DRIVER=postgres \ + -v /opt/hive/conf:/hive_custom_conf --env HIVE_CUSTOM_CONF_DIR=/hive_custom_conf \ + --mount type=bind,source=`mvn help:evaluate -Dexpression=settings.localRepository -q -DforceStdout`/org/postgresql/postgresql/42.5.1/postgresql-42.5.1.jar,target=/opt/hive/lib/postgres.jar \ --name metastore apache/hive:${HIVE_VERSION} ``` @@ -81,7 +82,7 @@ then add "--env SCHEMA_COMMAND=upgradeSchema" to the command. Launch the HiveServer2 with an embedded Metastore, ```shell - docker run -d -p 10000:10000 -p 10002:10002 --env SERVICE_NAME=hiveserver2 --name hiveserver2-standalone apache/hive:${HIVE_VERSION} + docker run -d -p 10000:10000 -p 10002:10002 --env SERVICE_NAME=hiveserver2 --name hiveserver2-standalone apache/hive:${HIVE_VERSION} ``` or specify a remote Metastore if it's available, ```shell @@ -109,12 +110,24 @@ To skip schematool initialisation or upgrade for metastore use `IS_RESUME="true" - HiveServer2, Metastore -To get a quick overview of both HiveServer2 and Metastore, you can run: +To get a quick overview of both HiveServer2 and Metastore, there is a `docker-compose.yml` placed under `packaging/src/docker` for this purpose, +specify the `POSTGRES_LOCAL_PATH` first: +```shell +export POSTGRES_LOCAL_PATH=your_local_path_to_postgres_driver +``` +Example: +```shell +mvn dependency:copy -Dartifact="org.postgresql:postgresql:42.5.1" && \ +export POSTGRES_LOCAL_PATH=`mvn help:evaluate -Dexpression=settings.localRepository -q -DforceStdout`/org/postgresql/postgresql/42.5.1/postgresql-42.5.1.jar +``` +If you don't install maven or have problem in resolving the postgres driver, you can always download this jar yourself, +change the `POSTGRES_LOCAL_PATH` to the path of the downloaded jar. +Then, ```shell - cd packaging/src/docker - docker compose up -d - ``` +docker compose up -d +``` +HiveServer2, Metastore and Postgres services will be started as a consequence. Volumes are used to persist data generated by Hive inside Postgres and HiveServer2 containers, - hive_db @@ -123,6 +136,11 @@ Volumes are used to persist data generated by Hive inside Postgres and HiveServe The volume stores tables' files inside HiveServer2 container. +To stop/remove them all, +```shell +docker compose down +``` + #### Usage - HiveServer2 web diff --git a/packaging/src/docker/docker-compose.yml b/packaging/src/docker/docker-compose.yml index 6f9560b8952c..d0602f08f5c8 100644 --- a/packaging/src/docker/docker-compose.yml +++ b/packaging/src/docker/docker-compose.yml @@ -32,6 +32,9 @@ services: - '9083:9083' volumes: - warehouse:/opt/hive/data/warehouse + - type: bind + source: ${POSTGRES_LOCAL_PATH} + target: /opt/hive/lib/postgres.jar hiveserver2: image: apache/hive:${HIVE_VERSION} From 3ef1c3a0743b9538d09cd9307250150a21fc8537 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Tue, 16 Jan 2024 11:14:48 +0200 Subject: [PATCH 136/179] HIVE-27406: Addendum: Query runtime optimization (Denys Kuzmenko, reviewed by Laszlo Vegh, Sourabh Badhya) Closes #4968 --- ...DuplicateCompleteTxnComponentsCommand.java | 84 ++++++++----------- 1 file changed, 37 insertions(+), 47 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java index d2cd6353fc2f..ca481a05c833 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/commands/RemoveDuplicateCompleteTxnComponentsCommand.java @@ -42,57 +42,47 @@ public String getParameterizedQueryString(DatabaseProduct databaseProduct) throw switch (databaseProduct.dbType) { case MYSQL: case SQLSERVER: - return "DELETE \"tc\" " + - "FROM \"COMPLETED_TXN_COMPONENTS\" \"tc\" " + + return "DELETE tc " + + "FROM \"COMPLETED_TXN_COMPONENTS\" tc " + "INNER JOIN (" + - " SELECT \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", max(\"CTC_WRITEID\") \"highestWriteId\"" + - " FROM \"COMPLETED_TXN_COMPONENTS\"" + - " GROUP BY \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\") \"c\" " + - "ON \"tc\".\"CTC_DATABASE\" = \"c\".\"CTC_DATABASE\" AND \"tc\".\"CTC_TABLE\" = \"c\".\"CTC_TABLE\"" + - " AND (\"tc\".\"CTC_PARTITION\" = \"c\".\"CTC_PARTITION\" OR (\"tc\".\"CTC_PARTITION\" IS NULL AND \"c\".\"CTC_PARTITION\" IS NULL)) " + - "LEFT JOIN (" + - " SELECT \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", max(\"CTC_WRITEID\") \"updateWriteId\"" + - " FROM \"COMPLETED_TXN_COMPONENTS\"" + - " WHERE \"CTC_UPDATE_DELETE\" = 'Y'" + - " GROUP BY \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\") \"c2\" " + - "ON \"tc\".\"CTC_DATABASE\" = \"c2\".\"CTC_DATABASE\" AND \"tc\".\"CTC_TABLE\" = \"c2\".\"CTC_TABLE\"" + - " AND (\"tc\".\"CTC_PARTITION\" = \"c2\".\"CTC_PARTITION\" OR (\"tc\".\"CTC_PARTITION\" IS NULL AND \"c2\".\"CTC_PARTITION\" IS NULL)) " + - "WHERE \"tc\".\"CTC_WRITEID\" < \"c\".\"highestWriteId\" " + + " SELECT \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\"," + + " MAX(\"CTC_WRITEID\") highestWriteId," + + " MAX(CASE WHEN \"CTC_UPDATE_DELETE\" = 'Y' THEN \"CTC_WRITEID\" END) updateWriteId" + + " FROM \"COMPLETED_TXN_COMPONENTS\"" + + " GROUP BY \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\"" + + ") c ON " + + " tc.\"CTC_DATABASE\" = c.\"CTC_DATABASE\" " + + " AND tc.\"CTC_TABLE\" = c.\"CTC_TABLE\"" + + " AND (tc.\"CTC_PARTITION\" = c.\"CTC_PARTITION\" OR (tc.\"CTC_PARTITION\" IS NULL AND c.\"CTC_PARTITION\" IS NULL)) " + + "WHERE tc.\"CTC_WRITEID\" < c.\"highestWriteId\" " + (MYSQL == databaseProduct.dbType ? - " AND NOT \"tc\".\"CTC_WRITEID\" <=> \"c2\".\"updateWriteId\"" : - " AND (\"tc\".\"CTC_WRITEID\" != \"c2\".\"updateWriteId\" OR \"c2\".\"updateWriteId\" IS NULL)"); - case DERBY: + " AND NOT tc.\"CTC_WRITEID\" <=> c.updateWriteId" : + " AND (tc.\"CTC_WRITEID\" != c.updateWriteId OR c.updateWriteId IS NULL)"); + + case DERBY: case ORACLE: - case CUSTOM: - return "DELETE from \"COMPLETED_TXN_COMPONENTS\" \"tc\"" + - "WHERE EXISTS (" + - " SELECT 1" + - " FROM \"COMPLETED_TXN_COMPONENTS\"" + - " WHERE \"CTC_DATABASE\" = \"tc\".\"CTC_DATABASE\"" + - " AND \"CTC_TABLE\" = \"tc\".\"CTC_TABLE\"" + - " AND (\"CTC_PARTITION\" = \"tc\".\"CTC_PARTITION\" OR (\"CTC_PARTITION\" IS NULL AND \"tc\".\"CTC_PARTITION\" IS NULL))" + - " AND (\"tc\".\"CTC_UPDATE_DELETE\"='N' OR \"CTC_UPDATE_DELETE\"='Y')" + - " AND \"tc\".\"CTC_WRITEID\" < \"CTC_WRITEID\")"; case POSTGRES: - return "DELETE " + - "FROM \"COMPLETED_TXN_COMPONENTS\" \"tc\" " + - "USING (" + - " SELECT \"c1\".*, \"c2\".\"updateWriteId\" FROM" + - " (SELECT \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", max(\"CTC_WRITEID\") \"highestWriteId\"" + - " FROM \"COMPLETED_TXN_COMPONENTS\"" + - " GROUP BY \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\") \"c1\"" + - " LEFT JOIN" + - " (SELECT \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", max(\"CTC_WRITEID\") \"updateWriteId\"" + - " FROM \"COMPLETED_TXN_COMPONENTS\"" + - " WHERE \"CTC_UPDATE_DELETE\" = 'Y'" + - " GROUP BY \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\") \"c2\"" + - " ON \"c1\".\"CTC_DATABASE\" = \"c2\".\"CTC_DATABASE\" AND \"c1\".\"CTC_TABLE\" = \"c2\".\"CTC_TABLE\"" + - " AND (\"c1\".\"CTC_PARTITION\" = \"c2\".\"CTC_PARTITION\" OR (\"c1\".\"CTC_PARTITION\" IS NULL AND \"c2\".\"CTC_PARTITION\" IS NULL))" + - ") \"c\" " + - "WHERE \"tc\".\"CTC_DATABASE\" = \"c\".\"CTC_DATABASE\" AND \"tc\".\"CTC_TABLE\" = \"c\".\"CTC_TABLE\"" + - " AND (\"tc\".\"CTC_PARTITION\" = \"c\".\"CTC_PARTITION\" OR (\"tc\".\"CTC_PARTITION\" IS NULL AND \"c\".\"CTC_PARTITION\" IS NULL))" + - " AND \"tc\".\"CTC_WRITEID\" < \"c\".\"highestWriteId\" " + - " AND \"tc\".\"CTC_WRITEID\" IS DISTINCT FROM \"c\".\"updateWriteId\""; + case CUSTOM: + String existsSubQuery = + "EXISTS (" + + " SELECT 1" + + " FROM \"COMPLETED_TXN_COMPONENTS\" c" + + " WHERE tc.\"CTC_DATABASE\" = c.\"CTC_DATABASE\"" + + " AND tc.\"CTC_TABLE\" = c.\"CTC_TABLE\"" + + " AND %s" + + " AND (tc.\"CTC_UPDATE_DELETE\" = 'N' OR c.\"CTC_UPDATE_DELETE\" = 'Y')" + + " AND tc.\"CTC_WRITEID\" < c.\"CTC_WRITEID\"" + + ")"; + + return new StringBuilder() + .append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" tc WHERE ") + .append(String.format(existsSubQuery, + "tc.\"CTC_PARTITION\" = c.\"CTC_PARTITION\"")) + .append(" OR ") + .append(String.format(existsSubQuery, + "tc.\"CTC_PARTITION\" IS NULL AND c.\"CTC_PARTITION\" IS NULL")) + .toString(); + default: String msg = "Unknown database product: " + databaseProduct.dbType; throw new MetaException(msg); From 1760304401ff65d2f1fd7276d8a2bc241babb18a Mon Sep 17 00:00:00 2001 From: wecharyu Date: Mon, 15 Jan 2024 06:24:33 +0000 Subject: [PATCH 137/179] HIVE-27999: Run Sonar analysis using Java 17 (Wechar Yu reviewed by Stamatis Zampetakis, Attila Turoczy) 1. Use executor image built from https://github.com/kgyrtkirk/hive-dev-box/commit/c67d8af69b869bab1ccc7013a8bccec2b2410518 with JDK 17 support 2. Remove unecessary ping Close apache/hive#5004 --- Jenkinsfile | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index 0f99c42cc4aa..fc11bb3299ea 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -112,7 +112,7 @@ def sonarAnalysis(args) { """+args+" -DskipTests -Dit.skipTests -Dmaven.javadoc.skip" sh """#!/bin/bash -e - sw java 11 && . /etc/profile.d/java.sh + sw java 17 && . /etc/profile.d/java.sh export MAVEN_OPTS=-Xmx5G """+mvnCmd } @@ -121,7 +121,7 @@ def sonarAnalysis(args) { def hdbPodTemplate(closure) { podTemplate( containers: [ - containerTemplate(name: 'hdb', image: 'kgyrtkirk/hive-dev-box:executor', ttyEnabled: true, command: 'tini -- cat', + containerTemplate(name: 'hdb', image: 'wecharyu/hive-dev-box:executor', ttyEnabled: true, command: 'tini -- cat', alwaysPullImage: true, resourceRequestCpu: '1800m', resourceLimitCpu: '8000m', @@ -287,7 +287,6 @@ set -x echo 127.0.0.1 dev_$dbType | sudo tee -a /etc/hosts . /etc/profile.d/confs.sh sw hive-dev $PWD -ping -c2 dev_$dbType export DOCKER_NETWORK=host export DBNAME=metastore reinit_metastore $dbType From ce0823896aa6bf3be062764a61ecef1aba309db5 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Wed, 17 Jan 2024 17:04:14 +0800 Subject: [PATCH 138/179] HIVE-21520: Query 'Submit plan' time reported is incorrect (#4996) (Butao Zhang reviewed by Laszlo Bodor) --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java | 2 +- .../hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java index 48e907b20ab7..8935843ae83a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java @@ -258,7 +258,7 @@ public int execute() { LogUtils.putToMDC(LogUtils.DAGID_KEY, dagId); // finally monitor will print progress until the job is done - TezJobMonitor monitor = new TezJobMonitor(work.getAllWork(), dagClient, conf, dag, ctx, counters); + TezJobMonitor monitor = new TezJobMonitor(work.getAllWork(), dagClient, conf, dag, ctx, counters, perfLogger); rc = monitor.monitorExecution(); if (rc != 0) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java index 17ab6e39397a..fbbbf189e48f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java @@ -77,7 +77,7 @@ public class TezJobMonitor { private static final int MAX_RETRY_INTERVAL = 2500; private static final int MAX_RETRY_FAILURES = (MAX_RETRY_INTERVAL / MAX_CHECK_INTERVAL) + 1; - private final PerfLogger perfLogger = SessionState.getPerfLogger(); + private final PerfLogger perfLogger; private static final List shutdownList; private final List topSortedWorks; @@ -117,7 +117,7 @@ public static void initShutdownHook() { private final TezCounters counters; public TezJobMonitor(List topSortedWorks, final DAGClient dagClient, HiveConf conf, DAG dag, - Context ctx, final TezCounters counters) { + Context ctx, final TezCounters counters, PerfLogger perfLogger) { this.topSortedWorks = topSortedWorks; this.dagClient = dagClient; this.hiveConf = conf; @@ -126,6 +126,7 @@ public TezJobMonitor(List topSortedWorks, final DAGClient dagClient, H console = SessionState.getConsole(); updateFunction = updateFunction(); this.counters = counters; + this.perfLogger = perfLogger; } private RenderStrategy.UpdateFunction updateFunction() { From 1097dde68d829ce3863ee667afb9c6332b7f2f0a Mon Sep 17 00:00:00 2001 From: dengzh Date: Thu, 18 Jan 2024 14:12:19 +0800 Subject: [PATCH 139/179] HIVE-28001: Fix the flaky test TestLeaderElection (#5011) (Zhihua Deng, reviewed by Sai Hemanth Gantasala) --- .../hive/metastore/conf/MetastoreConf.java | 5 +- .../leader/LeaderElectionContext.java | 25 +++--- .../leader/LeaderElectionFactory.java | 6 +- .../metastore/leader/LeaseLeaderElection.java | 25 +++--- .../metastore/leader/TestLeaderElection.java | 78 ++++++++++++------- 5 files changed, 92 insertions(+), 47 deletions(-) diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index ffa3d6a06c54..792e27fc99cd 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -580,7 +580,7 @@ public enum ConfVars { "metastore.housekeeping.leader.election", "host", new StringSetValidator("host", "lock"), "Set to host, HMS will choose the leader by the configured metastore.housekeeping.leader.hostname.\n" + - "Set to lock, HMS will use the hive lock to elect the leader."), + "Set to lock, HMS will use the Hive lock to elect the leader."), METASTORE_HOUSEKEEPING_LEADER_AUDITTABLE("metastore.housekeeping.leader.auditTable", "metastore.housekeeping.leader.auditTable", "", "Audit the leader election event to a plain json table when configured."), @@ -593,6 +593,9 @@ public enum ConfVars { "metastore.housekeeping.leader.auditFiles.limit", 10, "Limit the number of small audit files when metastore.housekeeping.leader.newAuditFile is true.\n" + "If the number of audit files exceeds the limit, then the oldest will be deleted."), + METASTORE_HOUSEKEEPING_LEADER_LOCK_NAMESPACE("metastore.housekeeping.leader.lock.namespace", + "metastore.housekeeping.leader.lock.namespace", "", + "The database where the Hive lock sits when metastore.housekeeping.leader.election is set to lock."), METASTORE_HOUSEKEEPING_THREADS_ON("metastore.housekeeping.threads.on", "hive.metastore.housekeeping.threads.on", false, "Whether to run the tasks under metastore.task.threads.remote on this metastore instance or not.\n" + diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionContext.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionContext.java index 3c01d8030ca9..a3652d1c0019 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionContext.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionContext.java @@ -42,12 +42,12 @@ public class LeaderElectionContext { * For those tasks which belong to the same type, they will be running in the same leader. */ public enum TTYPE { - HOUSEKEEPING(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "sys", - "metastore_housekeeping_leader"), "housekeeping"), - WORKER(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "sys", - "metastore_worker_leader"), "compactor_worker"), - ALWAYS_TASKS(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "sys", - "metastore_always_tasks_leader"), "always_tasks"); + HOUSEKEEPING(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "__METASTORE_LEADER_ELECTION__", + "metastore_housekeeping"), "housekeeping"), + WORKER(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "__METASTORE_LEADER_ELECTION__", + "metastore_compactor_worker"), "compactor_worker"), + ALWAYS_TASKS(new TableName(Warehouse.DEFAULT_CATALOG_NAME, "__METASTORE_LEADER_ELECTION__", + "metastore_always_tasks"), "always_tasks"); // Mutex of TTYPE, which can be a nonexistent table private final TableName mutex; // Name of TTYPE @@ -127,9 +127,10 @@ public void start() throws Exception { throw new RuntimeException("Error claiming to be leader: " + leaderElection.getName(), e); } }); + daemon.setName("Metastore Election " + leaderElection.getName()); + daemon.setDaemon(true); + if (startAsDaemon) { - daemon.setName("Leader-Election-" + leaderElection.getName()); - daemon.setDaemon(true); daemon.start(); } else { daemon.run(); @@ -154,7 +155,13 @@ public static Object getLeaderMutex(Configuration conf, TTYPE ttype, String serv case "host": return servHost; case "lock": - return ttype.getTableName(); + TableName mutex = ttype.getTableName(); + String namespace = + MetastoreConf.getVar(conf, MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_LEADER_LOCK_NAMESPACE); + if (StringUtils.isNotEmpty(namespace)) { + return new TableName(mutex.getCat(), namespace, mutex.getTable()); + } + return mutex; default: throw new UnsupportedOperationException(method + " not supported for leader election"); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionFactory.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionFactory.java index 51dce28ac50c..5055ad8a0034 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionFactory.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaderElectionFactory.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.metastore.leader; +import java.io.IOException; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; @@ -26,7 +28,7 @@ */ public class LeaderElectionFactory { - public static LeaderElection create(Configuration conf) { + public static LeaderElection create(Configuration conf) throws IOException { String method = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_LEADER_ELECTION); switch (method.toLowerCase()) { @@ -35,7 +37,7 @@ public static LeaderElection create(Configuration conf) { case "lock": return new LeaseLeaderElection(); default: - throw new UnsupportedOperationException("Do not support " + method + " now"); + throw new UnsupportedOperationException(method + " is not supported for electing the leader"); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaseLeaderElection.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaseLeaderElection.java index 937174f5d92b..d6ad76dcce9b 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaseLeaderElection.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/LeaseLeaderElection.java @@ -42,6 +42,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.net.InetAddress; import java.util.ArrayList; import java.util.List; @@ -92,9 +93,17 @@ public class LeaseLeaderElection implements LeaderElection { public static final String METASTORE_RENEW_LEASE = "metastore.renew.leader.lease"; private String name; + private String userName; + private String hostName; - private void doWork(LockResponse resp, Configuration conf, + public LeaseLeaderElection() throws IOException { + userName = SecurityUtils.getUser(); + hostName = InetAddress.getLocalHost().getHostName(); + } + + private synchronized void doWork(LockResponse resp, Configuration conf, TableName tableName) throws LeaderException { + long start = System.currentTimeMillis(); lockId = resp.getLockid(); assert resp.getState() == LockState.ACQUIRED || resp.getState() == LockState.WAITING; shutdownWatcher(); @@ -121,6 +130,7 @@ private void doWork(LockResponse resp, Configuration conf, default: throw new IllegalStateException("Unexpected lock state: " + resp.getState()); } + LOG.debug("Spent {}ms to notify the listeners, isLeader: {}", System.currentTimeMillis() - start, isLeader); } private void notifyListener() { @@ -142,13 +152,6 @@ private void notifyListener() { public void tryBeLeader(Configuration conf, TableName table) throws LeaderException { requireNonNull(conf, "conf is null"); requireNonNull(table, "table is null"); - String user, hostName; - try { - user = SecurityUtils.getUser(); - hostName = InetAddress.getLocalHost().getHostName(); - } catch (Exception e) { - throw new LeaderException("Error while getting the username", e); - } if (store == null) { store = TxnUtils.getTxnStore(conf); @@ -165,7 +168,7 @@ public void tryBeLeader(Configuration conf, TableName table) throws LeaderExcept boolean lockable = false; Exception recentException = null; long start = System.currentTimeMillis(); - LockRequest req = new LockRequest(components, user, hostName); + LockRequest req = new LockRequest(components, userName, hostName); int numRetries = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.LOCK_NUMRETRIES); long maxSleep = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.LOCK_SLEEP_BETWEEN_RETRIES, TimeUnit.MILLISECONDS); @@ -175,6 +178,7 @@ public void tryBeLeader(Configuration conf, TableName table) throws LeaderExcept if (res.getState() == LockState.WAITING || res.getState() == LockState.ACQUIRED) { lockable = true; doWork(res, conf, table); + LOG.debug("Spent {}ms to lock the table {}, retries: {}", System.currentTimeMillis() - start, table, i); break; } } catch (NoSuchTxnException | TxnAbortedException e) { @@ -324,6 +328,7 @@ public void runInternal() { } catch (NoSuchTxnException | TxnAbortedException e) { throw new AssertionError("This should not happen, we didn't open txn", e); } catch (NoSuchLockException e) { + LOG.info("No such lock {} for NonLeaderWatcher, try to obtain the lock again...", lockId); reclaim(); } catch (Exception e) { // Wait for next cycle. @@ -379,6 +384,7 @@ public void runInternal() { } catch (NoSuchTxnException | TxnAbortedException e) { throw new AssertionError("This should not happen, we didn't open txn", e); } catch (NoSuchLockException e) { + LOG.info("No such lock {} for Heartbeater, try to obtain the lock again...", lockId); reclaim(); } catch (Exception e) { // Wait for next cycle. @@ -404,6 +410,7 @@ public ReleaseAndRequireWatcher(Configuration conf, super(conf, tableName); timeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS) + 3000; + setName("ReleaseAndRequireWatcher"); } @Override diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/leader/TestLeaderElection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/leader/TestLeaderElection.java index 99d13c00b4a7..59f7dbc8fd64 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/leader/TestLeaderElection.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/leader/TestLeaderElection.java @@ -27,6 +27,9 @@ import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.junit.Test; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -52,6 +55,28 @@ public void testConfigLeaderElection() throws Exception { assertFalse(election.isLeader()); } + static class TestLeaderListener implements LeaderElection.LeadershipStateListener { + AtomicBoolean flag; + TestLeaderListener(AtomicBoolean flag) { + this.flag = flag; + } + @Override + public void takeLeadership(LeaderElection election) throws Exception { + synchronized (flag) { + flag.set(true); + flag.notifyAll(); + } + } + + @Override + public void lossLeadership(LeaderElection election) throws Exception { + synchronized (flag) { + flag.set(false); + flag.notifyAll(); + } + } + } + @Test public void testLeaseLeaderElection() throws Exception { Configuration configuration = MetastoreConf.newMetastoreConf(); @@ -68,16 +93,7 @@ public void testLeaseLeaderElection() throws Exception { TableName mutex = new TableName("hive", "default", "leader_lease_ms"); LeaseLeaderElection instance1 = new LeaseLeaderElection(); AtomicBoolean flag1 = new AtomicBoolean(false); - instance1.addStateListener(new LeaderElection.LeadershipStateListener() { - @Override - public void takeLeadership(LeaderElection election) { - flag1.set(true); - } - @Override - public void lossLeadership(LeaderElection election) { - flag1.set(false); - } - }); + instance1.addStateListener(new TestLeaderListener(flag1)); instance1.tryBeLeader(configuration, mutex); // elect1 as a leader now assertTrue(flag1.get() && instance1.isLeader()); @@ -85,31 +101,22 @@ public void lossLeadership(LeaderElection election) { configuration.setBoolean(LeaseLeaderElection.METASTORE_RENEW_LEASE, true); LeaseLeaderElection instance2 = new LeaseLeaderElection(); AtomicBoolean flag2 = new AtomicBoolean(false); - instance2.addStateListener(new LeaderElection.LeadershipStateListener() { - @Override - public void takeLeadership(LeaderElection election) { - flag2.set(true); - } - @Override - public void lossLeadership(LeaderElection election) { - flag2.set(false); - } - }); + instance2.addStateListener(new TestLeaderListener(flag2)); instance2.tryBeLeader(configuration, mutex); - // instance2 should not be leader as elect1 holds the lease assertFalse(flag2.get() || instance2.isLeader()); - Thread.sleep(15 * 1000); + + ExecutorService service = Executors.newFixedThreadPool(4); + wait(service, flag1, flag2); // now instance1 lease is timeout, the instance2 should be leader now assertTrue(instance2.isLeader() && flag2.get()); - assertFalse(flag1.get() || instance1.isLeader()); assertTrue(flag2.get() && instance2.isLeader()); + // remove leader's lease (instance2) long lockId2 = instance2.getLockId(); txnStore.unlock(new UnlockRequest(lockId2)); - Thread.sleep(4 * 1000); - assertTrue(flag1.get() && instance1.isLeader()); + wait(service, flag1, flag2); assertFalse(flag2.get() || instance2.isLeader()); assertTrue(lockId2 > 0); assertFalse(instance2.getLockId() == lockId2); @@ -117,7 +124,7 @@ public void lossLeadership(LeaderElection election) { // remove leader's lease(instance1) long lockId1 = instance1.getLockId(); txnStore.unlock(new UnlockRequest(lockId1)); - Thread.sleep(4 * 1000); + wait(service, flag1, flag2); assertFalse(lockId1 == instance1.getLockId()); assertTrue(lockId1 > 0); @@ -128,4 +135,23 @@ public void lossLeadership(LeaderElection election) { } } + private void wait(ExecutorService service, Object... obj) throws Exception { + Future[] fs = new Future[obj.length]; + for (int i = 0; i < obj.length; i++) { + Object monitor = obj[i]; + fs[i] = service.submit(() -> { + try { + synchronized (monitor) { + monitor.wait(); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + } + for (Future f : fs) { + f.get(); + } + } + } From 9c4eb96f816105560e7d4809f1d608e7eca9e523 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Thu, 18 Jan 2024 13:38:06 +0530 Subject: [PATCH 140/179] HIVE-27749: Addendum: SchemaTool initSchema fails on Mariadb 10.2 - Fix INSERT query (#5009) (Sourabh Badhya reviewed by Attila Turoczy, Denys Kuzmenko) --- .../src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql | 2 +- .../src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql | 2 +- .../src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql | 4 ++-- .../tools/schematool/TestSchemaToolForMetastore.java | 3 ++- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql index 3016f8021f61..33085004cf81 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-alpha-2.mysql.sql @@ -894,7 +894,7 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE` ALTER TABLE `NOTIFICATION_SEQUENCE` ADD CONSTRAINT `ONE_ROW_CONSTRAINT` CHECK (`NNI_ID` = 1); -INSERT INTO `NOTIFICATION_SEQUENCE` (`NEXT_EVENT_ID`) SELECT * from (select 1 as `NOTIFICATION_SEQUENCE`) a WHERE (SELECT COUNT(*) FROM `NOTIFICATION_SEQUENCE`) = 0; +INSERT INTO `NOTIFICATION_SEQUENCE` (`NNI_ID`, `NEXT_EVENT_ID`) SELECT 1,1 FROM DUAL WHERE NOT EXISTS (SELECT 1 FROM `NOTIFICATION_SEQUENCE`); CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS` ( diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql index e9d37d357748..a99b9c5afb4f 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-1.mysql.sql @@ -896,7 +896,7 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE` ALTER TABLE `NOTIFICATION_SEQUENCE` ADD CONSTRAINT `ONE_ROW_CONSTRAINT` CHECK (`NNI_ID` = 1); -INSERT INTO `NOTIFICATION_SEQUENCE` (`NEXT_EVENT_ID`) SELECT * from (select 1 as `NOTIFICATION_SEQUENCE`) a WHERE (SELECT COUNT(*) FROM `NOTIFICATION_SEQUENCE`) = 0; +INSERT INTO `NOTIFICATION_SEQUENCE` (`NNI_ID`, `NEXT_EVENT_ID`) SELECT 1,1 FROM DUAL WHERE NOT EXISTS (SELECT 1 FROM `NOTIFICATION_SEQUENCE`); CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS` ( diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql index d97577f7d211..c1140eacf444 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql @@ -851,9 +851,9 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE` PRIMARY KEY (`NNI_ID`) ) ENGINE=InnoDB DEFAULT CHARSET=latin1; -ALTER TABLE `NOTIFICATION_SEQUENCE` MODIFY COLUMN `NNI_ID` BIGINT(20) GENERATED ALWAYS AS (1) STORED NOT NULL; +ALTER TABLE `NOTIFICATION_SEQUENCE` ADD CONSTRAINT `ONE_ROW_CONSTRAINT` CHECK (`NNI_ID` = 1); -INSERT INTO `NOTIFICATION_SEQUENCE` (`NEXT_EVENT_ID`) SELECT * from (select 1 as `NOTIFICATION_SEQUENCE`) a WHERE (SELECT COUNT(*) FROM `NOTIFICATION_SEQUENCE`) = 0; +INSERT INTO `NOTIFICATION_SEQUENCE` (`NNI_ID`, `NEXT_EVENT_ID`) SELECT 1,1 FROM DUAL WHERE NOT EXISTS (SELECT 1 FROM `NOTIFICATION_SEQUENCE`); CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS` ( diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java index e82b943915bf..804bf156dc62 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/tools/schematool/TestSchemaToolForMetastore.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.dbinstall.rules.DatabaseRule; import org.apache.hadoop.hive.metastore.dbinstall.rules.Derby; +import org.apache.hadoop.hive.metastore.dbinstall.rules.Mariadb; import org.apache.hadoop.hive.metastore.dbinstall.rules.Mssql; import org.apache.hadoop.hive.metastore.dbinstall.rules.Mysql; import org.apache.hadoop.hive.metastore.dbinstall.rules.Oracle; @@ -85,7 +86,7 @@ public static Collection databases() { dbs.add(new Object[] { new Mysql() }); dbs.add(new Object[] { new Oracle() }); dbs.add(new Object[] { new Postgres() }); -// dbs.add(new Object[] { new Mariadb() }); Disabled due to HIVE-27749 + dbs.add(new Object[] { new Mariadb() }); dbs.add(new Object[] { new Mssql() }); return dbs; } From d06fb43b617b3c52de39cf7a078e44108451b10c Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Sat, 20 Jan 2024 01:46:10 +0800 Subject: [PATCH 141/179] HIVE-27827: Improve performance of direct SQL implement for getPartitionsByFilter (#4831) (Wechar Yu, Reviewed by Sai Hemanth Gantasala) --- .../hive/metastore/MetaStoreDirectSql.java | 46 +++++++++++++++++-- .../hive/metastore/parser/ExpressionTree.java | 12 ++++- .../derby/hive-schema-4.0.0-beta-2.derby.sql | 2 +- ...ade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql | 4 ++ .../mssql/hive-schema-4.0.0-beta-2.mssql.sql | 4 +- ...ade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql | 5 ++ .../mysql/hive-schema-4.0.0-beta-2.mysql.sql | 3 +- ...ade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql | 5 ++ .../hive-schema-4.0.0-beta-2.oracle.sql | 4 +- ...de-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql | 5 ++ .../hive-schema-4.0.0-beta-2.postgres.sql | 9 +--- ...-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql | 5 ++ .../hive/metastore/tools/BenchmarkTool.java | 5 ++ .../hive/metastore/tools/HMSBenchmarks.java | 21 +++++++++ .../hive/metastore/tools/HMSClient.java | 5 ++ 15 files changed, 111 insertions(+), 24 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java index 18bffce36f7d..515721791bb7 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java @@ -30,6 +30,7 @@ import static org.apache.hadoop.hive.metastore.ColumnType.TIMESTAMP_TYPE_NAME; import static org.apache.hadoop.hive.metastore.ColumnType.TINYINT_TYPE_NAME; import static org.apache.hadoop.hive.metastore.ColumnType.VARCHAR_TYPE_NAME; +import static org.apache.hadoop.hive.metastore.utils.FileUtils.unescapePathName; import java.sql.Connection; import java.sql.Date; @@ -1409,7 +1410,8 @@ public void visit(LeafNode node) throws MetaException { return; } - String colTypeStr = ColumnType.getTypeName(partitionKeys.get(partColIndex).getType()); + FieldSchema partCol = partitionKeys.get(partColIndex); + String colTypeStr = ColumnType.getTypeName(partCol.getType()); FilterType colType = FilterType.fromType(colTypeStr); if (colType == FilterType.Invalid) { filterBuffer.setError("Filter pushdown not supported for type " + colTypeStr); @@ -1434,7 +1436,8 @@ public void visit(LeafNode node) throws MetaException { } if (colType == FilterType.Timestamp && valType == FilterType.String) { - nodeValue = MetaStoreUtils.convertStringToTimestamp((String)nodeValue); + // timestamp value may be escaped in client side, so we need unescape it here. + nodeValue = MetaStoreUtils.convertStringToTimestamp(unescapePathName((String) nodeValue)); valType = FilterType.Timestamp; } @@ -1520,10 +1523,43 @@ public void visit(LeafNode node) throws MetaException { if (node.operator == Operator.LIKE) { nodeValue0 = nodeValue0 + " ESCAPE '\\' "; } + String filter = node.isReverseOrder + ? nodeValue0 + " " + node.operator.getSqlOp() + " " + tableValue + : tableValue + " " + node.operator.getSqlOp() + " " + nodeValue0; + // For equals and not-equals filter, we can add partition name filter to improve performance. + boolean isOpEquals = Operator.isEqualOperator(node.operator); + if (isOpEquals || Operator.isNotEqualOperator(node.operator)) { + Map partKeyToVal = new HashMap<>(); + partKeyToVal.put(partCol.getName(), nodeValue.toString()); + String escapedNameFragment = Warehouse.makePartName(partKeyToVal, false); + if (colType == FilterType.Date) { + // Some engines like Pig will record both date and time values, in which case we need + // match PART_NAME by like clause. + escapedNameFragment += "%"; + } + if (colType != FilterType.Date && partColCount == 1) { + // Case where partition column type is not date and there is no other partition columns + params.add(escapedNameFragment); + filter += " and " + PARTITIONS + ".\"PART_NAME\"" + (isOpEquals ? " =? " : " !=? "); + } else { + if (partColCount == 1) { + // Case where partition column type is date and there is no other partition columns + params.add(escapedNameFragment); + } else if (partColIndex + 1 == partColCount) { + // Case where the partition column is at the end of the name. + params.add("%/" + escapedNameFragment); + } else if (partColIndex == 0) { + // Case where the partition column is at the beginning of the name. + params.add(escapedNameFragment + "/%"); + } else { + // Case where the partition column is in the middle of the name. + params.add("%/" + escapedNameFragment + "/%"); + } + filter += " and " + PARTITIONS + ".\"PART_NAME\"" + (isOpEquals ? " like ? " : " not like ? "); + } + } - filterBuffer.append(node.isReverseOrder - ? "(" + nodeValue0 + " " + node.operator.getSqlOp() + " " + tableValue + ")" - : "(" + tableValue + " " + node.operator.getSqlOp() + " " + nodeValue0 + ")"); + filterBuffer.append("(" + filter + ")"); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java index f2f91cbedfb7..2e325a4e431d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/parser/ExpressionTree.java @@ -101,6 +101,14 @@ public static Operator fromString(String inputOperator) { " for " + Operator.class.getSimpleName()); } + public static boolean isEqualOperator(Operator op) { + return op == EQUALS; + } + + public static boolean isNotEqualOperator(Operator op) { + return op == NOTEQUALS || op == NOTEQUALS2; + } + @Override public String toString() { return op; @@ -376,8 +384,8 @@ private void generateJDOFilterOverPartitions(Configuration conf, params.put(paramName, valueAsString); } - boolean isOpEquals = operator == Operator.EQUALS; - if (isOpEquals || operator == Operator.NOTEQUALS || operator == Operator.NOTEQUALS2) { + boolean isOpEquals = Operator.isEqualOperator(operator); + if (isOpEquals || Operator.isNotEqualOperator(operator)) { String partitionKey = partitionKeys.get(partitionColumnIndex).getName(); makeFilterForEquals(partitionKey, valueAsString, paramName, params, partitionColumnIndex, partitionColumnCount, isOpEquals, filterBuilder); diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql index 9feaa6a0c3ff..3b9d46700e40 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0-beta-2.derby.sql @@ -288,7 +288,7 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_TYPE" ON "APP"."TYPES" ("TYPE_NAME"); CREATE INDEX "APP"."PARTITIONCOLUMNPRIVILEGEINDEX" ON "APP"."PART_COL_PRIVS" ("AUTHORIZER", "PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); -CREATE UNIQUE INDEX "APP"."UNIQUEPARTITION" ON "APP"."PARTITIONS" ("PART_NAME", "TBL_ID"); +CREATE UNIQUE INDEX "APP"."UNIQUEPARTITION" ON "APP"."PARTITIONS" ("TBL_ID", "PART_NAME"); CREATE UNIQUE INDEX "APP"."UNIQUEFUNCTION" ON "APP"."FUNCS" ("FUNC_NAME", "DB_ID"); diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql index 2695e10742c8..525a7d5215ed 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.derby.sql @@ -2,5 +2,9 @@ DROP TABLE "APP"."INDEX_PARAMS"; DROP TABLE "APP"."IDXS"; +-- HIVE-27827 +DROP INDEX "APP"."UNIQUEPARTITION"; +CREATE UNIQUE INDEX "APP"."UNIQUEPARTITION" ON "APP"."PARTITIONS" ("TBL_ID", "PART_NAME"); + -- This needs to be the last thing done. Insert any changes above this line. UPDATE "APP".VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql index 190a245c2b66..2bbf62d5f1c1 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0-beta-2.mssql.sql @@ -747,9 +747,7 @@ ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK2 FOREIGN KEY (SD_ID) REFEREN CREATE INDEX PARTITIONS_N49 ON PARTITIONS (SD_ID); -CREATE INDEX PARTITIONS_N50 ON PARTITIONS (TBL_ID); - -CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (PART_NAME,TBL_ID); +CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (TBL_ID,PART_NAME); -- Constraints for table CDS for class(es) [org.apache.hadoop.hive.metastore.model.MColumnDescriptor] diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql index a7afd05ceb3a..f7ded61aef90 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mssql.sql @@ -4,6 +4,11 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAG DROP TABLE INDEX_PARAMS; DROP TABLE IDXS; +-- HIVE-27827 +DROP INDEX UNIQUEPARTITION ON PARTITIONS; +CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (TBL_ID,PART_NAME); +DROP INDEX PARTITIONS_N50 ON PARTITIONS; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql index c1140eacf444..f99632267a8c 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0-beta-2.mysql.sql @@ -191,8 +191,7 @@ CREATE TABLE IF NOT EXISTS `PARTITIONS` ( `TBL_ID` bigint(20) DEFAULT NULL, `WRITE_ID` bigint(20) DEFAULT 0, PRIMARY KEY (`PART_ID`), - UNIQUE KEY `UNIQUEPARTITION` (`PART_NAME`,`TBL_ID`), - KEY `PARTITIONS_N49` (`TBL_ID`), + UNIQUE KEY `UNIQUEPARTITION` (`TBL_ID`, `PART_NAME`), KEY `PARTITIONS_N50` (`SD_ID`), CONSTRAINT `PARTITIONS_FK1` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`), CONSTRAINT `PARTITIONS_FK2` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` (`SD_ID`) diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql index 9d3bcb028abe..a57b58b2f7b8 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.mysql.sql @@ -4,6 +4,11 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE DROP TABLE `INDEX_PARAMS`; DROP TABLE `IDXS`; +-- HIVE-27827 +DROP INDEX UNIQUEPARTITION ON PARTITIONS; +CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (TBL_ID, PART_NAME); +DROP INDEX PARTITIONS_N49 on PARTITIONS; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql index a2fb02c89111..d4026910343b 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0-beta-2.oracle.sql @@ -763,9 +763,7 @@ ALTER TABLE PARTITIONS ADD CONSTRAINT PARTITIONS_FK2 FOREIGN KEY (SD_ID) REFEREN CREATE INDEX PARTITIONS_N49 ON PARTITIONS (SD_ID); -CREATE INDEX PARTITIONS_N50 ON PARTITIONS (TBL_ID); - -CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (PART_NAME,TBL_ID); +CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (TBL_ID, PART_NAME); -- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege] ALTER TABLE TBL_COL_PRIVS ADD CONSTRAINT TBL_COL_PRIVS_FK1 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) INITIALLY DEFERRED ; diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql index 53c7cb140d77..ab163b6cd00a 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.oracle.sql @@ -5,6 +5,11 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS Status DROP TABLE INDEX_PARAMS; DROP TABLE IDXS; +-- HIVE-27827 +DROP INDEX UNIQUEPARTITION; +CREATE UNIQUE INDEX UNIQUEPARTITION ON PARTITIONS (TBL_ID, PART_NAME); +DROP INDEX PARTITIONS_N50; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0-beta-2', VERSION_COMMENT='Hive release version 4.0.0-beta-2' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2' AS Status from dual; diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql index b247bc3fda09..292bc200fca9 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0-beta-2.postgres.sql @@ -1027,7 +1027,7 @@ ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_pkey" PRIMARY K -- ALTER TABLE ONLY "PARTITIONS" - ADD CONSTRAINT "UNIQUEPARTITION" UNIQUE ("PART_NAME", "TBL_ID"); + ADD CONSTRAINT "UNIQUEPARTITION" UNIQUE ("TBL_ID", "PART_NAME"); -- @@ -1135,13 +1135,6 @@ CREATE INDEX "PARTITIONCOLUMNPRIVILEGEINDEX" ON "PART_COL_PRIVS" USING btree ("A CREATE INDEX "PARTITIONEVENTINDEX" ON "PARTITION_EVENTS" USING btree ("PARTITION_NAME"); --- --- Name: PARTITIONS_N49; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: --- - -CREATE INDEX "PARTITIONS_N49" ON "PARTITIONS" USING btree ("TBL_ID"); - - -- -- Name: PARTITIONS_N50; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace: -- diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql index 1d285bbe842f..af17c7d95244 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-beta-1-to-4.0.0-beta-2.postgres.sql @@ -4,6 +4,11 @@ SELECT 'Upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2'; DROP TABLE IF EXISTS "INDEX_PARAMS"; DROP TABLE IF EXISTS "IDXS"; +-- HIVE-27827 +ALTER TABLE ONLY "PARTITIONS" DROP CONSTRAINT "UNIQUEPARTITION"; +ALTER TABLE ONLY "PARTITIONS" ADD CONSTRAINT "UNIQUEPARTITION" UNIQUE ("TBL_ID", "PART_NAME"); +DROP INDEX "PARTITIONS_N49"; + -- These lines need to be last. Insert any changes above. UPDATE "VERSION" SET "SCHEMA_VERSION"='4.0.0-beta-2', "VERSION_COMMENT"='Hive release version 4.0.0-beta-2' where "VER_ID"=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-beta-1 to 4.0.0-beta-2'; diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java index f19576d69405..93556e7f0fa1 100644 --- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java +++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/BenchmarkTool.java @@ -57,6 +57,7 @@ import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkGetNotificationId; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkGetPartitionNames; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkGetPartitions; +import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkGetPartitionsByFilter; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkGetPartitionsByName; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkGetTable; import static org.apache.hadoop.hive.metastore.tools.HMSBenchmarks.benchmarkListAllTables; @@ -284,6 +285,8 @@ private void runNonAcidBenchmarks() { () -> benchmarkGetPartitionNames(bench, bData, 1)) .add("getPartitionsByNames", () -> benchmarkGetPartitionsByName(bench, bData, 1)) + .add("getPartitionsByFilter", + () -> benchmarkGetPartitionsByFilter(bench, bData, 1)) .add("renameTable", () -> benchmarkRenameTable(bench, bData, 1)) .add("dropDatabase", @@ -308,6 +311,8 @@ private void runNonAcidBenchmarks() { () -> benchmarkGetPartitionNames(bench, bData, howMany)) .add("getPartitionsByNames" + '.' + howMany, () -> benchmarkGetPartitionsByName(bench, bData, howMany)) + .add("getPartitionsByFilter" + '.' + howMany, + () -> benchmarkGetPartitionsByFilter(bench, bData, howMany)) .add("addPartitions" + '.' + howMany, () -> benchmarkCreatePartitions(bench, bData, howMany)) .add("dropPartitions" + '.' + howMany, diff --git a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java index a2f97eb31704..fdab07178354 100644 --- a/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java +++ b/standalone-metastore/metastore-tools/metastore-benchmarks/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSBenchmarks.java @@ -410,6 +410,27 @@ static DescriptiveStatistics benchmarkGetPartitionsByName(@NotNull MicroBenchmar } } + static DescriptiveStatistics benchmarkGetPartitionsByFilter(@NotNull MicroBenchmark bench, + @NotNull BenchData data, + int count) { + final HMSClient client = data.getClient(); + String dbName = data.dbName; + String tableName = data.tableName; + + BenchmarkUtils.createPartitionedTable(client, dbName, tableName); + try { + addManyPartitionsNoException(client, dbName, tableName, null, + Collections.singletonList("d"), count); + return bench.measure( + () -> + throwingSupplierWrapper(() -> + client.getPartitionsByFilter(dbName, tableName, "`date`='d0'")) + ); + } finally { + throwingSupplierWrapper(() -> client.dropTable(dbName, tableName)); + } + } + static DescriptiveStatistics benchmarkRenameTable(@NotNull MicroBenchmark bench, @NotNull BenchData data, int count) { diff --git a/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java b/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java index 6b7e2a450b18..61580aa7b225 100644 --- a/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java +++ b/standalone-metastore/metastore-tools/tools-common/src/main/java/org/apache/hadoop/hive/metastore/tools/HMSClient.java @@ -329,6 +329,11 @@ List getPartitionsByNames(@NotNull String dbName, @NotNull String tab return client.get_partitions_by_names(dbName, tableName, names); } + List getPartitionsByFilter(@NotNull String dbName, @NotNull String tableName, + @NotNull String filter) throws TException { + return client.get_partitions_by_filter(dbName, tableName, filter, (short) -1); + } + boolean alterTable(@NotNull String dbName, @NotNull String tableName, @NotNull Table newTable) throws TException { client.alter_table(dbName, tableName, newTable); From 72fd26d207a2943f0535fa96330bedc244fbe10a Mon Sep 17 00:00:00 2001 From: dengzh Date: Sat, 20 Jan 2024 10:02:24 +0800 Subject: [PATCH 142/179] HIVE-27994: Optimize renaming the partitioned table (#4995) (Zhihua Deng, reviewed by Butao Zhang, Sai Hemanth Gantasala) --- .../queries/clientpositive/rename_table.q | 40 ++ .../clientpositive/llap/rename_table.q.out | 378 ++++++++++++++++++ .../hive/metastore/DirectSqlUpdatePart.java | 75 ++-- .../hive/metastore/HiveAlterHandler.java | 76 ++-- .../hive/metastore/StatObjectConverter.java | 9 +- 5 files changed, 519 insertions(+), 59 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/rename_table.q create mode 100644 ql/src/test/results/clientpositive/llap/rename_table.q.out diff --git a/ql/src/test/queries/clientpositive/rename_table.q b/ql/src/test/queries/clientpositive/rename_table.q new file mode 100644 index 000000000000..1e4f2fcdd550 --- /dev/null +++ b/ql/src/test/queries/clientpositive/rename_table.q @@ -0,0 +1,40 @@ +--! qt:dataset:src +--! qt:dataset:part +-- This test verifies that if the table after rename can still fetch the column statistics +set hive.stats.kll.enable=true; +set metastore.stats.fetch.bitvector=true; +set metastore.stats.fetch.kll=true; +set hive.stats.autogather=true; +set hive.stats.column.autogather=true; + +CREATE TABLE rename_partition_table0 (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS ORC; + +INSERT OVERWRITE TABLE rename_partition_table0 PARTITION (part = '1') SELECT * FROM src where rand(1) < 0.5; +ALTER TABLE rename_partition_table0 ADD COLUMNS (new_col INT); +INSERT OVERWRITE TABLE rename_partition_table0 PARTITION (part = '2') SELECT src.*, 1 FROM src; + +ALTER TABLE rename_partition_table0 RENAME TO rename_partition_table1; +DESCRIBE FORMATTED rename_partition_table1; +DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='1') key; +DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='1') value; +DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') key; +DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') value; +DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') new_col; + +CREATE EXTERNAL TABLE rename_partition_table_ext0 (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS ORC; + +INSERT OVERWRITE TABLE rename_partition_table_ext0 PARTITION (part = '1') SELECT * FROM src where rand(1) < 0.5; +ALTER TABLE rename_partition_table_ext0 CHANGE COLUMN value val STRING CASCADE; +INSERT OVERWRITE TABLE rename_partition_table_ext0 PARTITION (part = '2') SELECT * FROM src; + +ALTER TABLE rename_partition_table_ext0 RENAME TO rename_partition_table_ext1; +DESCRIBE FORMATTED rename_partition_table_ext1; +DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='1') key; +DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='1') val; +DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='2') key; +DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='2') val; + +DROP TABLE rename_partition_table1; +DROP TABLE rename_partition_table_ext1; diff --git a/ql/src/test/results/clientpositive/llap/rename_table.q.out b/ql/src/test/results/clientpositive/llap/rename_table.q.out new file mode 100644 index 000000000000..014d629ea868 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/rename_table.q.out @@ -0,0 +1,378 @@ +PREHOOK: query: CREATE TABLE rename_partition_table0 (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS ORC +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@rename_partition_table0 +POSTHOOK: query: CREATE TABLE rename_partition_table0 (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS ORC +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@rename_partition_table0 +PREHOOK: query: INSERT OVERWRITE TABLE rename_partition_table0 PARTITION (part = '1') SELECT * FROM src where rand(1) < 0.5 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@rename_partition_table0@part=1 +POSTHOOK: query: INSERT OVERWRITE TABLE rename_partition_table0 PARTITION (part = '1') SELECT * FROM src where rand(1) < 0.5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@rename_partition_table0@part=1 +POSTHOOK: Lineage: rename_partition_table0 PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: rename_partition_table0 PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: ALTER TABLE rename_partition_table0 ADD COLUMNS (new_col INT) +PREHOOK: type: ALTERTABLE_ADDCOLS +PREHOOK: Input: default@rename_partition_table0 +PREHOOK: Output: default@rename_partition_table0 +POSTHOOK: query: ALTER TABLE rename_partition_table0 ADD COLUMNS (new_col INT) +POSTHOOK: type: ALTERTABLE_ADDCOLS +POSTHOOK: Input: default@rename_partition_table0 +POSTHOOK: Output: default@rename_partition_table0 +PREHOOK: query: INSERT OVERWRITE TABLE rename_partition_table0 PARTITION (part = '2') SELECT src.*, 1 FROM src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@rename_partition_table0@part=2 +POSTHOOK: query: INSERT OVERWRITE TABLE rename_partition_table0 PARTITION (part = '2') SELECT src.*, 1 FROM src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@rename_partition_table0@part=2 +POSTHOOK: Lineage: rename_partition_table0 PARTITION(part=2).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: rename_partition_table0 PARTITION(part=2).new_col SIMPLE [] +POSTHOOK: Lineage: rename_partition_table0 PARTITION(part=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: ALTER TABLE rename_partition_table0 RENAME TO rename_partition_table1 +PREHOOK: type: ALTERTABLE_RENAME +PREHOOK: Input: default@rename_partition_table0 +PREHOOK: Output: database:default +PREHOOK: Output: default@rename_partition_table0 +PREHOOK: Output: default@rename_partition_table1 +POSTHOOK: query: ALTER TABLE rename_partition_table0 RENAME TO rename_partition_table1 +POSTHOOK: type: ALTERTABLE_RENAME +POSTHOOK: Input: default@rename_partition_table0 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@rename_partition_table0 +POSTHOOK: Output: default@rename_partition_table1 +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table1 +# col_name data_type comment +key string +value string +new_col int + +# Partition Information +# col_name data_type comment +part string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + bucketing_version 2 +#### A masked pattern was here #### + numFiles 2 + numPartitions 2 + numRows 746 + rawDataSize 133296 + totalSize 4760 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='1') key +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='1') key +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table1 +col_name key +data_type string +min +max +num_nulls 0 +distinct_count 198 +avg_col_len 2.8089430894308944 +max_col_len 3 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='1') value +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='1') value +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table1 +col_name value +data_type string +min +max +num_nulls 0 +distinct_count 191 +avg_col_len 6.808943089430894 +max_col_len 7 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') key +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') key +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table1 +col_name key +data_type string +min +max +num_nulls 0 +distinct_count 316 +avg_col_len 2.812 +max_col_len 3 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') value +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') value +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table1 +col_name value +data_type string +min +max +num_nulls 0 +distinct_count 307 +avg_col_len 6.812 +max_col_len 7 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') new_col +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table1 PARTITION (part='2') new_col +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table1 +col_name new_col +data_type int +min 1 +max 1 +num_nulls 0 +distinct_count 1 +avg_col_len +max_col_len +num_trues +num_falses +bit_vector HL +histogram Q1: 1, Q2: 1, Q3: 1 +comment from deserializer +PREHOOK: query: CREATE EXTERNAL TABLE rename_partition_table_ext0 (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS ORC +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@rename_partition_table_ext0 +POSTHOOK: query: CREATE EXTERNAL TABLE rename_partition_table_ext0 (key STRING, value STRING) PARTITIONED BY (part STRING) +STORED AS ORC +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@rename_partition_table_ext0 +PREHOOK: query: INSERT OVERWRITE TABLE rename_partition_table_ext0 PARTITION (part = '1') SELECT * FROM src where rand(1) < 0.5 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@rename_partition_table_ext0@part=1 +POSTHOOK: query: INSERT OVERWRITE TABLE rename_partition_table_ext0 PARTITION (part = '1') SELECT * FROM src where rand(1) < 0.5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@rename_partition_table_ext0@part=1 +POSTHOOK: Lineage: rename_partition_table_ext0 PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: rename_partition_table_ext0 PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: ALTER TABLE rename_partition_table_ext0 CHANGE COLUMN value val STRING CASCADE +PREHOOK: type: ALTERTABLE_RENAMECOL +PREHOOK: Input: default@rename_partition_table_ext0 +PREHOOK: Output: default@rename_partition_table_ext0 +PREHOOK: Output: default@rename_partition_table_ext0@part=1 +POSTHOOK: query: ALTER TABLE rename_partition_table_ext0 CHANGE COLUMN value val STRING CASCADE +POSTHOOK: type: ALTERTABLE_RENAMECOL +POSTHOOK: Input: default@rename_partition_table_ext0 +POSTHOOK: Output: default@rename_partition_table_ext0 +POSTHOOK: Output: default@rename_partition_table_ext0@part=1 +PREHOOK: query: INSERT OVERWRITE TABLE rename_partition_table_ext0 PARTITION (part = '2') SELECT * FROM src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@rename_partition_table_ext0@part=2 +POSTHOOK: query: INSERT OVERWRITE TABLE rename_partition_table_ext0 PARTITION (part = '2') SELECT * FROM src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@rename_partition_table_ext0@part=2 +POSTHOOK: Lineage: rename_partition_table_ext0 PARTITION(part=2).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: rename_partition_table_ext0 PARTITION(part=2).val SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: ALTER TABLE rename_partition_table_ext0 RENAME TO rename_partition_table_ext1 +PREHOOK: type: ALTERTABLE_RENAME +PREHOOK: Input: default@rename_partition_table_ext0 +PREHOOK: Output: database:default +PREHOOK: Output: default@rename_partition_table_ext0 +PREHOOK: Output: default@rename_partition_table_ext1 +POSTHOOK: query: ALTER TABLE rename_partition_table_ext0 RENAME TO rename_partition_table_ext1 +POSTHOOK: type: ALTERTABLE_RENAME +POSTHOOK: Input: default@rename_partition_table_ext0 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@rename_partition_table_ext0 +POSTHOOK: Output: default@rename_partition_table_ext1 +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table_ext1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table_ext1 +# col_name data_type comment +key string +val string + +# Partition Information +# col_name data_type comment +part string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 +#### A masked pattern was here #### + numFiles 2 + numPartitions 2 + numRows 746 + rawDataSize 131296 + totalSize 4669 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde +InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='1') key +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table_ext1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='1') key +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table_ext1 +col_name key +data_type string +min +max +num_nulls 0 +distinct_count 198 +avg_col_len 2.8089430894308944 +max_col_len 3 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='1') val +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table_ext1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='1') val +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table_ext1 +col_name val +data_type string +min +max +num_nulls +distinct_count +avg_col_len +max_col_len +num_trues +num_falses +bit_vector +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='2') key +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table_ext1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='2') key +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table_ext1 +col_name key +data_type string +min +max +num_nulls 0 +distinct_count 316 +avg_col_len 2.812 +max_col_len 3 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='2') val +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@rename_partition_table_ext1 +POSTHOOK: query: DESCRIBE FORMATTED rename_partition_table_ext1 PARTITION (part='2') val +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@rename_partition_table_ext1 +col_name val +data_type string +min +max +num_nulls 0 +distinct_count 307 +avg_col_len 6.812 +max_col_len 7 +num_trues +num_falses +bit_vector HL +histogram +comment from deserializer +PREHOOK: query: DROP TABLE rename_partition_table1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@rename_partition_table1 +PREHOOK: Output: database:default +PREHOOK: Output: default@rename_partition_table1 +POSTHOOK: query: DROP TABLE rename_partition_table1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@rename_partition_table1 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@rename_partition_table1 +PREHOOK: query: DROP TABLE rename_partition_table_ext1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@rename_partition_table_ext1 +PREHOOK: Output: database:default +PREHOOK: Output: default@rename_partition_table_ext1 +POSTHOOK: query: DROP TABLE rename_partition_table_ext1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@rename_partition_table_ext1 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@rename_partition_table_ext1 diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java index f6e41f090943..67c293ee64f4 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java @@ -64,6 +64,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.locks.ReentrantLock; @@ -181,7 +182,7 @@ private void populateInsertUpdateMap(Map statsP e -> e.partitionId).collect(Collectors.toList() ); - prefix.append("select \"PART_ID\", \"COLUMN_NAME\" from \"PART_COL_STATS\" WHERE "); + prefix.append("select \"PART_ID\", \"COLUMN_NAME\", \"ENGINE\" from \"PART_COL_STATS\" WHERE "); TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, partIdList, "\"PART_ID\"", true, false); @@ -191,7 +192,7 @@ private void populateInsertUpdateMap(Map statsP LOG.debug("Going to execute query " + query); rs = statement.executeQuery(query); while (rs.next()) { - selectedParts.add(new PartColNameInfo(rs.getLong(1), rs.getString(2))); + selectedParts.add(new PartColNameInfo(rs.getLong(1), rs.getString(2), rs.getString(3))); } } finally { close(rs, statement, null); @@ -207,7 +208,8 @@ private void populateInsertUpdateMap(Map statsP statsDesc.setCatName(tbl.getCatName()); } for (ColumnStatisticsObj statisticsObj : colStats.getStatsObj()) { - PartColNameInfo temp = new PartColNameInfo(partId, statisticsObj.getColName()); + PartColNameInfo temp = new PartColNameInfo(partId, statisticsObj.getColName(), + colStats.getEngine()); if (selectedParts.contains(temp)) { updateMap.put(temp, StatObjectConverter. convertToMPartitionColumnStatistics(null, statsDesc, statisticsObj, colStats.getEngine())); @@ -221,25 +223,46 @@ private void populateInsertUpdateMap(Map statsP private void updatePartColStatTable(Map updateMap, Connection dbConn) throws SQLException, MetaException, NoSuchObjectException { - PreparedStatement pst = null; - for (Map.Entry entry : updateMap.entrySet()) { - PartColNameInfo partColNameInfo = (PartColNameInfo) entry.getKey(); - Long partId = partColNameInfo.partitionId; - MPartitionColumnStatistics mPartitionColumnStatistics = (MPartitionColumnStatistics) entry.getValue(); - String update = "UPDATE \"PART_COL_STATS\" SET "; - update += StatObjectConverter.getUpdatedColumnSql(mPartitionColumnStatistics); - update += " WHERE \"PART_ID\" = " + partId + " AND " - + " \"COLUMN_NAME\" = " + quoteString(mPartitionColumnStatistics.getColName()); - try { - pst = dbConn.prepareStatement(update); - StatObjectConverter.initUpdatedColumnStatement(mPartitionColumnStatistics, pst); - LOG.debug("Going to execute update " + update); - int numUpdate = pst.executeUpdate(); - if (numUpdate != 1) { - throw new MetaException("Invalid state of PART_COL_STATS for PART_ID " + partId); + Map>> updates = new HashMap<>(); + for (Map.Entry entry : updateMap.entrySet()) { + MPartitionColumnStatistics mPartitionColumnStatistics = entry.getValue(); + StringBuilder update = new StringBuilder("UPDATE \"PART_COL_STATS\" SET ") + .append(StatObjectConverter.getUpdatedColumnSql(mPartitionColumnStatistics)) + .append(" WHERE \"PART_ID\" = ? AND \"COLUMN_NAME\" = ? AND \"ENGINE\" = ?"); + updates.computeIfAbsent(update.toString(), k -> new ArrayList<>()).add(entry); + } + + for (Map.Entry>> entry : updates.entrySet()) { + List partIds = new ArrayList<>(); + try (PreparedStatement pst = dbConn.prepareStatement(entry.getKey())) { + List> entries = entry.getValue(); + for (Map.Entry partStats : entries) { + PartColNameInfo partColNameInfo = partStats.getKey(); + MPartitionColumnStatistics mPartitionColumnStatistics = partStats.getValue(); + int colIdx = StatObjectConverter.initUpdatedColumnStatement(mPartitionColumnStatistics, pst); + pst.setLong(colIdx++, partColNameInfo.partitionId); + pst.setString(colIdx++, mPartitionColumnStatistics.getColName()); + pst.setString(colIdx++, mPartitionColumnStatistics.getEngine()); + partIds.add(partColNameInfo.partitionId); + pst.addBatch(); + if (partIds.size() == maxBatchSize) { + LOG.debug("Going to execute updates on part: {}", partIds); + verifyUpdates(pst.executeBatch(), partIds); + partIds = new ArrayList<>(); + } + } + if (!partIds.isEmpty()) { + LOG.debug("Going to execute updates on part: {}", partIds); + verifyUpdates(pst.executeBatch(), partIds); } - } finally { - closeStmt(pst); + } + } + } + + private void verifyUpdates(int[] numUpdates, List partIds) throws MetaException { + for (int i = 0; i < numUpdates.length; i++) { + if (numUpdates[i] != 1) { + throw new MetaException("Invalid state of PART_COL_STATS for PART_ID " + partIds.get(i)); } } } @@ -1501,9 +1524,11 @@ public boolean equals(Object o) private static final class PartColNameInfo { long partitionId; String colName; - public PartColNameInfo(long partitionId, String colName) { + String engine; + public PartColNameInfo(long partitionId, String colName, String engine) { this.partitionId = partitionId; this.colName = colName; + this.engine = engine; } @Override @@ -1527,10 +1552,10 @@ public boolean equals(Object o) if (this.partitionId != other.partitionId) { return false; } - if (this.colName.equalsIgnoreCase(other.colName)) { - return true; + if (!this.colName.equalsIgnoreCase(other.colName)) { + return false; } - return false; + return Objects.equals(this.engine, other.engine); } } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java index 1c18631e1cc3..a2807961b757 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java @@ -18,10 +18,8 @@ package org.apache.hadoop.hive.metastore; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.common.AcidMetaDataFile.DataFormat; @@ -63,6 +61,7 @@ import java.net.URI; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -352,7 +351,7 @@ public void alterTable(RawStore msdb, Warehouse wh, String catName, String dbnam // also the location field in partition parts = msdb.getPartitions(catName, dbname, name, -1); - Multimap columnStatsNeedUpdated = ArrayListMultimap.create(); + Map, List> partsByCols = new HashMap<>(); for (Partition part : parts) { String oldPartLoc = part.getSd().getLocation(); if (dataWasMoved && oldPartLoc.contains(oldTblLocPath)) { @@ -363,44 +362,57 @@ public void alterTable(RawStore msdb, Warehouse wh, String catName, String dbnam } part.setDbName(newDbName); part.setTableName(newTblName); - List multiColStats = updateOrGetPartitionColumnStats(msdb, catName, dbname, name, - part.getValues(), part.getSd().getCols(), oldt, part, null, null); - for (ColumnStatistics colStats : multiColStats) { - columnStatsNeedUpdated.put(part, colStats); + partsByCols.computeIfAbsent(part.getSd().getCols(), k -> new ArrayList<>()).add(part); + } + Map> engineToColStats = new HashMap<>(); + if (rename) { + // If this is the table rename, get the partition column statistics first + for (Map.Entry, List> entry : partsByCols.entrySet()) { + List colNames = entry.getKey().stream().map(fs -> fs.getName()).collect(Collectors.toList()); + List partNames = new ArrayList<>(); + for (Partition part : entry.getValue()) { + partNames.add(Warehouse.makePartName(oldt.getPartitionKeys(), part.getValues())); + } + List> colStats = + msdb.getPartitionColumnStatistics(catName, dbname, name, partNames, colNames); + for (List cs : colStats) { + if (cs != null && !cs.isEmpty()) { + String engine = cs.get(0).getEngine(); + cs.stream().forEach(stats -> { + stats.getStatsDesc().setDbName(newDbName); + stats.getStatsDesc().setTableName(newTblName); + String partName = stats.getStatsDesc().getPartName(); + engineToColStats.computeIfAbsent(engine, key -> new HashMap<>()).put(partName, stats); + }); + } + } } } // Do not verify stats parameters on a partitioned table. msdb.alterTable(catName, dbname, name, newt, null); + int partitionBatchSize = MetastoreConf.getIntVar(handler.getConf(), + MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX); + String catalogName = catName; // alterPartition is only for changing the partition location in the table rename if (dataWasMoved) { - - int partsToProcess = parts.size(); - int partitionBatchSize = MetastoreConf.getIntVar(handler.getConf(), - MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX); - int batchStart = 0; - while (partsToProcess > 0) { - int batchEnd = Math.min(batchStart + partitionBatchSize, parts.size()); - List partBatch = parts.subList(batchStart, batchEnd); - int partBatchSize = partBatch.size(); - partsToProcess -= partBatchSize; - batchStart += partBatchSize; - List> partValues = new ArrayList<>(partBatchSize); - for (Partition part : partBatch) { - partValues.add(part.getValues()); + Batchable.runBatched(partitionBatchSize, parts, new Batchable() { + @Override + public List run(List input) throws Exception { + msdb.alterPartitions(catalogName, newDbName, newTblName, + input.stream().map(Partition::getValues).collect(Collectors.toList()), + input, newt.getWriteId(), writeIdList); + return Collections.emptyList(); } - msdb.alterPartitions(catName, newDbName, newTblName, partValues, - partBatch, newt.getWriteId(), writeIdList); - } + }); } Deadline.checkTimeout(); - Table table = msdb.getTable(catName, newDbName, newTblName); - MTable mTable = msdb.ensureGetMTable(catName, newDbName, newTblName); - for (Entry partColStats : columnStatsNeedUpdated.entries()) { - ColumnStatistics newPartColStats = partColStats.getValue(); - newPartColStats.getStatsDesc().setDbName(newDbName); - newPartColStats.getStatsDesc().setTableName(newTblName); - msdb.updatePartitionColumnStatistics(table, mTable, newPartColStats, - partColStats.getKey().getValues(), writeIdList, newt.getWriteId()); + if (rename) { + for (Entry> entry : engineToColStats.entrySet()) { + // We will send ALTER_TABLE event after the db change, set listeners to null so that no extra + // event that could pollute the replication will be sent. + msdb.updatePartitionColumnStatisticsInBatch(entry.getValue(), oldt, + null, writeIdList, newt.getWriteId()); + } } } else { msdb.alterTable(catName, dbname, name, newt, writeIdList); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java index 5848abd2064a..163c855833e2 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java @@ -288,11 +288,13 @@ public static String getUpdatedColumnSql(MPartitionColumnStatistics mStatsObj) { if (mStatsObj.getNumNulls() != null) { setStmt.append("\"NUM_NULLS\" = ? ,"); } - setStmt.append("\"ENGINE\" = ? "); + setStmt.append("\"ENGINE\" = ? ,"); + setStmt.append("\"DB_NAME\" = ? ,"); + setStmt.append("\"TABLE_NAME\" = ? "); return setStmt.toString(); } - public static void initUpdatedColumnStatement(MPartitionColumnStatistics mStatsObj, + public static int initUpdatedColumnStatement(MPartitionColumnStatistics mStatsObj, PreparedStatement pst) throws SQLException { int colIdx = 1; if (mStatsObj.getAvgColLen() != null) { @@ -339,6 +341,9 @@ public static void initUpdatedColumnStatement(MPartitionColumnStatistics mStatsO pst.setObject(colIdx++, mStatsObj.getNumNulls()); } pst.setString(colIdx++, mStatsObj.getEngine()); + pst.setString(colIdx++, mStatsObj.getDbName()); + pst.setString(colIdx++, mStatsObj.getTableName()); + return colIdx; } public static ColumnStatisticsObj getTableColumnStatisticsObj( From 46fa50d8c3a0ee0323f21a5a4cd4c116b4bed5af Mon Sep 17 00:00:00 2001 From: gaoxiong Date: Sat, 20 Jan 2024 13:58:34 +0800 Subject: [PATCH 143/179] HIVE-27960: Invalid function error when using custom udaf (#4981)(gaoxiong, reviewed by Butao Zhang) --- data/files/datasets/temp_udaf/load.hive.sql | 1 + .../apache/hadoop/hive/ql/exec/Registry.java | 4 +- .../queries/clientpositive/ptf_register_use.q | 5 ++ .../llap/ptf_register_use.q.out | 84 +++++++++++++++++++ 4 files changed, 92 insertions(+), 2 deletions(-) create mode 100644 data/files/datasets/temp_udaf/load.hive.sql create mode 100644 ql/src/test/queries/clientpositive/ptf_register_use.q create mode 100644 ql/src/test/results/clientpositive/llap/ptf_register_use.q.out diff --git a/data/files/datasets/temp_udaf/load.hive.sql b/data/files/datasets/temp_udaf/load.hive.sql new file mode 100644 index 000000000000..f0e59b3feaee --- /dev/null +++ b/data/files/datasets/temp_udaf/load.hive.sql @@ -0,0 +1 @@ +create function ptf_register_use_func as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFRowNumber'; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java index d01da9f8f37e..8c8d90f25377 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java @@ -384,8 +384,8 @@ private void addToCurrentFunctions(String functionName, FunctionInfo functionInf public WindowFunctionInfo getWindowFunctionInfo(String functionName) throws SemanticException { // First try without qualifiers - would resolve builtin/temp functions FunctionInfo info = getFunctionInfo(WINDOW_FUNC_PREFIX + functionName); - // Try qualifying with current db name for permanent functions - if (info == null) { + // Try qualifying with current db name for permanent functions and try register function to session + if (info == null && FunctionRegistry.getFunctionInfo(functionName) != null) { String qualifiedName = FunctionUtils.qualifyFunctionName( functionName, SessionState.get().getCurrentDatabase().toLowerCase()); info = getFunctionInfo(WINDOW_FUNC_PREFIX + qualifiedName); diff --git a/ql/src/test/queries/clientpositive/ptf_register_use.q b/ql/src/test/queries/clientpositive/ptf_register_use.q new file mode 100644 index 000000000000..ff899a283dc0 --- /dev/null +++ b/ql/src/test/queries/clientpositive/ptf_register_use.q @@ -0,0 +1,5 @@ +--! qt:dataset:temp_udaf + +explain select ptf_register_use_func() over(); + +select ptf_register_use_func() over(); diff --git a/ql/src/test/results/clientpositive/llap/ptf_register_use.q.out b/ql/src/test/results/clientpositive/llap/ptf_register_use.q.out new file mode 100644 index 000000000000..3a7884306612 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/ptf_register_use.q.out @@ -0,0 +1,84 @@ +PREHOOK: query: explain select ptf_register_use_func() over() +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: explain select ptf_register_use_func() over() +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: _dummy_table + Row Limit Per Split: 1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: 0 (type: int) + null sort order: a + sort order: + + Map-reduce partition columns: 0 (type: int) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: 0 ASC NULLS FIRST + partition by: 0 + raw input shape: + window functions: + window function definition + alias: ptf_register_use_func_window_0 + name: ptf_register_use_func + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ptf_register_use_func_window_0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ptf_register_use_func() over() +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: select ptf_register_use_func() over() +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +1 From 529db3968fd30cabd308908a1d24b04ffb4498e1 Mon Sep 17 00:00:00 2001 From: cxzl25 <3898450+cxzl25@users.noreply.github.com> Date: Sat, 20 Jan 2024 14:06:07 +0800 Subject: [PATCH 144/179] HIVE-27979: HMS alter_partitions log adds table name (#4978)(dzcxzl,reviewed by Ayush Saxena, Butao Zhang) --- .../main/java/org/apache/hadoop/hive/metastore/HMSHandler.java | 3 ++- .../hadoop/hive/metastore/utils/MetaStoreServerUtils.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index 54eb8b3c2560..a4367d5716ab 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -5919,7 +5919,8 @@ private void alter_partitions_with_environment_context(String catName, String db if (LOG.isInfoEnabled()) { for (Partition tmpPart : new_parts) { - LOG.info("New partition values:" + tmpPart.getValues()); + LOG.info("New partition values: catalog: {} database: {} table: {} partition: {}", + catName, db_name, tbl_name, tmpPart.getValues()); } } // all partitions are altered atomically diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java index f4afe2aa65d7..3c9e0960bc1e 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java @@ -663,7 +663,8 @@ public static boolean updatePartitionStatsFast(PartitionSpecProxy.PartitionItera if (!madeDir) { // The partition location already existed and may contain data. Lets try to // populate those statistics that don't require a full scan of the data. - LOG.info("Updating partition stats fast for: {}", part.getTableName()); + LOG.info("Updating partition stats fast for: catalog: {} database: {} table: {} partition: {}", + part.getCatName(), part.getDbName(), part.getTableName(), part.getCurrent().getValues()); List fileStatus = wh.getFileStatusesForLocation(part.getLocation()); // TODO: this is invalid for ACID tables, and we cannot access AcidUtils here. populateQuickStats(fileStatus, params); From 5093bb1ffba84dd093fbec2b8d6b2fec2764087b Mon Sep 17 00:00:00 2001 From: M Lorek Date: Sun, 21 Jan 2024 16:49:21 +0000 Subject: [PATCH 145/179] HIVE-28008: ParquetFileReader is not closed in ParquetHiveSerDe.readSchema (#5013). (Michal Lorek, reviewed by Ayush Saxena, Butao Zhang, Attila Turoczy) --- .../hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java index 881d56cd31f2..a3a123ee00cb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java @@ -412,8 +412,9 @@ public List readSchema(Configuration conf, String file) throws SerD FileMetaData metadata; try { HadoopInputFile inputFile = HadoopInputFile.fromPath(new Path(file), conf); - ParquetFileReader reader = ParquetFileReader.open(inputFile); - metadata = reader.getFileMetaData(); + try(ParquetFileReader reader = ParquetFileReader.open(inputFile)) { + metadata = reader.getFileMetaData(); + } } catch (Exception e) { throw new SerDeException(ErrorMsg.PARQUET_FOOTER_ERROR.getErrorCodedMsg(), e); } From 93ef45e7f8dc4b2cb717f7ad126e24103afa72ea Mon Sep 17 00:00:00 2001 From: Dayakar M <59791497+mdayakar@users.noreply.github.com> Date: Mon, 22 Jan 2024 10:40:24 +0530 Subject: [PATCH 146/179] HIVE-27489: HPL/SQL does not support table aliases on column names in loops (Dayakar M, reviewed by Krisztian Kasa, Attila Turoczy) Co-authored-by: mdayakar --- .../java/org/apache/hive/hplsql/Stmt.java | 6 +++++- itests/hive-unit/pom.xml | 6 ++++++ .../hive/beeline/TestHplSqlViaBeeLine.java | 19 +++++++++++++++++++ 3 files changed, 30 insertions(+), 1 deletion(-) diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java index eac3a6d0d2de..78a25a0f2668 100644 --- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java +++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java @@ -957,7 +957,11 @@ public Integer forCursor(HplsqlParser.For_cursor_stmtContext ctx) { int cols = query.columnCount(); Row row = new Row(); for (int i = 0; i < cols; i++) { - row.addColumnDefinition(query.metadata().columnName(i), query.metadata().columnTypeName(i)); + String columnName = query.metadata().columnName(i); + if (columnName.contains(".")) { + columnName = columnName.substring(columnName.lastIndexOf('.') + 1); + } + row.addColumnDefinition(columnName, query.metadata().columnTypeName(i)); } Var var = new Var(cursor, row); exec.addVariable(var); diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index 7c3b2333b9eb..fd64a7853171 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -494,6 +494,12 @@ postgresql test + + net.sf.supercsv + super-csv + ${super-csv.version} + test + diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java index 4118013769fe..eb2285ba16c5 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestHplSqlViaBeeLine.java @@ -869,6 +869,25 @@ public void testUSERHplSQLFunction() throws Throwable { testScriptFile(SCRIPT_TEXT, args(), System.getProperty("user.name")); } + @Test + public void testTableAliasInColumnName() throws Throwable { + String SCRIPT_TEXT = + "DROP TABLE IF EXISTS input;\n" + + "DROP TABLE IF EXISTS result;\n" + + "CREATE TABLE input (col1 string, col2 int);\n" + + "CREATE TABLE result (res string);\n" + + "INSERT INTO input VALUES('Hive', 2023);\n" + + "CREATE PROCEDURE p1() AS\n" + + " BEGIN\n" + + " FOR rec IN (select tab.col1, tab.col2 num from input tab) LOOP\n" + + " INSERT INTO result VALUES(rec.num || ' = ' || rec.col1);\n" + + " END LOOP;\n" + + "END;\n" + + "p1();\n" + + "SELECT * FROM result;\n"; + testScriptFile(SCRIPT_TEXT, args(), "2023 = Hive"); + } + private static List args() { return Arrays.asList("-d", BeeLine.BEELINE_DEFAULT_JDBC_DRIVER, "-u", miniHS2.getBaseJdbcURL() + ";mode=hplsql", "-n", userName); From f71a50417b4f1271496d9e4a04aefd5cbc906f71 Mon Sep 17 00:00:00 2001 From: Krisztian Kasa Date: Mon, 22 Jan 2024 14:18:27 +0100 Subject: [PATCH 147/179] HIVE-28009: Shared work optimizer ignores schema merge setting in case of virtual column difference (Krisztian Kasa, reviewed by Denys Kuzmenko) --- ..._iceberg_copy_on_write_unpartitioned.q.out | 28 +- ...te_iceberg_copy_on_write_partitioned.q.out | 86 ++-- ..._iceberg_copy_on_write_unpartitioned.q.out | 88 ++-- .../ql/optimizer/SharedWorkOptimizer.java | 5 +- .../sharedwork_virtualcol_schema_merge.q | 35 ++ .../sharedwork_virtualcol_schema_merge.q.out | 432 ++++++++++++++++++ 6 files changed, 571 insertions(+), 103 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/sharedwork_virtualcol_schema_merge.q create mode 100644 ql/src/test/results/clientpositive/llap/sharedwork_virtualcol_schema_merge.q.out diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out index 6728fbf39bf7..fb70b3e6af98 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out @@ -166,20 +166,6 @@ STAGE PLANS: Map-reduce partition columns: _col4 (type: int) Statistics: Num rows: 4 Data size: 832 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) - Filter Operator - predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: FILE__PATH (type: string), a (type: int) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col1 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: string) Filter Operator predicate: (FILE__PATH is not null and a is not null) (type: boolean) Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE @@ -194,6 +180,20 @@ STAGE PLANS: Map-reduce partition columns: _col4 (type: int) Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int) + Filter Operator + predicate: (a is not null and FILE__PATH is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: FILE__PATH (type: string), a (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col1 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col1 (type: int) + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string) Filter Operator predicate: a is not null (type: boolean) Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out index 7a5b872fa115..724b0631f95a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out @@ -1227,9 +1227,9 @@ STAGE PLANS: Reducer 3 <- Reducer 14 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) Reducer 6 <- Union 5 (SIMPLE_EDGE) - Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) - Reducer 8 <- Reducer 7 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 5 (CONTAINS) #### A masked pattern was here #### Vertices: Map 1 @@ -1253,29 +1253,29 @@ STAGE PLANS: value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) Select Operator - expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 @@ -1546,6 +1546,27 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice Reducer 7 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + Reducer 8 Reduce Operator Tree: Merge Join Operator condition map: @@ -1562,7 +1583,7 @@ STAGE PLANS: Map-reduce partition columns: _col5 (type: string) Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) - Reducer 8 + Reducer 9 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -1602,27 +1623,6 @@ STAGE PLANS: Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) - Reducer 9 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) - Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Union 5 Vertex: Union 5 @@ -1772,29 +1772,29 @@ STAGE PLANS: value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) Select Operator - expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out index 98c24ce144c9..edbb1b2523e2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out @@ -1199,9 +1199,9 @@ STAGE PLANS: Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE) Reducer 3 <- Reducer 13 (XPROD_EDGE), Reducer 2 (XPROD_EDGE) Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) - Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 5 (CONTAINS) - Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 5 (CONTAINS) + Reducer 7 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 8 <- Reducer 7 (SIMPLE_EDGE), Union 5 (CONTAINS) Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: @@ -1226,29 +1226,29 @@ STAGE PLANS: value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) Select Operator - expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 @@ -1454,6 +1454,28 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice + Reducer 7 Reduce Operator Tree: Merge Join Operator condition map: @@ -1470,7 +1492,7 @@ STAGE PLANS: Map-reduce partition columns: _col5 (type: string) Statistics: Num rows: 1 Data size: 293 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) - Reducer 7 + Reducer 8 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -1511,28 +1533,6 @@ STAGE PLANS: output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice - Reducer 8 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 311 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 3 Data size: 913 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice Reducer 9 Reduce Operator Tree: Merge Join Operator @@ -1732,29 +1732,29 @@ STAGE PLANS: value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) + expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) + Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) Select Operator - expressions: a (type: int), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) + expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint) + Statistics: Num rows: 9 Data size: 2637 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java index 67437aed3620..2f80bcfdb583 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SharedWorkOptimizer.java @@ -649,8 +649,9 @@ protected void mergeSchema(TableScanOperator discardableTsOp, TableScanOperator } private static boolean compatibleSchema(TableScanOperator tsOp1, TableScanOperator tsOp2) { - return tsOp1.getNeededColumns().equals(tsOp2.getNeededColumns()) - && tsOp1.getNeededColumnIDs().equals(tsOp2.getNeededColumnIDs()); + return Objects.equals(tsOp1.getNeededColumns(), tsOp2.getNeededColumns()) + && Objects.equals(tsOp1.getNeededColumnIDs(), tsOp2.getNeededColumnIDs()) + && Objects.equals(tsOp1.getConf().getVirtualCols(), tsOp2.getConf().getVirtualCols()); } diff --git a/ql/src/test/queries/clientpositive/sharedwork_virtualcol_schema_merge.q b/ql/src/test/queries/clientpositive/sharedwork_virtualcol_schema_merge.q new file mode 100644 index 000000000000..c471201222a9 --- /dev/null +++ b/ql/src/test/queries/clientpositive/sharedwork_virtualcol_schema_merge.q @@ -0,0 +1,35 @@ +set hive.optimize.shared.work.merge.ts.schema=false; + +create table t1(a int); + +-- 3 map vertices scans table t1 +explain +WITH t AS ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a, row_number() OVER (partition by INPUT__FILE__NAME) rn from t1 + where a = 1 + ) q + where rn=1 +) +select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from t1 where NOT (a = 1) AND INPUT__FILE__NAME IN (select INPUT__FILE__NAME from t) +union all +select * from t; + + + +set hive.optimize.shared.work.merge.ts.schema=true; + +-- 2 of 3 map vertices scanning table t1 are merged: +-- One projects BLOCK__OFFSET__INSIDE__FILE and INPUT__FILE__NAME and the second one projects INPUT__FILE__NAME only. +-- These are merged to one scan which projects both. +explain +WITH t AS ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a, row_number() OVER (partition by INPUT__FILE__NAME) rn from t1 + where a = 1 + ) q + where rn=1 +) +select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from t1 where NOT (a = 1) AND INPUT__FILE__NAME IN (select INPUT__FILE__NAME from t) +union all +select * from t; diff --git a/ql/src/test/results/clientpositive/llap/sharedwork_virtualcol_schema_merge.q.out b/ql/src/test/results/clientpositive/llap/sharedwork_virtualcol_schema_merge.q.out new file mode 100644 index 000000000000..0f42feefbde1 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/sharedwork_virtualcol_schema_merge.q.out @@ -0,0 +1,432 @@ +PREHOOK: query: create table t1(a int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1(a int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: explain +WITH t AS ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a, row_number() OVER (partition by INPUT__FILE__NAME) rn from t1 + where a = 1 + ) q + where rn=1 +) +select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from t1 where NOT (a = 1) AND INPUT__FILE__NAME IN (select INPUT__FILE__NAME from t) +union all +select * from t +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: explain +WITH t AS ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a, row_number() OVER (partition by INPUT__FILE__NAME) rn from t1 + where a = 1 + ) q + where rn=1 +) +select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from t1 where NOT (a = 1) AND INPUT__FILE__NAME IN (select INPUT__FILE__NAME from t) +union all +select * from t +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 7 <- Map 6 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: ((a <> 1) and INPUT__FILE__NAME is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((a <> 1) and INPUT__FILE__NAME is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: a (type: int), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: bigint) + Execution mode: llap + LLAP IO: all inputs + Map 4 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: ((a = 1) and INPUT__FILE__NAME is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((a = 1) and INPUT__FILE__NAME is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: INPUT__FILE__NAME (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: INPUT__FILE__NAME (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Execution mode: llap + LLAP IO: all inputs + Map 6 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: (a = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (a = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: INPUT__FILE__NAME (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: INPUT__FILE__NAME (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: BLOCK__OFFSET__INSIDE__FILE (type: bigint) + Execution mode: llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col2 (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col2 (type: string), 1 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Union 3 + Vertex: Union 3 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +WITH t AS ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a, row_number() OVER (partition by INPUT__FILE__NAME) rn from t1 + where a = 1 + ) q + where rn=1 +) +select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from t1 where NOT (a = 1) AND INPUT__FILE__NAME IN (select INPUT__FILE__NAME from t) +union all +select * from t +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: explain +WITH t AS ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from ( + select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a, row_number() OVER (partition by INPUT__FILE__NAME) rn from t1 + where a = 1 + ) q + where rn=1 +) +select BLOCK__OFFSET__INSIDE__FILE, INPUT__FILE__NAME, a from t1 where NOT (a = 1) AND INPUT__FILE__NAME IN (select INPUT__FILE__NAME from t) +union all +select * from t +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 1 (SIMPLE_EDGE) + Reducer 6 <- Map 5 (SIMPLE_EDGE), Union 3 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: (((a <> 1) and INPUT__FILE__NAME is not null) or ((a = 1) and INPUT__FILE__NAME is not null)) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((a <> 1) and INPUT__FILE__NAME is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: a (type: int), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int), _col1 (type: bigint) + Filter Operator + predicate: ((a = 1) and INPUT__FILE__NAME is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: INPUT__FILE__NAME (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: INPUT__FILE__NAME (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Execution mode: llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: (a = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (a = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: INPUT__FILE__NAME (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: INPUT__FILE__NAME (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + value expressions: BLOCK__OFFSET__INSIDE__FILE (type: bigint) + Execution mode: llap + LLAP IO: all inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col2 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col2 (type: string), _col0 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + output shape: _col2: string + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: bigint), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + PTF Operator + Function definitions: + Input definition + input alias: ptf_0 + type: WINDOWING + Windowing table definition + input alias: ptf_1 + name: windowingtablefunction + order by: _col2 ASC NULLS FIRST + partition by: _col2 + raw input shape: + window functions: + window function definition + alias: row_number_window_0 + name: row_number + window function: GenericUDAFRowNumberEvaluator + window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) + isPivotResult: true + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (row_number_window_0 = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col2 (type: string), 1 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Union 3 + Vertex: Union 3 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + From 1aeaff2057a2f4c241f8bcc53a2a529e6e7f45d4 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Mon, 22 Jan 2024 19:30:40 +0530 Subject: [PATCH 148/179] HIVE-27991: Utilise FanoutWriters when inserting records in an Iceberg table when the records are unsorted (#4988) (Sourabh Badhya reviewed by Butao Zhang) --- .../mr/hive/HiveIcebergOutputFormat.java | 1 + .../mr/hive/HiveIcebergStorageHandler.java | 1 + .../hive/writer/HiveIcebergRecordWriter.java | 14 +++++++++++--- .../iceberg/mr/hive/writer/WriterBuilder.java | 9 ++++++++- .../ctas_iceberg_partitioned_orc.q.out | 2 +- .../hadoop/hive/ql/exec/FileSinkOperator.java | 3 +++ .../hive/ql/plan/DynamicPartitionCtx.java | 10 ++++++++++ .../HiveCustomStorageHandlerUtils.java | 19 +++++++++++++++++++ 8 files changed, 54 insertions(+), 5 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java index c356898c65eb..508e3459269b 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java @@ -77,6 +77,7 @@ private static HiveIcebergWriter writer(JobConf jc) { .attemptID(taskAttemptID) .poolSize(poolSize) .operation(HiveCustomStorageHandlerUtils.getWriteOperation(jc, tableName)) + .isFanoutEnabled(!HiveCustomStorageHandlerUtils.getWriteOperationIsSorted(jc, tableName)) .build(); } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index d07b820e6d04..cd50aa929c08 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -702,6 +702,7 @@ public DynamicPartitionCtx createDPContext( addCustomSortExpr(table, hmsTable, writeOperation, customSortExprs, getSortTransformSpec(table)); } + dpCtx.setHasCustomSortExprs(!customSortExprs.isEmpty()); return dpCtx; } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java index 26c4e0947d01..b43376ec7f99 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java @@ -29,9 +29,11 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.ClusteredDataWriter; import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.FanoutDataWriter; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.mr.hive.FilesForCommit; import org.apache.iceberg.mr.mapred.Container; @@ -41,9 +43,8 @@ class HiveIcebergRecordWriter extends HiveIcebergWriterBase { HiveIcebergRecordWriter(Schema schema, Map specs, int currentSpecId, FileWriterFactory fileWriterFactory, OutputFileFactory fileFactory, FileIO io, - long targetFileSize) { - super(schema, specs, io, - new ClusteredDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize)); + long targetFileSize, boolean fanoutEnabled) { + super(schema, specs, io, getIcebergDataWriter(fileWriterFactory, fileFactory, io, targetFileSize, fanoutEnabled)); this.currentSpecId = currentSpecId; } @@ -58,4 +59,11 @@ public FilesForCommit files() { List dataFiles = ((DataWriteResult) writer.result()).dataFiles(); return FilesForCommit.onlyData(dataFiles); } + + private static PartitioningWriter getIcebergDataWriter(FileWriterFactory fileWriterFactory, + OutputFileFactory fileFactory, FileIO io, + long targetFileSize, boolean fanoutEnabled) { + return fanoutEnabled ? new FanoutDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize) + : new ClusteredDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize); + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java index 2f7177b63810..c68bf8fb1d6a 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java @@ -45,6 +45,8 @@ public class WriterBuilder { private String queryId; private int poolSize; private Operation operation; + private boolean fanoutEnabled; + // A task may write multiple output files using multiple writers. Each of them must have a unique operationId. private static AtomicInteger operationNum = new AtomicInteger(0); @@ -85,6 +87,11 @@ public WriterBuilder operation(Operation newOperation) { return this; } + public WriterBuilder isFanoutEnabled(boolean isFanoutEnabled) { + this.fanoutEnabled = isFanoutEnabled; + return this; + } + public HiveIcebergWriter build() { Map properties = table.properties(); @@ -133,7 +140,7 @@ public HiveIcebergWriter build() { break; case OTHER: writer = new HiveIcebergRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, - io, targetFileSize); + io, targetFileSize, fanoutEnabled); break; default: // Update and Merge should be splitted to inserts and deletes diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out index d0c2aef2bf19..f6a407b35bea 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out @@ -333,5 +333,5 @@ POSTHOOK: query: select * from tbl_ice POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice POSTHOOK: Output: hdfs://### HDFS PATH ### -1 one 3 1 two 4 +1 one 3 diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index d7903747ae9c..e05e36e60be4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TEMPORARY_TABLE_STORAGE; import static org.apache.hadoop.hive.ql.security.authorization.HiveCustomStorageHandlerUtils.setWriteOperation; +import static org.apache.hadoop.hive.ql.security.authorization.HiveCustomStorageHandlerUtils.setWriteOperationIsSorted; import java.io.IOException; import java.io.Serializable; @@ -633,6 +634,8 @@ protected void initializeOp(Configuration hconf) throws HiveException { jc = new JobConf(hconf); setWriteOperation(jc, getConf().getTableInfo().getTableName(), getConf().getWriteOperation()); + setWriteOperationIsSorted(jc, getConf().getTableInfo().getTableName(), + dpCtx != null && dpCtx.hasCustomSortExprs()); try { createHiveOutputFormat(jc); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPartitionCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPartitionCtx.java index 3497f3120cd4..580b4499217e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPartitionCtx.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPartitionCtx.java @@ -52,6 +52,7 @@ public class DynamicPartitionCtx implements Serializable { private String defaultPartName; // default partition name in case of null or empty value private int maxPartsPerNode; // maximum dynamic partitions created per mapper/reducer private Pattern whiteListPattern; + private boolean hasCustomSortExprs = false; /** * Expressions describing a custom way of sorting the table before write. Expressions can reference simple * column descriptions or a tree of expressions containing more columns and UDFs. @@ -147,6 +148,7 @@ public DynamicPartitionCtx(DynamicPartitionCtx dp) { this.maxPartsPerNode = dp.maxPartsPerNode; this.whiteListPattern = dp.whiteListPattern; this.customSortExpressions = dp.customSortExpressions; + this.hasCustomSortExprs = dp.customSortExpressions != null && !dp.customSortExpressions.isEmpty(); this.customSortOrder = dp.customSortOrder; this.customSortNullOrder = dp.customSortNullOrder; } @@ -258,4 +260,12 @@ public List getCustomSortNullOrder() { public void setCustomSortNullOrder(List customSortNullOrder) { this.customSortNullOrder = customSortNullOrder; } + + public boolean hasCustomSortExprs() { + return hasCustomSortExprs; + } + + public void setHasCustomSortExprs(boolean hasCustomSortExprs) { + this.hasCustomSortExprs = hasCustomSortExprs; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveCustomStorageHandlerUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveCustomStorageHandlerUtils.java index 8be4cfc5b8f8..2f2f7d781c00 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveCustomStorageHandlerUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveCustomStorageHandlerUtils.java @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.common.StatsSetupConst; @@ -31,6 +32,7 @@ public class HiveCustomStorageHandlerUtils { public static final String WRITE_OPERATION_CONFIG_PREFIX = "file.sink.write.operation."; + public static final String WRITE_OPERATION_IS_SORTED = "file.sink.write.operation.sorted."; public static String getTablePropsForCustomStorageHandler(Map tableProperties) { StringBuilder properties = new StringBuilder(); @@ -71,4 +73,21 @@ public static void setWriteOperation(Configuration conf, String tableName, Conte conf.set(WRITE_OPERATION_CONFIG_PREFIX + tableName, operation.name()); } + + public static void setWriteOperationIsSorted(Configuration conf, String tableName, boolean isSorted) { + if (conf == null || tableName == null) { + return; + } + + conf.set(WRITE_OPERATION_IS_SORTED + tableName, Boolean.toString(isSorted)); + } + + public static boolean getWriteOperationIsSorted(Configuration conf, String tableName) { + if (conf == null || tableName == null) { + return false; + } + + String isSortedString = conf.get(WRITE_OPERATION_IS_SORTED + tableName); + return Boolean.parseBoolean(isSortedString); + } } From 36d32ec7fd6ac9053e6a9d28f01dd431149a5ac4 Mon Sep 17 00:00:00 2001 From: Ryu Kobayashi Date: Tue, 23 Jan 2024 13:05:48 +0900 Subject: [PATCH 149/179] HIVE-26713: StringExpr ArrayIndexOutOfBoundsException with LIKE '%xxx%' (#4999)(Ryu Kobayashi, reviewed by Attila Turoczy, Butao Zhang) --- data/files/control_characters.txt | 1 + .../clientpositive/like_control_characters.q | 13 +++ .../llap/like_control_characters.q.out | 93 +++++++++++++++++++ .../exec/vector/expressions/StringExpr.java | 10 +- .../vector/expressions/TestStringExpr.java | 24 ++++- 5 files changed, 138 insertions(+), 3 deletions(-) create mode 100644 data/files/control_characters.txt create mode 100644 ql/src/test/queries/clientpositive/like_control_characters.q create mode 100644 ql/src/test/results/clientpositive/llap/like_control_characters.q.out diff --git a/data/files/control_characters.txt b/data/files/control_characters.txt new file mode 100644 index 000000000000..4e3fc6c45351 --- /dev/null +++ b/data/files/control_characters.txt @@ -0,0 +1 @@ +abcdeÿfghi \ No newline at end of file diff --git a/ql/src/test/queries/clientpositive/like_control_characters.q b/ql/src/test/queries/clientpositive/like_control_characters.q new file mode 100644 index 000000000000..5f9772ed2ef9 --- /dev/null +++ b/ql/src/test/queries/clientpositive/like_control_characters.q @@ -0,0 +1,13 @@ +set hive.mapred.mode=nonstrict; +set hive.explain.user=false; +set hive.vectorized.execution.enabled=true; + +create temporary table foo (col string); + +-- SORT_QUERY_RESULTS + +LOAD DATA LOCAL INPATH '../../data/files/control_characters.txt' INTO TABLE foo; + +explain select col, count(*) from foo where col like '%fg%' group by col; +select col, count(*) from foo where col like '%fg%' group by col; + diff --git a/ql/src/test/results/clientpositive/llap/like_control_characters.q.out b/ql/src/test/results/clientpositive/llap/like_control_characters.q.out new file mode 100644 index 000000000000..14aa86328db4 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/like_control_characters.q.out @@ -0,0 +1,93 @@ +PREHOOK: query: create temporary table foo (col string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@foo +POSTHOOK: query: create temporary table foo (col string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@foo +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/control_characters.txt' INTO TABLE foo +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@foo +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/control_characters.txt' INTO TABLE foo +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@foo +PREHOOK: query: explain select col, count(*) from foo where col like '%fg%' group by col +PREHOOK: type: QUERY +PREHOOK: Input: default@foo +#### A masked pattern was here #### +POSTHOOK: query: explain select col, count(*) from foo where col like '%fg%' group by col +POSTHOOK: type: QUERY +POSTHOOK: Input: default@foo +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: foo + filterExpr: (col like '%fg%') (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (col like '%fg%') (type: boolean) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + keys: col (type: string) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select col, count(*) from foo where col like '%fg%' group by col +PREHOOK: type: QUERY +PREHOOK: Input: default@foo +#### A masked pattern was here #### +POSTHOOK: query: select col, count(*) from foo where col like '%fg%' group by col +POSTHOOK: type: QUERY +POSTHOOK: Input: default@foo +#### A masked pattern was here #### +abcde�fghi 1 diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java index b6d3184ffedb..34097167ac13 100644 --- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java +++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java @@ -342,7 +342,15 @@ public int find(byte[] input, int start, int len) { } s_tmp--; } - next += shift[input[next] & MAX_BYTE]; + + // if the character string contains control characters, + // overflow occurs. + int shiftIndex = input[next] & MAX_BYTE; + if (shiftIndex >= MAX_BYTE) { + next++; + } else { + next += shift[shiftIndex]; + } } return -1; } diff --git a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestStringExpr.java b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestStringExpr.java index 6fb66115277d..483eb68b9795 100644 --- a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestStringExpr.java +++ b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestStringExpr.java @@ -20,9 +20,11 @@ import org.junit.Test; -import java.nio.charset.StandardCharsets; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.*; +import java.io.ByteArrayOutputStream; +import java.nio.charset.StandardCharsets; public class TestStringExpr { @Test @@ -49,6 +51,24 @@ public void test() throws Exception { assertEquals("Testing match at end of string", 24, find(pattern, input4)); } + @Test + public void testControlCharacters() throws Exception { + StringExpr.Finder pattern = compile("pattern"); + assertNotNull(pattern); + + byte b = -1; + byte[] controlBytes1 = "abcedf".getBytes(StandardCharsets.UTF_8); + byte[] controlBytes2 = "pattern".getBytes(StandardCharsets.UTF_8); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + outputStream.write(controlBytes1); + outputStream.write(b); + outputStream.write(controlBytes2); + byte[] controlChar = outputStream.toByteArray(); + outputStream.close(); + + assertEquals("Testing valid match", 7, pattern.find(controlChar, 0, controlChar.length)); + } + private StringExpr.Finder compile(String pattern) { return StringExpr.compile(pattern.getBytes(StandardCharsets.UTF_8)); } From 0c61018596aebc949b90e0938f4c4b046659dee6 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Tue, 23 Jan 2024 15:21:15 +0200 Subject: [PATCH 150/179] HIVE-28016: Iceberg: NULL column values handling in COW mode (Denys Kuzmenko, reviewed by Attila Turoczy, Krisztian Kasa) Closes #5016 --- ...lete_iceberg_copy_on_write_unpartitioned.q | 10 + ...copy_on_write.q => delete_iceberg_mixed.q} | 13 +- .../delete_iceberg_unpartitioned_parquet.q | 10 + ...te_iceberg_copy_on_write_partitioned.q.out | 277 +++++++------ ..._iceberg_copy_on_write_unpartitioned.q.out | 316 +++++++++------ ...write.q.out => delete_iceberg_mixed.q.out} | 47 ++- ...delete_iceberg_unpartitioned_parquet.q.out | 55 +++ ...ge_iceberg_copy_on_write_partitioned.q.out | 2 +- ..._iceberg_copy_on_write_unpartitioned.q.out | 93 +++-- ...te_iceberg_copy_on_write_partitioned.q.out | 359 ++++++++--------- ..._iceberg_copy_on_write_unpartitioned.q.out | 369 ++++++++---------- .../rewrite/CopyOnWriteDeleteRewriter.java | 3 +- .../rewrite/CopyOnWriteMergeRewriter.java | 12 +- .../rewrite/CopyOnWriteUpdateRewriter.java | 3 +- 14 files changed, 840 insertions(+), 729 deletions(-) rename iceberg/iceberg-handler/src/test/queries/positive/{iceberg_copy_on_write.q => delete_iceberg_mixed.q} (88%) rename iceberg/iceberg-handler/src/test/results/positive/{iceberg_copy_on_write.q.out => delete_iceberg_mixed.q.out} (87%) diff --git a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q index cf192789234a..815207398237 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_copy_on_write_unpartitioned.q @@ -38,3 +38,13 @@ explain delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_sta delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); select count(*) from tbl_ice; -- 0 + +-- null cases +drop table if exists tbl_ice_with_nulls; +create table tbl_ice_with_nulls (id int, name string) stored by iceberg tblproperties('format-version'='2', 'write.delete.mode'='copy-on-write'); + +insert into tbl_ice_with_nulls values +(1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(7, 'SDF'),(9, null),(8,'POIKL'),(10, 'YUIO'); + +delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf'); +select * from tbl_ice_with_nulls order by id; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q similarity index 88% rename from iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q rename to iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q index 3bfe3dee23f4..bbad2f380e35 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_copy_on_write.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q @@ -5,7 +5,6 @@ --! qt:replace:/("total-files-size":")\d+/$1#FileSize#/ --! qt:replace:/("removed-files-size":")\d+/$1#FileSize#/ - -- create an unpartitioned table with skip delete data set to false create table ice01 (id int, name string) Stored by Iceberg stored as ORC TBLPROPERTIES('format-version'='2'); @@ -17,7 +16,6 @@ insert into ice01 values (1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR') -- delete using MOR delete from ice01 where id>9 OR id=8; - select * from ice01; -- should be 2 files, one data file and one positional delete file. @@ -34,21 +32,24 @@ select * from ice01; -- should be only one data file. select summary from default.ice01.snapshots; -select * from ice01; +-- null cases -delete from ice01 where name=null; +delete from ice01 where null; +select * from ice01; +delete from ice01 where not null; select * from ice01; -delete from ice01 where name!=null; +delete from ice01 where name = null; +select * from ice01; +delete from ice01 where name != null; select * from ice01; --disable cbo due to HIVE-27070 set hive.cbo.enable=false; delete from ice01 where name is null; - select * from ice01; -- clean up diff --git a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_unpartitioned_parquet.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_unpartitioned_parquet.q index c0430591e3ca..89c2822374f8 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_unpartitioned_parquet.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_unpartitioned_parquet.q @@ -28,3 +28,13 @@ insert into tbl_standard_other values (10, 'ten'), (444, 'tutu'); delete from tbl_ice where a in (select t1.a from tbl_ice t1 join tbl_standard_other t2 on t1.a = t2.a); select count(*) from tbl_ice; -- 0 + +-- null cases +drop table if exists tbl_ice_with_nulls; +create table tbl_ice_with_nulls (id int, name string) stored by iceberg tblproperties('format-version'='2'); + +insert into tbl_ice_with_nulls values +(1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(7, 'SDF'),(9, null),(8,'POIKL'),(10, 'YUIO'); + +delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf'); +select * from tbl_ice_with_nulls order by id; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out index e5fe8e90db7d..333d5146d592 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out @@ -67,11 +67,11 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product -Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product -Warning: Shuffle Join MERGEJOIN[226][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 11' is a cross product -Warning: Shuffle Join MERGEJOIN[233][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product -Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[225][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[227][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[223][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[221][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 14' is a cross product Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 16' is a cross product PREHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY @@ -92,28 +92,29 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Reducer 20 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) - Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) + Reducer 10 <- Reducer 19 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 24 (XPROD_EDGE) Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) Reducer 13 <- Reducer 12 (SIMPLE_EDGE) - Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 19 (XPROD_EDGE) - Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE) - Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) + Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 20 (XPROD_EDGE) + Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 24 (XPROD_EDGE) Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) Reducer 18 <- Reducer 17 (SIMPLE_EDGE), Union 7 (CONTAINS) - Reducer 19 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE) - Reducer 20 <- Map 1 (SIMPLE_EDGE) + Reducer 19 <- Map 1 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 20 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 22 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 23 <- Map 1 (SIMPLE_EDGE) Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) - Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) - Reducer 5 <- Reducer 23 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) - Reducer 6 <- Reducer 24 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Union 7 (CONTAINS) + Reducer 4 <- Reducer 23 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 25 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) + Reducer 6 <- Reducer 13 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS) Reducer 8 <- Union 7 (SIMPLE_EDGE) - Reducer 9 <- Map 1 (XPROD_EDGE), Reducer 19 (XPROD_EDGE) + Reducer 9 <- Map 1 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -156,6 +157,18 @@ STAGE PLANS: Filter Operator predicate: (a <= 5) (type: boolean) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: a (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator @@ -169,18 +182,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Group By Operator - keys: a (type: int) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count(), count(a) minReductionHashAggr: 0.4 @@ -192,14 +193,11 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Filter Operator - predicate: (c > 800) (type: boolean) - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - minReductionHashAggr: 0.8333333 + minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE @@ -208,6 +206,9 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) + Filter Operator + predicate: (c > 800) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: c (type: int) minReductionHashAggr: 0.6666666 @@ -220,6 +221,19 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8333333 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Group By Operator aggregations: count(), count(c) minReductionHashAggr: 0.8333333 @@ -439,39 +453,6 @@ STAGE PLANS: Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 19 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 - Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) - Reducer 20 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -504,6 +485,34 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 2 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 6 Data size: 1822 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 20 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Reducer 21 Execution mode: vectorized Reduce Operator Tree: @@ -530,11 +539,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Reducer 23 Execution mode: vectorized Reduce Operator Tree: @@ -569,6 +573,24 @@ STAGE PLANS: Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) Reducer 24 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 25 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -595,38 +617,14 @@ STAGE PLANS: expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col5 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean) Reducer 4 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col5 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col2 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - Reducer 5 Reduce Operator Tree: Merge Join Operator condition map: @@ -634,14 +632,14 @@ STAGE PLANS: keys: 0 _col2 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 - Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12 + Statistics: Num rows: 6 Data size: 1942 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) - Reducer 6 + Statistics: Num rows: 6 Data size: 1942 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col12 (type: boolean) + Reducer 5 Reduce Operator Tree: Merge Join Operator condition map: @@ -649,26 +647,47 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 - Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12, _col13, _col14 + Statistics: Num rows: 6 Data size: 2038 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 - Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col14 + Statistics: Num rows: 6 Data size: 2038 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) - Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null) or (_col14 is not null and (_col11 <> 0L)) or ((_col2 is null or (_col12 < _col11)) and null and (_col11 <> 0L) and _col14 is null)) is null or (((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) and ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)))) (type: boolean) + Statistics: Num rows: 6 Data size: 2038 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) - Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 8 Execution mode: vectorized Reduce Operator Tree: @@ -723,11 +742,11 @@ STAGE PLANS: Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product -Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product -Warning: Shuffle Join MERGEJOIN[226][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 11' is a cross product -Warning: Shuffle Join MERGEJOIN[233][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product -Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[225][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[227][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[223][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[221][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 14' is a cross product Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 16' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY @@ -1006,7 +1025,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) @@ -1458,7 +1477,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out index 22256bab0106..061d697e03f9 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out @@ -48,10 +48,10 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - filterExpr: (((a <> 22) and (b <> 'one') and (b <> 'four')) or (b) IN ('one', 'four') or (a = 22)) (type: boolean) + filterExpr: (((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null or (b) IN ('one', 'four') or (a = 22)) (type: boolean) Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + predicate: ((((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) and FILE__PATH is not null) (type: boolean) Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) @@ -250,11 +250,11 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product -Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product -Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product -Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product -Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[223][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[225][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[221][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[227][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[219][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product PREHOOK: query: explain delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY @@ -275,27 +275,28 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Reducer 21 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 10 <- Reducer 23 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) Reducer 12 <- Reducer 11 (SIMPLE_EDGE) - Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) - Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) - Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) + Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 19 (XPROD_EDGE) + Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE) + Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 23 (XPROD_EDGE) Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Union 7 (CONTAINS) - Reducer 18 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 19 <- Map 1 (SIMPLE_EDGE) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE) + Reducer 18 <- Map 1 (SIMPLE_EDGE) + Reducer 19 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE) Reducer 20 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 22 <- Map 1 (SIMPLE_EDGE) Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 20 (XPROD_EDGE) - Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) - Reducer 5 <- Reducer 22 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) - Reducer 6 <- Reducer 23 (XPROD_EDGE), Reducer 5 (XPROD_EDGE), Union 7 (CONTAINS) - Reducer 8 <- Map 1 (XPROD_EDGE), Reducer 18 (XPROD_EDGE) - Reducer 9 <- Reducer 19 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) + Reducer 4 <- Reducer 22 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE) + Reducer 5 <- Reducer 24 (XPROD_EDGE), Reducer 4 (XPROD_EDGE) + Reducer 6 <- Reducer 12 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 8 <- Map 1 (XPROD_EDGE), Reducer 21 (XPROD_EDGE) + Reducer 9 <- Reducer 18 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -338,6 +339,18 @@ STAGE PLANS: Filter Operator predicate: (a <= 5) (type: boolean) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: a (type: int) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Select Operator Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator @@ -351,18 +364,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Group By Operator - keys: a (type: int) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count(), count(a) minReductionHashAggr: 0.4 @@ -374,14 +375,11 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Filter Operator - predicate: (c > 800) (type: boolean) - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - minReductionHashAggr: 0.8333333 + minReductionHashAggr: 0.4 mode: hash outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE @@ -390,6 +388,9 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) + Filter Operator + predicate: (c > 800) (type: boolean) + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator keys: c (type: int) minReductionHashAggr: 0.6666666 @@ -402,6 +403,19 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8333333 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Group By Operator aggregations: count(), count(c) minReductionHashAggr: 0.8333333 @@ -607,24 +621,6 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice Reducer 18 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 19 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -657,6 +653,19 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) + Reducer 19 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Reducer 2 Reduce Operator Tree: Merge Join Operator @@ -698,11 +707,6 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Reducer 22 Execution mode: vectorized Reduce Operator Tree: @@ -737,6 +741,24 @@ STAGE PLANS: Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) Reducer 23 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 24 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -763,38 +785,14 @@ STAGE PLANS: expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) Statistics: Num rows: 6 Data size: 1918 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col5 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean) Reducer 4 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col5 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col2 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - Reducer 5 Reduce Operator Tree: Merge Join Operator condition map: @@ -802,14 +800,14 @@ STAGE PLANS: keys: 0 _col2 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 - Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12 + Statistics: Num rows: 6 Data size: 1942 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 4 Data size: 1220 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) - Reducer 6 + Statistics: Num rows: 6 Data size: 1942 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col12 (type: boolean) + Reducer 5 Reduce Operator Tree: Merge Join Operator condition map: @@ -817,27 +815,48 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 - Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12, _col13, _col14 + Statistics: Num rows: 6 Data size: 2038 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 - Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col14 + Statistics: Num rows: 6 Data size: 2038 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) - Statistics: Num rows: 4 Data size: 1284 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null) or (_col14 is not null and (_col11 <> 0L)) or ((_col2 is null or (_col12 < _col11)) and null and (_col11 <> 0L) and _col14 is null)) is null or (((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) and ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)))) (type: boolean) + Statistics: Num rows: 6 Data size: 2038 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 6 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 4 Data size: 1204 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 2107 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice Reducer 8 Reduce Operator Tree: Merge Join Operator @@ -892,11 +911,11 @@ STAGE PLANS: Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[222][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 3' is a cross product -Warning: Shuffle Join MERGEJOIN[220][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 8' is a cross product -Warning: Shuffle Join MERGEJOIN[224][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 10' is a cross product -Warning: Shuffle Join MERGEJOIN[231][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 6' is a cross product -Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[223][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[225][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[221][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[227][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[219][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product Warning: Shuffle Join MERGEJOIN[229][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY @@ -1161,7 +1180,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) Statistics: Num rows: 2 Data size: 642 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) @@ -1593,7 +1612,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) Statistics: Num rows: 1 Data size: 321 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) @@ -1810,3 +1829,64 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice POSTHOOK: Output: hdfs://### HDFS PATH ### 0 +PREHOOK: query: drop table if exists tbl_ice_with_nulls +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice_with_nulls +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create table tbl_ice_with_nulls (id int, name string) stored by iceberg tblproperties('format-version'='2', 'write.delete.mode'='copy-on-write') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice_with_nulls +POSTHOOK: query: create table tbl_ice_with_nulls (id int, name string) stored by iceberg tblproperties('format-version'='2', 'write.delete.mode'='copy-on-write') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice_with_nulls +PREHOOK: query: insert into tbl_ice_with_nulls values +(1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(7, 'SDF'),(9, null),(8,'POIKL'),(10, 'YUIO') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice_with_nulls +POSTHOOK: query: insert into tbl_ice_with_nulls values +(1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(7, 'SDF'),(9, null),(8,'POIKL'),(10, 'YUIO') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice_with_nulls +Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[211][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product +Warning: Shuffle Join MERGEJOIN[213][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[214][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[209][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[215][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[216][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[207][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[217][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[218][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 16' is a cross product +PREHOOK: query: delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf') +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice_with_nulls +PREHOOK: Output: default@tbl_ice_with_nulls +POSTHOOK: query: delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice_with_nulls +POSTHOOK: Output: default@tbl_ice_with_nulls +PREHOOK: query: select * from tbl_ice_with_nulls order by id +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice_with_nulls +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice_with_nulls order by id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice_with_nulls +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 ABC +2 CBS +3 NULL +4 POPI +5 AQWR +6 POIU +7 SDF +8 POIKL +9 NULL diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out similarity index 87% rename from iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out rename to iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out index 4d80795bc156..4599a458520a 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out @@ -84,17 +84,17 @@ Stage-4 <-Reducer 2 [CONTAINS] File Output Operator [FS_46] table:{"name:":"default.ice01"} - Select Operator [SEL_44] (rows=2 width=295) + Select Operator [SEL_44] (rows=3 width=295) Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - Merge Join Operator [MERGEJOIN_43] (rows=2 width=295) + Merge Join Operator [MERGEJOIN_43] (rows=3 width=295) Conds:RS_57._col4=RS_63._col0(Left Semi),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] <-Map 1 [SIMPLE_EDGE] vectorized SHUFFLE [RS_57] PartitionCols:_col4 - Select Operator [SEL_55] (rows=2 width=295) + Select Operator [SEL_55] (rows=5 width=295) Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - Filter Operator [FIL_53] (rows=2 width=91) - predicate:((id <= 4) and (id <> 2) and FILE__PATH is not null) + Filter Operator [FIL_53] (rows=5 width=91) + predicate:((((id <= 4) and (id <> 2)) or ((id > 4) or (id = 2)) is null) and ((id <= 4) or (id <> 2) or ((id > 4) or (id = 2)) is null) and FILE__PATH is not null) TableScan [TS_0] (rows=7 width=78) default@ice01,ice01,Tbl:COMPLETE,Col:COMPLETE,Output:["id","name"] <-Reducer 4 [SIMPLE_EDGE] vectorized @@ -165,6 +165,35 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### {"added-data-files":"1","added-records":"9","added-files-size":"#FileSize#","changed-partition-count":"1","total-records":"9","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"0","total-position-deletes":"0","total-equality-deletes":"0"} {"added-position-delete-files":"1","added-delete-files":"1","added-files-size":"#FileSize#","added-position-deletes":"2","changed-partition-count":"1","total-records":"9","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"1","total-position-deletes":"2","total-equality-deletes":"0"} {"added-data-files":"1","deleted-data-files":"1","added-records":"3","deleted-records":"9","added-files-size":"#FileSize#","removed-files-size":"#FileSize#","changed-partition-count":"1","total-records":"3","total-files-size":"#FileSize#","total-data-files":"1","total-delete-files":"1","total-position-deletes":"2","total-equality-deletes":"0"} +PREHOOK: query: delete from ice01 where null +PREHOOK: type: QUERY +PREHOOK: Input: default@ice01 +PREHOOK: Output: default@ice01 +POSTHOOK: query: delete from ice01 where null +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice01 +POSTHOOK: Output: default@ice01 +PREHOOK: query: select * from ice01 +PREHOOK: type: QUERY +PREHOOK: Input: default@ice01 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice01 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice01 +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 ABC +3 NULL +4 POPI +PREHOOK: query: delete from ice01 where not null +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Input: default@ice01 +PREHOOK: Output: default@ice01 +POSTHOOK: query: delete from ice01 where not null +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Input: default@ice01 +POSTHOOK: Output: default@ice01 PREHOOK: query: select * from ice01 PREHOOK: type: QUERY PREHOOK: Input: default@ice01 @@ -176,12 +205,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 1 ABC 3 NULL 4 POPI -PREHOOK: query: delete from ice01 where name=null +PREHOOK: query: delete from ice01 where name = null PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Input: default@ice01 PREHOOK: Output: default@ice01 -POSTHOOK: query: delete from ice01 where name=null +POSTHOOK: query: delete from ice01 where name = null POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Input: default@ice01 @@ -197,12 +226,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 1 ABC 3 NULL 4 POPI -PREHOOK: query: delete from ice01 where name!=null +PREHOOK: query: delete from ice01 where name != null PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Input: default@ice01 PREHOOK: Output: default@ice01 -POSTHOOK: query: delete from ice01 where name!=null +POSTHOOK: query: delete from ice01 where name != null POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Input: default@ice01 diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out index 3672e210f2e1..85b3b2a7b56d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out @@ -154,3 +154,58 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@tbl_ice POSTHOOK: Output: hdfs://### HDFS PATH ### 0 +PREHOOK: query: drop table if exists tbl_ice_with_nulls +PREHOOK: type: DROPTABLE +PREHOOK: Output: database:default +POSTHOOK: query: drop table if exists tbl_ice_with_nulls +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: database:default +PREHOOK: query: create table tbl_ice_with_nulls (id int, name string) stored by iceberg tblproperties('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl_ice_with_nulls +POSTHOOK: query: create table tbl_ice_with_nulls (id int, name string) stored by iceberg tblproperties('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl_ice_with_nulls +PREHOOK: query: insert into tbl_ice_with_nulls values +(1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(7, 'SDF'),(9, null),(8,'POIKL'),(10, 'YUIO') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@tbl_ice_with_nulls +POSTHOOK: query: insert into tbl_ice_with_nulls values +(1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(7, 'SDF'),(9, null),(8,'POIKL'),(10, 'YUIO') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@tbl_ice_with_nulls +Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[62][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product +PREHOOK: query: delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf') +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice_with_nulls +PREHOOK: Output: default@tbl_ice_with_nulls +POSTHOOK: query: delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice_with_nulls +POSTHOOK: Output: default@tbl_ice_with_nulls +PREHOOK: query: select * from tbl_ice_with_nulls order by id +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl_ice_with_nulls +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from tbl_ice_with_nulls order by id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl_ice_with_nulls +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 ABC +2 CBS +3 NULL +4 POPI +5 AQWR +6 POIU +7 SDF +8 POIKL +9 NULL diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out index 308d2a8d62a4..f22d0bb16200 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out @@ -200,7 +200,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col4 IS DISTINCT FROM _col7) (type: boolean) + predicate: ((_col4 <> _col7) or _col4 is null or _col7 is null) (type: boolean) Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out index fb70b3e6af98..0a4ba96cea21 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out @@ -74,13 +74,13 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE), Union 3 (CONTAINS) - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE), Union 3 (CONTAINS) - Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 11 <- Map 10 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE), Union 3 (CONTAINS) + Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE), Union 3 (CONTAINS) - Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) Reducer 7 <- Reducer 6 (SIMPLE_EDGE) - Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE) Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 3 (CONTAINS) #### A masked pattern was here #### Vertices: @@ -130,27 +130,6 @@ STAGE PLANS: Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized Map 10 - Map Operator Tree: - TableScan - alias: target_ice - filterExpr: (a <= 100) (type: boolean) - Statistics: Num rows: 4 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (a <= 100) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), c (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col4 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col4 (type: int) - Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int) - Execution mode: vectorized - Map 11 Map Operator Tree: TableScan alias: target_ice @@ -166,8 +145,31 @@ STAGE PLANS: Map-reduce partition columns: _col4 (type: int) Statistics: Num rows: 4 Data size: 832 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + Execution mode: vectorized + Map 12 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: string), _col2 (type: int) + Execution mode: vectorized + Map 13 + Map Operator Tree: + TableScan + alias: target_ice + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (FILE__PATH is not null and a is not null) (type: boolean) + predicate: FILE__PATH is not null (type: boolean) Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), b (type: string), c (type: int) @@ -180,6 +182,20 @@ STAGE PLANS: Map-reduce partition columns: _col4 (type: int) Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), _col6 (type: int) + Filter Operator + predicate: (a <= 100) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), a (type: int), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col4 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col4 (type: int) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: int) Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE @@ -209,24 +225,7 @@ STAGE PLANS: Statistics: Num rows: 4 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col4 (type: string), _col5 (type: int) Execution mode: vectorized - Map 13 - Map Operator Tree: - TableScan - alias: src - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: a (type: int), b (type: string), c (type: int) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: string), _col2 (type: int) - Execution mode: vectorized - Reducer 12 + Reducer 11 Reduce Operator Tree: Merge Join Operator condition map: @@ -284,7 +283,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col4 IS DISTINCT FROM _col7) (type: boolean) + predicate: ((_col4 <> _col7) or ((_col4 = _col7) or _col4 is null) is null) (type: boolean) Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) @@ -627,7 +626,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 12 Data size: 3616 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col4 IS DISTINCT FROM _col7) (type: boolean) + predicate: ((_col4 <> _col7) or _col4 is null or _col7 is null) (type: boolean) Statistics: Num rows: 12 Data size: 3616 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out index 724b0631f95a..579504cb11ed 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out @@ -71,10 +71,10 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - filterExpr: ((a <> 22) and (b <> 'one') and (b <> 'four')) (type: boolean) + filterExpr: (((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) (type: boolean) Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + predicate: ((((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) and FILE__PATH is not null) (type: boolean) Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) @@ -287,14 +287,14 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice -Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 10' is a cross product -Warning: Shuffle Join MERGEJOIN[283][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 14' is a cross product -Warning: Shuffle Join MERGEJOIN[288][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 16' is a cross product -Warning: Shuffle Join MERGEJOIN[303][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 13' is a cross product -Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product -Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product +Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 12' is a cross product +Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 16' is a cross product +Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product +Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product PREHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -314,36 +314,34 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Reducer 30 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) - Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE) - Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) - Reducer 13 <- Reducer 12 (XPROD_EDGE), Reducer 25 (XPROD_EDGE), Union 7 (CONTAINS) - Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 32 (XPROD_EDGE) - Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) - Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 24 (XPROD_EDGE) - Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) + Reducer 10 <- Reducer 29 (XPROD_EDGE), Reducer 9 (XPROD_EDGE) + Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE) + Reducer 12 <- Reducer 11 (XPROD_EDGE), Reducer 26 (XPROD_EDGE) + Reducer 13 <- Reducer 12 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 28 (XPROD_EDGE) + Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE) + Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 23 (XPROD_EDGE) + Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE) Reducer 18 <- Reducer 17 (SIMPLE_EDGE) - Reducer 19 <- Map 1 (XPROD_EDGE), Reducer 31 (XPROD_EDGE) + Reducer 19 <- Map 1 (XPROD_EDGE), Reducer 30 (XPROD_EDGE) Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 28 (XPROD_EDGE) - Reducer 20 <- Reducer 19 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) - Reducer 21 <- Reducer 20 (XPROD_EDGE), Reducer 26 (XPROD_EDGE) - Reducer 22 <- Reducer 21 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE), Union 7 (CONTAINS) - Reducer 23 <- Map 1 (SIMPLE_EDGE) - Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 20 <- Reducer 19 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE) + Reducer 21 <- Reducer 20 (XPROD_EDGE), Reducer 23 (XPROD_EDGE) + Reducer 22 <- Reducer 21 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 24 <- Map 1 (SIMPLE_EDGE) Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 26 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 27 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 27 <- Map 1 (SIMPLE_EDGE) Reducer 28 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 29 <- Map 1 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) + Reducer 29 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE) Reducer 30 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 31 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 32 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 4 <- Reducer 27 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) - Reducer 5 <- Reducer 23 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 4 <- Reducer 25 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) + Reducer 5 <- Reducer 24 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS) Reducer 8 <- Union 7 (SIMPLE_EDGE) - Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE) + Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -395,6 +393,19 @@ STAGE PLANS: Filter Operator predicate: (c > 800) (type: boolean) Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Group By Operator keys: c (type: int) minReductionHashAggr: 0.7777778 @@ -431,48 +442,9 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Select Operator - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - minReductionHashAggr: 0.8888889 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Select Operator - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - minReductionHashAggr: 0.8888889 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Filter Operator predicate: (a <= 5) (type: boolean) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Group By Operator keys: a (type: int) minReductionHashAggr: 0.4 @@ -485,17 +457,6 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(a) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator @@ -509,6 +470,17 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) + Group By Operator + aggregations: count(), count(a) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator @@ -537,38 +509,14 @@ STAGE PLANS: expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col5 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean) Reducer 11 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col5 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col2 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - Reducer 12 Reduce Operator Tree: Merge Join Operator condition map: @@ -576,14 +524,14 @@ STAGE PLANS: keys: 0 _col2 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 - Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12 + Statistics: Num rows: 9 Data size: 2913 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) - Reducer 13 + Statistics: Num rows: 9 Data size: 2913 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col12 (type: boolean) + Reducer 12 Reduce Operator Tree: Merge Join Operator condition map: @@ -591,26 +539,47 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 - Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12, _col13, _col14 + Statistics: Num rows: 9 Data size: 3057 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 - Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col14 + Statistics: Num rows: 9 Data size: 3057 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) - Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null) or (_col14 is not null and (_col11 <> 0L)) or ((_col2 is null or (_col12 < _col11)) and null and (_col11 <> 0L) and _col14 is null)) is null or (((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) and ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)))) (type: boolean) + Statistics: Num rows: 9 Data size: 3057 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) - Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 13 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: iceberg_bucket(_col4, 16) (type: int), iceberg_truncate(_col5, 3) (type: string) + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 14 Reduce Operator Tree: Merge Join Operator @@ -816,6 +785,24 @@ STAGE PLANS: Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 23 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 24 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -855,7 +842,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 24 + Reducer 25 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -868,7 +855,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 25 + Reducer 26 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -881,46 +868,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 26 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Reducer 27 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 28 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 29 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -960,22 +908,25 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 3 + Reducer 28 + Execution mode: vectorized Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 - Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) - Reducer 30 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 29 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -988,20 +939,22 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 31 - Execution mode: vectorized + Reducer 3 Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 32 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) + Reducer 30 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -1141,14 +1094,14 @@ STAGE PLANS: Stats Work Basic Stats Work: -Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 10' is a cross product -Warning: Shuffle Join MERGEJOIN[283][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 14' is a cross product -Warning: Shuffle Join MERGEJOIN[288][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 16' is a cross product -Warning: Shuffle Join MERGEJOIN[303][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 13' is a cross product -Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product -Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product +Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 10' is a cross product +Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 12' is a cross product +Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 14' is a cross product +Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 16' is a cross product +Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product +Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -1496,7 +1449,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null) (type: boolean) Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) @@ -2100,7 +2053,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null) (type: boolean) Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out index edbb1b2523e2..2242fdae23a5 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out @@ -71,10 +71,10 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - filterExpr: ((a <> 22) and (b <> 'one') and (b <> 'four')) (type: boolean) + filterExpr: (((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) (type: boolean) Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((a <> 22) and (b <> 'one') and (b <> 'four') and FILE__PATH is not null) (type: boolean) + predicate: ((((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) and FILE__PATH is not null) (type: boolean) Statistics: Num rows: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint) @@ -274,14 +274,14 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice -Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product -Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 13' is a cross product -Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 15' is a cross product -Warning: Shuffle Join MERGEJOIN[301][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 12' is a cross product -Warning: Shuffle Join MERGEJOIN[278][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 18' is a cross product -Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 20' is a cross product +Warning: Shuffle Join MERGEJOIN[285][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 18' is a cross product +Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 20' is a cross product PREHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -301,35 +301,33 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 10 <- Reducer 17 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) - Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) - Reducer 12 <- Reducer 11 (XPROD_EDGE), Reducer 24 (XPROD_EDGE), Union 7 (CONTAINS) - Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 31 (XPROD_EDGE) - Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) - Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 23 (XPROD_EDGE) - Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE) + Reducer 10 <- Reducer 23 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 25 (XPROD_EDGE) + Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 13 <- Map 1 (XPROD_EDGE), Reducer 27 (XPROD_EDGE) + Reducer 14 <- Reducer 13 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE) + Reducer 15 <- Reducer 14 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) + Reducer 16 <- Reducer 15 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE) Reducer 17 <- Reducer 16 (SIMPLE_EDGE) - Reducer 18 <- Map 1 (XPROD_EDGE), Reducer 30 (XPROD_EDGE) - Reducer 19 <- Reducer 18 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) + Reducer 18 <- Map 1 (XPROD_EDGE), Reducer 29 (XPROD_EDGE) + Reducer 19 <- Reducer 18 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE) Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 27 (XPROD_EDGE) - Reducer 20 <- Reducer 19 (XPROD_EDGE), Reducer 25 (XPROD_EDGE) - Reducer 21 <- Reducer 20 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE), Union 7 (CONTAINS) - Reducer 22 <- Map 1 (SIMPLE_EDGE) - Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 20 <- Reducer 19 (XPROD_EDGE), Reducer 22 (XPROD_EDGE) + Reducer 21 <- Reducer 20 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE), Union 7 (CONTAINS) + Reducer 22 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 23 <- Map 1 (SIMPLE_EDGE) Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 26 <- Map 1 (CUSTOM_SIMPLE_EDGE) + Reducer 26 <- Map 1 (SIMPLE_EDGE) Reducer 27 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 28 <- Map 1 (SIMPLE_EDGE) + Reducer 28 <- Map 1 (CUSTOM_SIMPLE_EDGE) Reducer 29 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) - Reducer 30 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 31 <- Map 1 (CUSTOM_SIMPLE_EDGE) - Reducer 4 <- Reducer 26 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) - Reducer 5 <- Reducer 22 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE) + Reducer 4 <- Reducer 24 (XPROD_EDGE), Reducer 3 (XPROD_EDGE) + Reducer 5 <- Reducer 23 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE) Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS) - Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE) - Reducer 9 <- Reducer 29 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) + Reducer 8 <- Map 1 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE) + Reducer 9 <- Reducer 28 (XPROD_EDGE), Reducer 8 (XPROD_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -381,6 +379,19 @@ STAGE PLANS: Filter Operator predicate: (c > 800) (type: boolean) Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Group By Operator keys: c (type: int) minReductionHashAggr: 0.7777778 @@ -417,48 +428,9 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Select Operator - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - minReductionHashAggr: 0.8888889 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Select Operator - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - minReductionHashAggr: 0.8888889 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Filter Operator predicate: (a <= 5) (type: boolean) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count() - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Group By Operator keys: a (type: int) minReductionHashAggr: 0.4 @@ -471,17 +443,6 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: count(), count(a) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator @@ -495,6 +456,17 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) + Group By Operator + aggregations: count(), count(a) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator @@ -510,23 +482,6 @@ STAGE PLANS: value expressions: _col0 (type: bigint) Execution mode: vectorized Reducer 10 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Semi Join 0 to 1 - keys: - 0 _col5 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col2 (type: int) - null sort order: z - sort order: + - Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - Reducer 11 Reduce Operator Tree: Merge Join Operator condition map: @@ -534,14 +489,14 @@ STAGE PLANS: keys: 0 _col2 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12 - Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12 + Statistics: Num rows: 9 Data size: 2913 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 1830 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col12 (type: boolean) - Reducer 12 + Statistics: Num rows: 9 Data size: 2913 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col12 (type: boolean) + Reducer 11 Reduce Operator Tree: Merge Join Operator condition map: @@ -549,27 +504,48 @@ STAGE PLANS: keys: 0 1 - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col12, _col13, _col14 - Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col12, _col13, _col14 + Statistics: Num rows: 9 Data size: 3057 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col11, _col12, _col14 - Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean), _col13 (type: bigint), _col14 (type: bigint), _col12 (type: boolean) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col14 + Statistics: Num rows: 9 Data size: 3057 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)) (type: boolean) - Statistics: Num rows: 6 Data size: 1926 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null) or (_col14 is not null and (_col11 <> 0L)) or ((_col2 is null or (_col12 < _col11)) and null and (_col11 <> 0L) and _col14 is null)) is null or (((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) and ((_col11 = 0L) or (_col14 is null and (_col12 >= _col11) and _col2 is not null)))) (type: boolean) + Statistics: Num rows: 9 Data size: 3057 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.tbl_ice + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col5 (type: string) + null sort order: z + sort order: + + Map-reduce partition columns: _col5 (type: string) + Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + Reducer 12 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col5 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col0 (type: int), _col1 (type: string), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 6 Data size: 1806 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 19 Data size: 5791 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.tbl_ice Reducer 13 Reduce Operator Tree: Merge Join Operator @@ -776,6 +752,24 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.tbl_ice Reducer 22 + Execution mode: vectorized + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 23 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -815,7 +809,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 23 + Reducer 24 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -828,7 +822,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) - Reducer 24 + Reducer 25 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -841,46 +835,7 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 25 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) Reducer 26 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 27 - Execution mode: vectorized - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 28 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -920,48 +875,38 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: boolean) - Reducer 29 + Reducer 27 Execution mode: vectorized Reduce Operator Tree: Group By Operator - aggregations: count(VALUE._col0), count(VALUE._col1) + aggregations: count(VALUE._col0) mode: mergepartial - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint), _col1 (type: bigint) - Reducer 3 - Reduce Operator Tree: - Merge Join Operator - condition map: - Left Outer Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 - Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) - Reducer 30 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reducer 28 Execution mode: vectorized Reduce Operator Tree: Group By Operator - aggregations: count(VALUE._col0) + aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 31 + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint) + Reducer 29 Execution mode: vectorized Reduce Operator Tree: Group By Operator @@ -974,6 +919,21 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Merge Join Operator + condition map: + Left Outer Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col8 + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 9 Data size: 2733 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col8 (type: boolean) Reducer 4 Reduce Operator Tree: Merge Join Operator @@ -1081,20 +1041,13 @@ STAGE PLANS: expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col8 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + Reduce Output Operator + key expressions: _col2 (type: int) + null sort order: z + sort order: + + Map-reduce partition columns: _col2 (type: int) Statistics: Num rows: 9 Data size: 2877 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col5 (type: string) - null sort order: z - sort order: + - Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 9 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: bigint), _col10 (type: boolean) Union 7 Vertex: Union 7 @@ -1115,14 +1068,14 @@ STAGE PLANS: Stats Work Basic Stats Work: -Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[280][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 9' is a cross product -Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 13' is a cross product -Warning: Shuffle Join MERGEJOIN[286][tables = [$hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 15' is a cross product -Warning: Shuffle Join MERGEJOIN[301][tables = [$hdt$_1, $hdt$_2, $hdt$_0, $hdt$_3]] in Stage 'Reducer 12' is a cross product -Warning: Shuffle Join MERGEJOIN[278][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 18' is a cross product -Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 20' is a cross product +Warning: Shuffle Join MERGEJOIN[285][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 9' is a cross product +Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 11' is a cross product +Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 13' is a cross product +Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 15' is a cross product +Warning: Shuffle Join MERGEJOIN[279][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 18' is a cross product +Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 20' is a cross product PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -1418,7 +1371,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null) (type: boolean) Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) @@ -2034,7 +1987,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10 Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null)) (type: boolean) + predicate: ((_col7 = 0L) or (_col10 is null and (_col8 >= _col7) and _col0 is not null) or ((_col10 is not null and (_col7 <> 0L)) or ((_col0 is null or (_col8 < _col7)) and null and (_col7 <> 0L) and _col10 is null)) is null) (type: boolean) Statistics: Num rows: 10 Data size: 3178 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java index 5c4366b5ba40..c1396199c8f2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteDeleteRewriter.java @@ -65,7 +65,8 @@ public ParseUtils.ReparseResult rewrite(Context context, DeleteStatement deleteB sqlGenerator.append(sqlGenerator.getTargetTableFullName()); // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. - sqlGenerator.append("\nwhere NOT (").append(whereClause).append(")"); + sqlGenerator.append("\nwhere "); + sqlGenerator.append("( NOT(%s) OR (%s) IS NULL )".replace("%s", whereClause)); sqlGenerator.append("\n"); // Add the file path filter that matches the delete condition. sqlGenerator.append("AND ").append(filePathCol); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java index c99324a8be72..f422d5514bbd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java @@ -227,14 +227,14 @@ protected void handleWhenMatchedDelete(String onClauseAsString, String extraPred String whereClauseStr = columnRefsFunc.apply(whereClause.toString()); String filePathCol = HiveUtils.unparseIdentifier(TARGET_PREFIX + VirtualColumn.FILE_PATH.getName(), conf); - sqlGenerator.append("\n").indent(); - sqlGenerator.append("NOT(").append(whereClauseStr.replace("=","<=>")); - if (isNotBlank(onClausePredicate)) { - sqlGenerator.append(" OR "); - sqlGenerator.append(columnRefsFunc.apply(mergeStatement.getOnClausePredicate())); + whereClause.append(" OR ").append(onClausePredicate); } - sqlGenerator.append(")\n").indent(); + sqlGenerator.append("\n").indent(); + sqlGenerator.append("( NOT(%s) OR (%s) IS NULL )".replace("%s", columnRefsFunc.apply( + whereClause.toString()))); + + sqlGenerator.append("\n").indent(); // Add the file path filter that matches the delete condition. sqlGenerator.append("AND ").append(filePathCol); sqlGenerator.append(" IN ( select ").append(filePathCol).append(" from t )"); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java index bf88c1da3d47..5279f6361786 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteUpdateRewriter.java @@ -116,7 +116,8 @@ public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateB sqlGenerator.append(" from "); sqlGenerator.appendTargetTableName(); // Add the inverted where clause, since we want to hold the records which doesn't satisfy the condition. - sqlGenerator.append("\nwhere NOT (").append(whereClause).append(")"); + sqlGenerator.append("\nwhere "); + sqlGenerator.append("( NOT(%s) OR (%s) IS NULL )".replace("%s", whereClause)); sqlGenerator.append("\n").indent(); // Add the file path filter that matches the delete condition. sqlGenerator.append("AND ").append(filePathCol); From 8d70977ff0d668cdd62f9dd8ee49675623b823a7 Mon Sep 17 00:00:00 2001 From: Shubham Sharma Date: Tue, 23 Jan 2024 19:54:26 +0530 Subject: [PATCH 151/179] HIVE-27942: Missing aux jar errors during LLAP launch (#4927)(Shubham Sharma, reviewed by Akshat Mathur, Ayush Saxena, Butao Zhang) --- .../hadoop/hive/llap/cli/service/AsyncTaskCopyAuxJars.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyAuxJars.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyAuxJars.java index d185d1f58f8e..675f45284af1 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyAuxJars.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/service/AsyncTaskCopyAuxJars.java @@ -45,8 +45,8 @@ class AsyncTaskCopyAuxJars implements Callable { private static final String[] DEFAULT_AUX_CLASSES = new String[] {"org.apache.hive.hcatalog.data.JsonSerDe", "org.apache.hadoop.hive.druid.DruidStorageHandler", - "org.apache.hive.storage.jdbc.JdbcStorageHandler", "org.apache.commons.dbcp.BasicDataSourceFactory", - "org.apache.commons.pool.impl.GenericObjectPool", "org.apache.hadoop.hive.kafka.KafkaStorageHandler", + "org.apache.hive.storage.jdbc.JdbcStorageHandler", "org.apache.commons.dbcp2.BasicDataSourceFactory", + "org.apache.commons.pool2.impl.GenericObjectPool", "org.apache.hadoop.hive.kafka.KafkaStorageHandler", "org.apache.hadoop.hive.kudu.KuduStorageHandler"}; private static final String HBASE_SERDE_CLASS = "org.apache.hadoop.hive.hbase.HBaseSerDe"; From 055f43c04a072e80ed6236fccb7023f4e0220f1a Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Wed, 24 Jan 2024 16:50:45 +0800 Subject: [PATCH 152/179] HIVE-28005: Remove upgrade-acid module (#5017). (Butao Zhang, reviewed by Ayush Saxena, Denys Kuzmenko) --- packaging/src/main/assembly/src.xml | 1 - pom.xml | 1 - upgrade-acid/pom.xml | 68 -- upgrade-acid/pre-upgrade/pom.xml | 360 --------- .../upgrade/acid/CloseableThreadLocal.java | 61 -- .../hive/upgrade/acid/CompactTablesState.java | 59 -- .../hive/upgrade/acid/CompactionMetaInfo.java | 68 -- .../NamedForkJoinWorkerThreadFactory.java | 40 - .../hive/upgrade/acid/PreUpgradeTool.java | 750 ------------------ .../hadoop/hive/upgrade/acid/RunOptions.java | 110 --- .../acid/TestCloseableThreadLocal.java | 86 -- .../hive/upgrade/acid/TestPreUpgradeTool.java | 495 ------------ .../hive/upgrade/acid/TestRunOptions.java | 67 -- 13 files changed, 2166 deletions(-) delete mode 100644 upgrade-acid/pom.xml delete mode 100644 upgrade-acid/pre-upgrade/pom.xml delete mode 100644 upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CloseableThreadLocal.java delete mode 100644 upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactTablesState.java delete mode 100644 upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactionMetaInfo.java delete mode 100644 upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/NamedForkJoinWorkerThreadFactory.java delete mode 100644 upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/PreUpgradeTool.java delete mode 100644 upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/RunOptions.java delete mode 100644 upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestCloseableThreadLocal.java delete mode 100644 upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestPreUpgradeTool.java delete mode 100644 upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestRunOptions.java diff --git a/packaging/src/main/assembly/src.xml b/packaging/src/main/assembly/src.xml index 6cd94585cbf9..4d4551798ac0 100644 --- a/packaging/src/main/assembly/src.xml +++ b/packaging/src/main/assembly/src.xml @@ -110,7 +110,6 @@ standalone-metastore/pom.xml streaming/**/* testutils/**/* - upgrade-acid/**/* vector-code-gen/**/* kudu-handler/**/* parser/**/* diff --git a/pom.xml b/pom.xml index b6959e10ceef..1317235e40c8 100644 --- a/pom.xml +++ b/pom.xml @@ -58,7 +58,6 @@ testutils packaging standalone-metastore - upgrade-acid kafka-handler diff --git a/upgrade-acid/pom.xml b/upgrade-acid/pom.xml deleted file mode 100644 index d3fb9b28545f..000000000000 --- a/upgrade-acid/pom.xml +++ /dev/null @@ -1,68 +0,0 @@ - - - - 4.0.0 - - - org.apache - apache - 23 - - - org.apache.hive - hive-upgrade-acid - 4.0.0-beta-2-SNAPSHOT - Hive Upgrade Acid - pom - - - UTF-8 - 1.8 - 1.8 - false - ${settings.localRepository} - .. - - 1.0b3 - ${basedir}/${hive.path.to.root}/checkstyle - 2.17 - 5.6.2 - 5.6.2 - 2.16.0 - 3.0.0-M4 - - - pre-upgrade - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - ${maven.checkstyle.plugin.version} - - ${checkstyle.conf.dir}/checkstyle.xml - config_loc=${checkstyle.conf.dir} - true - - - - org.codehaus.mojo - versions-maven-plugin - ${maven.versions.plugin.version} - - - - diff --git a/upgrade-acid/pre-upgrade/pom.xml b/upgrade-acid/pre-upgrade/pom.xml deleted file mode 100644 index 989205bd80aa..000000000000 --- a/upgrade-acid/pre-upgrade/pom.xml +++ /dev/null @@ -1,360 +0,0 @@ - - - - - org.apache.hive - hive-upgrade-acid - 4.0.0-beta-2-SNAPSHOT - ../pom.xml - - 4.0.0 - - hive-pre-upgrade - Hive Pre Upgrade Acid - jar - - ../.. - - ${project.build.directory}/testconf - file:// - ${project.basedir}/src/test/resources - ${project.build.directory}/tmp - ${project.build.directory}/warehouse - file:// - 1 - true - 2.3.3 - 2.7.2 - - - - - commons-cli - commons-cli - 1.2 - provided - - - org.apache.hive - hive-metastore - ${hdp.hive.version} - provided - - - tomcat - jasper-compiler - - - tomcat - jasper-runtime - - - org.apache.zookeeper - zookeeper - - - org.apache.curator - curator-framework - - - jdk.tools - jdk.tools - - - - - org.apache.hive - hive-exec - ${hdp.hive.version} - provided - - - org.codehaus.groovy - groovy-all - - - org.apache.zookeeper - zookeeper - - - org.apache.curator - curator-framework - - - org.pentaho - pentaho-aggdesigner-algorithm - - - - - org.apache.hadoop - hadoop-common - ${hdp.hadoop.version} - provided - - - commons-beanutils - commons-beanutils - - - org.apache.zookeeper - zookeeper - - - org.apache.curator - curator-client - - - org.apache.curator - curator-recipes - - - org.apache.curator - curator-framework - - - - - - org.apache.hadoop - hadoop-mapreduce-client-common - 2.7.2 - provided - - - org.apache.zookeeper - zookeeper - - - io.netty - netty - - - - - org.apache.orc - orc-core - 1.3.3 - provided - - - org.junit.jupiter - junit-jupiter-engine - ${junit.jupiter.version} - test - - - org.junit.vintage - junit-vintage-engine - ${junit.vintage.version} - test - - - - - - ${basedir}/src/main/resources - - package.jdo - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - ant-contrib - ant-contrib - ${ant.contrib.version} - - - ant - ant - - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - ${maven.checkstyle.plugin.version} - - - org.codehaus.mojo - exec-maven-plugin - ${maven.exec.plugin.version} - - - org.apache.maven.plugins - maven-surefire-plugin - ${maven.surefire.plugin.version} - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - setup-test-dirs - process-test-resources - - run - - - - - - - - - - - - - - - - - - setup-metastore-scripts - process-test-resources - - run - - - - - - - - - - - - - - - - - - org.apache.maven.plugins - maven-failsafe-plugin - - - - integration-test - verify - - - - - true - false - -Xmx2048m - false - - true - ${test.tmp.dir} - ${test.tmp.dir} - true - - - ${log4j.conf.dir} - - ${skipITests} - - - - - org.apache.maven.plugins - maven-surefire-plugin - - true - false - ${test.forkcount} - -Xmx2048m - false - - ${project.build.directory} - true - ${derby.version} - ${test.tmp.dir}/derby.log - - ${test.log4j.scheme}${test.conf.dir}/hive-log4j2.properties - true - ${test.tmp.dir} - - jdbc:derby:${test.tmp.dir}/junit_metastore_db;create=true - false - ${test.tmp.dir} - ${test.warehouse.scheme}${test.warehouse.dir} - - - - ${log4j.conf.dir} - ${test.conf.dir} - - ${test.conf.dir}/conf - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - diff --git a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CloseableThreadLocal.java b/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CloseableThreadLocal.java deleted file mode 100644 index fbe0a80d4883..000000000000 --- a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CloseableThreadLocal.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Consumer; -import java.util.function.Supplier; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class has similar functionality as java.lang.ThreadLocal. - * Plus it provides a close function to clean up unmanaged resources in all threads where the resource was initialized. - * @param - type of resource - */ -public class CloseableThreadLocal { - - private static final Logger LOG = LoggerFactory.getLogger(CloseableThreadLocal.class); - - private final ConcurrentHashMap threadLocalMap; - private final Supplier initialValue; - private final Consumer closeFunction; - - public CloseableThreadLocal(Supplier initialValue, Consumer closeFunction, int poolSize) { - this.initialValue = initialValue; - threadLocalMap = new ConcurrentHashMap<>(poolSize); - this.closeFunction = closeFunction; - } - - public T get() { - return threadLocalMap.computeIfAbsent(Thread.currentThread(), thread -> initialValue.get()); - } - - public void close() { - threadLocalMap.values().forEach(this::closeQuietly); - } - - private void closeQuietly(T resource) { - try { - closeFunction.accept(resource); - } catch (Exception e) { - LOG.warn("Error while closing resource.", e); - } - } -} diff --git a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactTablesState.java b/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactTablesState.java deleted file mode 100644 index beb934c83eb8..000000000000 --- a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactTablesState.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import static java.util.Collections.emptyList; - -import java.util.ArrayList; -import java.util.List; - -/** - * Store result of database and table scan: compaction commands and meta info. - */ -public final class CompactTablesState { - - public static CompactTablesState empty() { - return new CompactTablesState(emptyList(), new CompactionMetaInfo()); - } - - public static CompactTablesState compactions(List compactionCommands, CompactionMetaInfo compactionMetaInfo) { - return new CompactTablesState(compactionCommands, compactionMetaInfo); - } - - private final List compactionCommands; - private final CompactionMetaInfo compactionMetaInfo; - - private CompactTablesState(List compactionCommands, CompactionMetaInfo compactionMetaInfo) { - this.compactionCommands = compactionCommands; - this.compactionMetaInfo = compactionMetaInfo; - } - - public List getCompactionCommands() { - return compactionCommands; - } - - public CompactionMetaInfo getMetaInfo() { - return compactionMetaInfo; - } - - public CompactTablesState merge(CompactTablesState other) { - List compactionCommands = new ArrayList<>(this.compactionCommands); - compactionCommands.addAll(other.compactionCommands); - return new CompactTablesState(compactionCommands, this.compactionMetaInfo.merge(other.compactionMetaInfo)); - } -} diff --git a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactionMetaInfo.java b/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactionMetaInfo.java deleted file mode 100644 index 72b4ec63a9e0..000000000000 --- a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/CompactionMetaInfo.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import java.util.HashSet; -import java.util.Set; - -/** - * Store result of compaction calls. - */ -public class CompactionMetaInfo { - /** - * total number of bytes to be compacted across all compaction commands. - */ - private long numberOfBytes; - /** - * IDs of compactions launched by this utility. - */ - private final Set compactionIds; - - public CompactionMetaInfo() { - compactionIds = new HashSet<>(); - numberOfBytes = 0; - } - - private CompactionMetaInfo(Set initialCompactionIds, long initialNumberOfBytes) { - this.compactionIds = new HashSet<>(initialCompactionIds); - numberOfBytes = initialNumberOfBytes; - } - - public CompactionMetaInfo merge(CompactionMetaInfo other) { - CompactionMetaInfo result = new CompactionMetaInfo(this.compactionIds, this.numberOfBytes); - result.numberOfBytes += other.numberOfBytes; - result.compactionIds.addAll(other.compactionIds); - return result; - } - - public long getNumberOfBytes() { - return numberOfBytes; - } - - public void addBytes(long bytes) { - numberOfBytes += bytes; - } - - public Set getCompactionIds() { - return compactionIds; - } - - public void addCompactionId(long compactionId) { - compactionIds.add(compactionId); - } -} diff --git a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/NamedForkJoinWorkerThreadFactory.java b/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/NamedForkJoinWorkerThreadFactory.java deleted file mode 100644 index 2b95f7be961d..000000000000 --- a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/NamedForkJoinWorkerThreadFactory.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.ForkJoinWorkerThread; - -/** - * This class allows specifying a prefix for ForkJoinPool thread names. - */ -public class NamedForkJoinWorkerThreadFactory implements ForkJoinPool.ForkJoinWorkerThreadFactory { - - NamedForkJoinWorkerThreadFactory(String namePrefix) { - this.namePrefix = namePrefix; - } - - private final String namePrefix; - - @Override - public ForkJoinWorkerThread newThread(ForkJoinPool pool) { - ForkJoinWorkerThread worker = ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool); - worker.setName(namePrefix + worker.getName()); - return worker; - } -} diff --git a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/PreUpgradeTool.java b/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/PreUpgradeTool.java deleted file mode 100644 index b72b236b4677..000000000000 --- a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/PreUpgradeTool.java +++ /dev/null @@ -1,750 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import static org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.escapeSQLString; - -import java.io.FileWriter; -import java.io.IOException; -import java.io.PrintWriter; -import java.nio.ByteBuffer; -import java.nio.charset.CharacterCodingException; -import java.nio.charset.Charset; -import java.nio.charset.CharsetDecoder; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.ForkJoinPool; -import java.util.stream.Collectors; - -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.GnuParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hive.common.ValidTxnList; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaHook; -import org.apache.hadoop.hive.metastore.HiveMetaHookLoader; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient; -import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.CompactionResponse; -import org.apache.hadoop.hive.metastore.api.CompactionType; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.metastore.txn.TxnStore; -import org.apache.hadoop.hive.metastore.txn.TxnUtils; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.orc.OrcFile; -import org.apache.hadoop.hive.ql.io.orc.Reader; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.shims.HadoopShims; -import org.apache.hadoop.security.AccessControlException; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.common.util.HiveVersionInfo; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.annotations.VisibleForTesting; - -/** - * This utility is designed to help with upgrading Hive 2.x to Hive 3.0. On-disk layout for - * transactional tables has changed in 3.0 and require pre-processing before upgrade to ensure - * they are readable by Hive 3.0. Some transactional tables (identified by this utility) require - * Major compaction to be run on them before upgrading to 3.0. Once this compaction starts, no - * more update/delete/merge statements may be executed on these tables until upgrade is finished. - * - * Additionally, a new type of transactional tables was added in 3.0 - insert-only tables. These - * tables support ACID semantics and work with any Input/OutputFormat. Any Managed tables may - * be made insert-only transactional table. These tables don't support Update/Delete/Merge commands. - * - * Note that depending on the number of tables/partitions and amount of data in them compactions - * may take a significant amount of time and resources. The script output by this utility includes - * some heuristics that may help estimate the time required. If no script is produced, no action - * is needed. For compactions to run an instance of standalone Hive Metastore must be running. - * Please make sure hive.compactor.worker.threads is sufficiently high - this specifies the limit - * of concurrent compactions that may be run. Each compaction job is a Map-Reduce job. - * hive.compactor.job.queue may be used to set a Yarn queue ame where all compaction jobs will be - * submitted. - * - * "execute" option may be supplied to have the utility automatically execute the - * equivalent of the generated commands - * - * "location" option may be supplied followed by a path to set the location for the generated - * scripts. - * - * Random: - * This utility connects to the Metastore via API. It may be necessary to set - * -Djavax.security.auth.useSubjectCredsOnly=false in Kerberized environment if errors like - * "org.ietf.jgss.GSSException: No valid credentials provided ( - * Mechanism level: Failed to find any Kerberos tgt)" - * show up after kinit. - * - * See also org.apache.hadoop.hive.ql.util.UpgradeTool in Hive 3.x - */ -public class PreUpgradeTool implements AutoCloseable { - private static final Logger LOG = LoggerFactory.getLogger(PreUpgradeTool.class); - private static final int PARTITION_BATCH_SIZE = 10000; - - public static void main(String[] args) throws Exception { - Options cmdLineOptions = createCommandLineOptions(); - CommandLineParser parser = new GnuParser(); - CommandLine line; - try { - line = parser.parse(cmdLineOptions, args); - } catch (ParseException e) { - System.err.println("PreUpgradeTool: Parsing failed. Reason: " + e.getLocalizedMessage()); - printAndExit(cmdLineOptions); - return; - } - if (line.hasOption("help")) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("upgrade-acid", cmdLineOptions); - return; - } - RunOptions runOptions = RunOptions.fromCommandLine(line); - LOG.info("Starting with " + runOptions.toString()); - - try { - String hiveVer = HiveVersionInfo.getShortVersion(); - LOG.info("Using Hive Version: " + HiveVersionInfo.getVersion() + " build: " + - HiveVersionInfo.getBuildVersion()); - if(!hiveVer.startsWith("2.")) { - throw new IllegalStateException("preUpgrade requires Hive 2.x. Actual: " + hiveVer); - } - try (PreUpgradeTool tool = new PreUpgradeTool(runOptions)) { - tool.prepareAcidUpgradeInternal(); - } - } catch(Exception ex) { - LOG.error("PreUpgradeTool failed", ex); - throw ex; - } - } - - private final HiveConf conf; - private final CloseableThreadLocal metaStoreClient; - private final ThreadLocal txns; - private final RunOptions runOptions; - - public PreUpgradeTool(RunOptions runOptions) { - this.runOptions = runOptions; - this.conf = hiveConf != null ? hiveConf : new HiveConf(); - this.metaStoreClient = new CloseableThreadLocal<>(this::getHMS, IMetaStoreClient::close, - runOptions.getTablePoolSize()); - this.txns = ThreadLocal.withInitial(() -> { - /* - This API changed from 2.x to 3.0. so this won't even compile with 3.0 - but it doesn't need to since we only run this preUpgrade - */ - try { - TxnStore txnHandler = TxnUtils.getTxnStore(conf); - return TxnUtils.createValidCompactTxnList(txnHandler.getOpenTxnsInfo()); - } catch (MetaException e) { - throw new RuntimeException(e); - } - }); - } - - private static void printAndExit(Options cmdLineOptions) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("upgrade-acid", cmdLineOptions); - System.exit(1); - } - - static Options createCommandLineOptions() { - try { - Options cmdLineOptions = new Options(); - cmdLineOptions.addOption(new Option("help", "Generates a script to execute on 2.x" + - " cluster. This requires 2.x binaries on the classpath and hive-site.xml.")); - Option exec = new Option("execute", - "Executes commands equivalent to generated scrips"); - exec.setOptionalArg(true); - cmdLineOptions.addOption(exec); - Option locationOption = new Option("location", true, - "Location to write scripts to. Default is CWD."); - locationOption.setArgName("path of directory"); - cmdLineOptions.addOption(locationOption); - - Option dbRegexOption = new Option("d", - "Regular expression to match database names on which this tool will be run. Default: all databases"); - dbRegexOption.setLongOpt("dbRegex"); - dbRegexOption.setArgs(1); - dbRegexOption.setArgName("regex"); - cmdLineOptions.addOption(dbRegexOption); - - Option tableRegexOption = new Option("t", - "Regular expression to match table names on which this tool will be run. Default: all tables"); - tableRegexOption.setLongOpt("tableRegex"); - tableRegexOption.setArgs(1); - tableRegexOption.setArgName("regex"); - cmdLineOptions.addOption(tableRegexOption); - - Option tableTypeOption = new Option("tt", - String.format("Table type to match tables on which this tool will be run. Possible values: %s " + - "Default: all tables", - Arrays.stream(TableType.values()).map(Enum::name).collect(Collectors.joining("|")))); - tableTypeOption.setLongOpt("tableType"); - tableTypeOption.setArgs(1); - tableTypeOption.setArgName("table type"); - cmdLineOptions.addOption(tableTypeOption); - - Option tablePoolSizeOption = new Option("tn", "Number of threads to process tables."); - tablePoolSizeOption.setLongOpt("tablePoolSize"); - tablePoolSizeOption.setArgs(1); - tablePoolSizeOption.setArgName("pool size"); - cmdLineOptions.addOption(tablePoolSizeOption); - - return cmdLineOptions; - } catch(Exception ex) { - LOG.error("init()", ex); - throw ex; - } - } - - private static HiveMetaHookLoader getHookLoader() { - return new HiveMetaHookLoader() { - @Override - public HiveMetaHook getHook( - org.apache.hadoop.hive.metastore.api.Table tbl) { - return null; - } - }; - } - - public IMetaStoreClient getHMS() { - UserGroupInformation loggedInUser = null; - try { - loggedInUser = UserGroupInformation.getLoginUser(); - } catch (IOException e) { - LOG.warn("Unable to get logged in user via UGI. err: {}", e.getMessage()); - } - boolean secureMode = loggedInUser != null && loggedInUser.hasKerberosCredentials(); - if (secureMode) { - conf.setBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL, true); - } - try { - LOG.info("Creating metastore client for {}", "PreUpgradeTool"); - /* I'd rather call return RetryingMetaStoreClient.getProxy(conf, true) - which calls HiveMetaStoreClient(HiveConf, Boolean) which exists in - (at least) 2.1.0.2.6.5.0-292 and later but not in 2.1.0.2.6.0.3-8 (the HDP 2.6 release) - i.e. RetryingMetaStoreClient.getProxy(conf, true) is broken in 2.6.0*/ - IMetaStoreClient client = RetryingMetaStoreClient.getProxy(conf, - new Class[]{HiveConf.class, HiveMetaHookLoader.class, Boolean.class}, - new Object[]{conf, getHookLoader(), Boolean.TRUE}, null, HiveMetaStoreClient.class.getName()); - if (hiveConf != null) { - SessionState ss = SessionState.start(conf); - ss.applyAuthorizationPolicy(); - } - return client; - } catch (MetaException | HiveException e) { - throw new RuntimeException("Error connecting to Hive Metastore URI: " - + conf.getVar(HiveConf.ConfVars.METASTOREURIS) + ". " + e.getMessage(), e); - } - } - - /* - * todo: change script comments to a preamble instead of a footer - */ - private void prepareAcidUpgradeInternal() - throws HiveException, TException, IOException { - if (!isAcidEnabled(conf)) { - LOG.info("acid is off, there can't be any acid tables - nothing to compact"); - return; - } - IMetaStoreClient hms = metaStoreClient.get(); - LOG.debug("Looking for databases"); - String exceptionMsg = null; - List databases; - CompactTablesState compactTablesState; - try { - databases = hms.getDatabases(runOptions.getDbRegex()); //TException - LOG.debug("Found " + databases.size() + " databases to process"); - - ForkJoinPool processTablePool = new ForkJoinPool( - runOptions.getTablePoolSize(), - new NamedForkJoinWorkerThreadFactory("Table-"), - getUncaughtExceptionHandler(), - false - ); - compactTablesState = databases.stream() - .map(dbName -> processDatabase(dbName, processTablePool, runOptions)) - .reduce(CompactTablesState::merge) - .orElse(CompactTablesState.empty()); - - } catch (Exception e) { - if (isAccessControlException(e)) { - exceptionMsg = "Unable to get databases. Pre-upgrade tool requires read-access " + - "to databases and tables to determine if a table has to be compacted. " + - "Set " + HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname + " config to " + - "false to allow read-access to databases and tables and retry the pre-upgrade tool again.."; - } - throw new HiveException(exceptionMsg, e); - } - - makeCompactionScript(compactTablesState, runOptions.getOutputDir()); - - if(runOptions.isExecute()) { - while(compactTablesState.getMetaInfo().getCompactionIds().size() > 0) { - LOG.debug("Will wait for " + compactTablesState.getMetaInfo().getCompactionIds().size() + - " compactions to complete"); - ShowCompactResponse resp = hms.showCompactions(); - for(ShowCompactResponseElement e : resp.getCompacts()) { - final String state = e.getState(); - boolean removed; - switch (state) { - case TxnStore.CLEANING_RESPONSE: - case TxnStore.SUCCEEDED_RESPONSE: - removed = compactTablesState.getMetaInfo().getCompactionIds().remove(e.getId()); - if(removed) { - LOG.debug("Required compaction succeeded: " + e.toString()); - } - break; - case TxnStore.ATTEMPTED_RESPONSE: - case TxnStore.FAILED_RESPONSE: - removed = compactTablesState.getMetaInfo().getCompactionIds().remove(e.getId()); - if(removed) { - LOG.warn("Required compaction failed: " + e.toString()); - } - break; - case TxnStore.INITIATED_RESPONSE: - //may flood the log - //LOG.debug("Still waiting on: " + e.toString()); - break; - case TxnStore.WORKING_RESPONSE: - LOG.debug("Still working on: " + e.toString()); - break; - default://shouldn't be any others - LOG.error("Unexpected state for : " + e.toString()); - } - } - if(compactTablesState.getMetaInfo().getCompactionIds().size() > 0) { - try { - if (callback != null) { - callback.onWaitForCompaction(); - } - Thread.sleep(pollIntervalMs); - } catch (InterruptedException ex) { - //this only responds to ^C - } - } - } - } - } - - private Thread.UncaughtExceptionHandler getUncaughtExceptionHandler() { - return (t, e) -> LOG.error(String.format("Thread %s exited with error", t.getName()), e); - } - - private CompactTablesState processDatabase( - String dbName, ForkJoinPool threadPool, RunOptions runOptions) { - try { - IMetaStoreClient hms = metaStoreClient.get(); - - List tables; - if (runOptions.getTableType() == null) { - tables = hms.getTables(dbName, runOptions.getTableRegex()); - LOG.debug("found {} tables in {}", tables.size(), dbName); - } else { - tables = hms.getTables(dbName, runOptions.getTableRegex(), runOptions.getTableType()); - LOG.debug("found {} {} in {}", tables.size(), runOptions.getTableType().name(), dbName); - } - - return threadPool.submit( - () -> tables.parallelStream() - .map(table -> processTable(dbName, table, runOptions)) - .reduce(CompactTablesState::merge)).get() - .orElse(CompactTablesState.empty()); - } catch (Exception e) { - if (isAccessControlException(e)) { - // we may not have access to read all tables from this db - throw new RuntimeException("Unable to access " + dbName + ". Pre-upgrade tool requires read-access " + - "to databases and tables to determine if a table has to be compacted. " + - "Set " + HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname + " config to " + - "false to allow read-access to databases and tables and retry the pre-upgrade tool again..", e); - } - throw new RuntimeException(e); - } - } - - private CompactTablesState processTable( - String dbName, String tableName, RunOptions runOptions) { - try { - IMetaStoreClient hms = metaStoreClient.get(); - final CompactionMetaInfo compactionMetaInfo = new CompactionMetaInfo(); - - Table t = hms.getTable(dbName, tableName); - LOG.debug("processing table " + Warehouse.getQualifiedName(t)); - List compactionCommands = - getCompactionCommands(t, conf, hms, compactionMetaInfo, runOptions.isExecute(), txns.get()); - return CompactTablesState.compactions(compactionCommands, compactionMetaInfo); - /*todo: handle renaming files somewhere*/ - } catch (Exception e) { - if (isAccessControlException(e)) { - // this could be external table with 0 permission for hive user - throw new RuntimeException( - "Unable to access " + dbName + "." + tableName + ". Pre-upgrade tool requires read-access " + - "to databases and tables to determine if a table has to be compacted. " + - "Set " + HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname + " config to " + - "false to allow read-access to databases and tables and retry the pre-upgrade tool again..", e); - } - throw new RuntimeException(e); - } - } - - private boolean isAccessControlException(final Exception e) { - // hadoop security AccessControlException - if ((e instanceof MetaException && e.getCause() instanceof AccessControlException) || - ExceptionUtils.getRootCause(e) instanceof AccessControlException) { - return true; - } - - // java security AccessControlException - if ((e instanceof MetaException && e.getCause() instanceof java.security.AccessControlException) || - ExceptionUtils.getRootCause(e) instanceof java.security.AccessControlException) { - return true; - } - - // metastore in some cases sets the AccessControlException as message instead of wrapping the exception - return e instanceof MetaException - && e.getMessage().startsWith("java.security.AccessControlException: Permission denied"); - } - - /** - * Generates a set compaction commands to run on pre Hive 3 cluster. - */ - private static void makeCompactionScript(CompactTablesState result, String scriptLocation) throws IOException { - if (result.getCompactionCommands().isEmpty()) { - LOG.info("No compaction is necessary"); - return; - } - String fileName = "compacts_" + System.currentTimeMillis() + ".sql"; - LOG.debug("Writing compaction commands to " + fileName); - try(PrintWriter pw = createScript( - result.getCompactionCommands(), fileName, scriptLocation)) { - //add post script - pw.println("-- Generated total of " + result.getCompactionCommands().size() + " compaction commands"); - if(result.getMetaInfo().getNumberOfBytes() < Math.pow(2, 20)) { - //to see it working in UTs - pw.println("-- The total volume of data to be compacted is " + - String.format("%.6fMB", result.getMetaInfo().getNumberOfBytes()/Math.pow(2, 20))); - } else { - pw.println("-- The total volume of data to be compacted is " + - String.format("%.3fGB", result.getMetaInfo().getNumberOfBytes()/Math.pow(2, 30))); - } - pw.println(); - //todo: should be at the top of the file... - pw.println( - "-- Please note that compaction may be a heavyweight and time consuming process.\n" + - "-- Submitting all of these commands will enqueue them to a scheduling queue from\n" + - "-- which they will be picked up by compactor Workers. The max number of\n" + - "-- concurrent Workers is controlled by hive.compactor.worker.threads configured\n" + - "-- for the standalone metastore process. Compaction itself is a Map-Reduce job\n" + - "-- which is submitted to the YARN queue identified by hive.compactor.job.queue\n" + - "-- property if defined or 'default' if not defined. It's advisable to set the\n" + - "-- capacity of this queue appropriately"); - } - } - - private static PrintWriter createScript(List commands, String fileName, - String scriptLocation) throws IOException { - FileWriter fw = new FileWriter(scriptLocation + "/" + fileName); - PrintWriter pw = new PrintWriter(fw); - for(String cmd : commands) { - pw.println(cmd + ";"); - } - return pw; - } - /** - * @return any compaction commands to run for {@code Table t} - */ - private static List getCompactionCommands(Table t, HiveConf conf, - IMetaStoreClient hms, CompactionMetaInfo compactionMetaInfo, boolean execute, - ValidTxnList txns) throws IOException, TException, HiveException { - if(!isFullAcidTable(t)) { - return Collections.emptyList(); - } - if(t.getPartitionKeysSize() <= 0) { - //not partitioned - if(!needsCompaction(new Path(t.getSd().getLocation()), conf, compactionMetaInfo, txns)) { - return Collections.emptyList(); - } - - List cmds = new ArrayList<>(); - cmds.add(getCompactionCommand(t, null)); - if(execute) { - scheduleCompaction(t, null, hms, compactionMetaInfo); - } - return cmds; - } - List partNames = hms.listPartitionNames(t.getDbName(), t.getTableName(), (short)-1); - int batchSize = PARTITION_BATCH_SIZE; - int numWholeBatches = partNames.size()/batchSize; - List compactionCommands = new ArrayList<>(); - for(int i = 0; i < numWholeBatches; i++) { - List partitionList = hms.getPartitionsByNames(t.getDbName(), t.getTableName(), - partNames.subList(i * batchSize, (i + 1) * batchSize)); - getCompactionCommands(t, partitionList, hms, execute, compactionCommands, - compactionMetaInfo, conf, txns); - } - if(numWholeBatches * batchSize < partNames.size()) { - //last partial batch - List partitionList = hms.getPartitionsByNames(t.getDbName(), t.getTableName(), - partNames.subList(numWholeBatches * batchSize, partNames.size())); - getCompactionCommands(t, partitionList, hms, execute, compactionCommands, - compactionMetaInfo, conf, txns); - } - return compactionCommands; - } - private static void getCompactionCommands(Table t, List partitionList, IMetaStoreClient hms, - boolean execute, List compactionCommands, CompactionMetaInfo compactionMetaInfo, - HiveConf conf, ValidTxnList txns) - throws IOException, TException, HiveException { - for (Partition p : partitionList) { - if (needsCompaction(new Path(p.getSd().getLocation()), conf, compactionMetaInfo, txns)) { - compactionCommands.add(getCompactionCommand(t, p)); - if (execute) { - scheduleCompaction(t, p, hms, compactionMetaInfo); - } - } - } - } - private static void scheduleCompaction(Table t, Partition p, IMetaStoreClient db, - CompactionMetaInfo compactionMetaInfo) throws HiveException, MetaException { - String partName = p == null ? null : - Warehouse.makePartName(t.getPartitionKeys(), p.getValues()); - try { - CompactionResponse resp = - //this gives an easy way to get at compaction ID so we can only wait for those this - //utility started - db.compact2(t.getDbName(), t.getTableName(), partName, CompactionType.MAJOR, null); - if (!resp.isAccepted()) { - LOG.info(Warehouse.getQualifiedName(t) + (p == null ? "" : "/" + partName) + - " is already being compacted with id=" + resp.getId()); - } else { - LOG.info("Scheduled compaction for " + Warehouse.getQualifiedName(t) + - (p == null ? "" : "/" + partName) + " with id=" + resp.getId()); - } - compactionMetaInfo.addCompactionId(resp.getId()); - } catch (TException e) { - throw new HiveException(e); - } - } - - /** - * - * @param location - path to a partition (or table if not partitioned) dir - */ - private static boolean needsCompaction(Path location, HiveConf conf, - CompactionMetaInfo compactionMetaInfo, ValidTxnList txns) throws IOException { - FileSystem fs = location.getFileSystem(conf); - FileStatus[] deltas = fs.listStatus(location, new PathFilter() { - @Override - public boolean accept(Path path) { - //checking for delete_delta is only so that this functionality can be exercised by code 3.0 - //which cannot produce any deltas with mix of update/insert events - return path.getName().startsWith("delta_") || path.getName().startsWith("delete_delta_"); - } - }); - if(deltas == null || deltas.length == 0) { - //base_n cannot contain update/delete. Original files are all 'insert' and we need to compact - //only if there are update/delete events. - return false; - } - /*getAcidState() is smart not to return any deltas in current if there is a base that covers - * them, i.e. if they were compacted but not yet cleaned. This means re-checking if - * compaction is needed should cheap(er)*/ - AcidUtils.Directory dir = AcidUtils.getAcidState(location, conf, txns); - deltaLoop: for(AcidUtils.ParsedDelta delta : dir.getCurrentDirectories()) { - FileStatus[] buckets = fs.listStatus(delta.getPath(), new PathFilter() { - @Override - public boolean accept(Path path) { - //since this is inside a delta dir created by Hive 2.x or earlier it can only contain - //bucket_x or bucket_x__flush_length - return path.getName().startsWith("bucket_"); - } - }); - for(FileStatus bucket : buckets) { - if(bucket.getPath().getName().endsWith("_flush_length")) { - //streaming ingest dir - cannot have update/delete events - continue deltaLoop; - } - if(needsCompaction(bucket, fs)) { - //found delete events - this 'location' needs compacting - compactionMetaInfo.addBytes(getDataSize(location, conf)); - - //if there are un-compacted original files, they will be included in compaction, so - //count at the size for 'cost' estimation later - for(HadoopShims.HdfsFileStatusWithId origFile : dir.getOriginalFiles()) { - FileStatus fileStatus = origFile.getFileStatus(); - if(fileStatus != null) { - compactionMetaInfo.addBytes(fileStatus.getLen()); - } - } - return true; - } - } - } - return false; - } - - /** - * @param location - path to a partition (or table if not partitioned) dir - */ - private static long getDataSize(Path location, HiveConf conf) throws IOException { - FileSystem fs = location.getFileSystem(conf); - ContentSummary cs = fs.getContentSummary(location); - return cs.getLength(); - } - - - private static final Charset UTF_8 = StandardCharsets.UTF_8; - private static final ThreadLocal UTF8_DECODER = - ThreadLocal.withInitial(UTF_8::newDecoder); - private static final String ACID_STATS = "hive.acid.stats"; - - private static boolean needsCompaction(FileStatus bucket, FileSystem fs) throws IOException { - //create reader, look at footer - //no need to check side file since it can only be in a streaming ingest delta - Reader orcReader = OrcFile.createReader(bucket.getPath(), OrcFile.readerOptions(fs.getConf()).filesystem(fs)); - if (orcReader.hasMetadataValue(ACID_STATS)) { - try { - ByteBuffer val = orcReader.getMetadataValue(ACID_STATS).duplicate(); - String acidStats = UTF8_DECODER.get().decode(val).toString(); - String[] parts = acidStats.split(","); - long updates = Long.parseLong(parts[1]); - long deletes = Long.parseLong(parts[2]); - return deletes > 0 || updates > 0; - } catch (CharacterCodingException e) { - throw new IllegalArgumentException("Bad string encoding for " + ACID_STATS, e); - } - } else { - throw new IllegalStateException("AcidStats missing in " + bucket.getPath()); - } - } - - private static String getCompactionCommand(Table t, Partition p) { - StringBuilder sb = new StringBuilder("ALTER TABLE ").append(Warehouse.getQualifiedName(t)); - if(t.getPartitionKeysSize() > 0) { - assert p != null : "must supply partition for partitioned table " + - Warehouse.getQualifiedName(t); - sb.append(" PARTITION("); - for (int i = 0; i < t.getPartitionKeysSize(); i++) { - sb.append(t.getPartitionKeys().get(i).getName()).append('=').append( - genPartValueString(t.getPartitionKeys().get(i).getType(), p.getValues().get(i))). - append(","); - } - //replace trailing ',' - sb.setCharAt(sb.length() - 1, ')'); - } - return sb.append(" COMPACT 'major'").toString(); - } - - /** - * This is copy-pasted from {@link org.apache.hadoop.hive.ql.parse.ColumnStatsSemanticAnalyzer}, - * which can't be refactored since this is linked against Hive 2.x . - */ - private static String genPartValueString(String partColType, String partVal) { - String returnVal; - if (partColType.equals(serdeConstants.STRING_TYPE_NAME) || - partColType.contains(serdeConstants.VARCHAR_TYPE_NAME) || - partColType.contains(serdeConstants.CHAR_TYPE_NAME)) { - returnVal = "'" + escapeSQLString(partVal) + "'"; - } else if (partColType.equals(serdeConstants.TINYINT_TYPE_NAME)) { - returnVal = partVal + "Y"; - } else if (partColType.equals(serdeConstants.SMALLINT_TYPE_NAME)) { - returnVal = partVal + "S"; - } else if (partColType.equals(serdeConstants.INT_TYPE_NAME)) { - returnVal = partVal; - } else if (partColType.equals(serdeConstants.BIGINT_TYPE_NAME)) { - returnVal = partVal + "L"; - } else if (partColType.contains(serdeConstants.DECIMAL_TYPE_NAME)) { - returnVal = partVal + "BD"; - } else if (partColType.equals(serdeConstants.DATE_TYPE_NAME) || - partColType.equals(serdeConstants.TIMESTAMP_TYPE_NAME)) { - returnVal = partColType + " '" + escapeSQLString(partVal) + "'"; - } else { - //for other usually not used types, just quote the value - returnVal = "'" + escapeSQLString(partVal) + "'"; - } - - return returnVal; - } - private static boolean isFullAcidTable(Table t) { - if (t.getParametersSize() <= 0) { - //cannot be acid - return false; - } - String transacationalValue = t.getParameters() - .get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL); - if ("true".equalsIgnoreCase(transacationalValue)) { - System.out.println("Found Acid table: " + Warehouse.getQualifiedName(t)); - return true; - } - return false; - } - private static boolean isAcidEnabled(HiveConf hiveConf) { - String txnMgr = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER); - boolean concurrency = hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); - String dbTxnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; - return txnMgr.equals(dbTxnMgr) && concurrency; - } - - @Override - public void close() { - metaStoreClient.close(); - } - - @VisibleForTesting - abstract static class Callback { - /** - * This is a hack enable Unit testing. Derby can't handle multiple concurrent threads but - * somehow Compactor needs to run to test "execute" mode. This callback can be used - * to run Worker. For TESTING ONLY. - */ - void onWaitForCompaction() throws MetaException {} - } - @VisibleForTesting - static Callback callback; - @VisibleForTesting - static int pollIntervalMs = 1000*30; - /** - * can set it from tests to test when config needs something other than default values. - */ - @VisibleForTesting - static HiveConf hiveConf = null; -} diff --git a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/RunOptions.java b/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/RunOptions.java deleted file mode 100644 index 534b971a7719..000000000000 --- a/upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid/RunOptions.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import org.apache.commons.cli.CommandLine; -import org.apache.hadoop.hive.metastore.TableType; - -/** - * This class's instance holds the option values were passed by the user via the command line. - */ -public class RunOptions { - - public static RunOptions fromCommandLine(CommandLine commandLine) { - String tableTypeText = commandLine.getOptionValue("tableType"); - - int defaultPoolSize = Runtime.getRuntime().availableProcessors(); - if (defaultPoolSize < 1) - defaultPoolSize = 1; - - int tablePoolSize = getIntOptionValue(commandLine, "tablePoolSize", defaultPoolSize); - if (tablePoolSize < 1) - throw new IllegalArgumentException("Please specify a positive integer option value for tablePoolSize"); - - return new RunOptions( - commandLine.getOptionValue("location", "."), - commandLine.hasOption("execute"), - commandLine.getOptionValue("dbRegex", ".*"), - commandLine.getOptionValue("tableRegex", ".*"), - tableTypeText == null ? null : TableType.valueOf(tableTypeText), - tablePoolSize); - } - - private static int getIntOptionValue(CommandLine commandLine, String optionName, int defaultValue) { - if (commandLine.hasOption(optionName)) { - try { - return Integer.parseInt(commandLine.getOptionValue(optionName)); - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Please specify a positive integer option value for " + optionName, e); - } - } - return defaultValue; - } - - private final String outputDir; - private final boolean execute; - private final String dbRegex; - private final String tableRegex; - private final TableType tableType; - private final int tablePoolSize; - - private RunOptions(String outputDir, boolean execute, String dbRegex, String tableRegex, TableType tableType, int tablePoolSize) { - this.outputDir = outputDir; - this.execute = execute; - this.dbRegex = dbRegex; - this.tableRegex = tableRegex; - this.tableType = tableType; - this.tablePoolSize = tablePoolSize; - } - - public String getOutputDir() { - return outputDir; - } - - public boolean isExecute() { - return execute; - } - - public String getDbRegex() { - return dbRegex; - } - - public String getTableRegex() { - return tableRegex; - } - - public TableType getTableType() { - return tableType; - } - - public int getTablePoolSize() { - return tablePoolSize; - } - - @Override - public String toString() { - return "RunOptions{" + - "outputDir='" + outputDir + '\'' + - ", execute=" + execute + - ", dbRegex='" + dbRegex + '\'' + - ", tableRegex='" + tableRegex + '\'' + - ", tableType=" + tableType + - ", tablePoolSize=" + tablePoolSize + - '}'; - } -} diff --git a/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestCloseableThreadLocal.java b/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestCloseableThreadLocal.java deleted file mode 100644 index 2584a3be5285..000000000000 --- a/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestCloseableThreadLocal.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - -import org.junit.Test; - -public class TestCloseableThreadLocal { - - private static class AutoCloseableStub implements AutoCloseable { - - private boolean closed = false; - - public boolean isClosed() { - return closed; - } - - @Override - public void close() { - closed = true; - } - } - - @Test - public void testResourcesAreInitiallyNotClosed() { - CloseableThreadLocal closeableThreadLocal = - new CloseableThreadLocal<>(AutoCloseableStub::new, AutoCloseableStub::close, 1); - - assertThat(closeableThreadLocal.get().isClosed(), is(false)); - } - - @Test - public void testAfterCallingCloseAllInstancesAreClosed() throws ExecutionException, InterruptedException { - CloseableThreadLocal closeableThreadLocal = - new CloseableThreadLocal<>(AutoCloseableStub::new, AutoCloseableStub::close, 2); - - AutoCloseableStub asyncInstance = CompletableFuture.supplyAsync(closeableThreadLocal::get).get(); - AutoCloseableStub syncInstance = closeableThreadLocal.get(); - - closeableThreadLocal.close(); - - assertThat(asyncInstance.isClosed(), is(true)); - assertThat(syncInstance.isClosed(), is(true)); - } - - @Test - public void testSubsequentGetsInTheSameThreadGivesBackTheSameObject() { - CloseableThreadLocal closeableThreadLocal = - new CloseableThreadLocal<>(AutoCloseableStub::new, AutoCloseableStub::close, 2); - - AutoCloseableStub ref1 = closeableThreadLocal.get(); - AutoCloseableStub ref2 = closeableThreadLocal.get(); - assertThat(ref1, is(ref2)); - } - - @Test - public void testDifferentThreadsHasDifferentInstancesOfTheResource() throws ExecutionException, InterruptedException { - CloseableThreadLocal closeableThreadLocal = - new CloseableThreadLocal<>(AutoCloseableStub::new, AutoCloseableStub::close, 2); - - AutoCloseableStub asyncInstance = CompletableFuture.supplyAsync(closeableThreadLocal::get).get(); - AutoCloseableStub syncInstance = closeableThreadLocal.get(); - assertThat(asyncInstance, is(not(syncInstance))); - } -} \ No newline at end of file diff --git a/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestPreUpgradeTool.java b/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestPreUpgradeTool.java deleted file mode 100644 index 2064baa544c0..000000000000 --- a/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestPreUpgradeTool.java +++ /dev/null @@ -1,495 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import static org.hamcrest.CoreMatchers.allOf; -import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.StringContains.containsString; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.attribute.PosixFilePermission; -import java.nio.file.attribute.PosixFilePermissions; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; -import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement; -import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; -import org.apache.hadoop.hive.metastore.txn.TxnStore; -import org.apache.hadoop.hive.metastore.txn.TxnUtils; -import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.io.HiveInputFormat; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.ql.txn.compactor.Worker; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; - -public class TestPreUpgradeTool { - private static final String TEST_DATA_DIR = new File(System.getProperty("java.io.tmpdir") + - File.separator + TestPreUpgradeTool.class.getCanonicalName() + "-" + System.currentTimeMillis() - ).getPath().replaceAll("\\\\", "/"); - - private String getTestDataDir() { - return TEST_DATA_DIR; - } - - /** - * preUpgrade: test tables that need to be compacted, waits for compaction - * postUpgrade: generates scripts w/o asserts - */ - @Test - public void testUpgrade() throws Exception { - int[][] data = {{1, 2}, {3, 4}, {5, 6}}; - int[][] dataPart = {{1, 2, 10}, {3, 4, 11}, {5, 6, 12}}; - runStatementOnDriver("drop table if exists TAcid"); - runStatementOnDriver("drop table if exists TAcidPart"); - runStatementOnDriver("drop table if exists TFlat"); - runStatementOnDriver("drop table if exists TFlatText"); - - try { - runStatementOnDriver( - "create table TAcid (a int, b int) clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); - runStatementOnDriver( - "create table TAcidPart (a int, b int) partitioned by (p tinyint) clustered by (b) into 2 buckets stored" + - " as orc TBLPROPERTIES ('transactional'='true')"); - //on 2.x these are guaranteed to not be acid - runStatementOnDriver("create table TFlat (a int, b int) stored as orc tblproperties('transactional'='false')"); - runStatementOnDriver( - "create table TFlatText (a int, b int) stored as textfile tblproperties('transactional'='false')"); - - - //this needs major compaction - runStatementOnDriver("insert into TAcid" + makeValuesClause(data)); - runStatementOnDriver("update TAcid set a = 1 where b = 2"); - - //this table needs to be converted to CRUD Acid - runStatementOnDriver("insert into TFlat" + makeValuesClause(data)); - - //this table needs to be converted to MM - runStatementOnDriver("insert into TFlatText" + makeValuesClause(data)); - - //p=10 needs major compaction - runStatementOnDriver("insert into TAcidPart partition(p)" + makeValuesClause(dataPart)); - runStatementOnDriver("update TAcidPart set a = 1 where b = 2 and p = 10"); - - //todo: add partitioned table that needs conversion to MM/Acid - - //todo: rename files case - String[] args = {"-location", getTestDataDir(), "-execute"}; - PreUpgradeTool.callback = new PreUpgradeTool.Callback() { - @Override - void onWaitForCompaction() throws MetaException { - runWorker(hiveConf); - } - }; - PreUpgradeTool.pollIntervalMs = 1; - PreUpgradeTool.hiveConf = hiveConf; - PreUpgradeTool.main(args); - - String[] scriptFiles = getScriptFiles(); - assertThat(scriptFiles.length, is(1)); - - List scriptContent = loadScriptContent(new File(getTestDataDir(), scriptFiles[0])); - assertThat(scriptContent.size(), is(2)); - assertThat(scriptContent, hasItem(is("ALTER TABLE default.tacid COMPACT 'major';"))); - assertThat(scriptContent, hasItem(is("ALTER TABLE default.tacidpart PARTITION(p=10Y) COMPACT 'major';"))); - - TxnStore txnHandler = TxnUtils.getTxnStore(hiveConf); - - ShowCompactResponse resp = txnHandler.showCompact(new ShowCompactRequest()); - Assert.assertEquals(2, resp.getCompactsSize()); - for (ShowCompactResponseElement e : resp.getCompacts()) { - Assert.assertEquals(e.toString(), TxnStore.CLEANING_RESPONSE, e.getState()); - } - - // Check whether compaction was successful in the first run - File secondRunDataDir = new File(getTestDataDir(), "secondRun"); - if (!secondRunDataDir.exists()) { - if (!secondRunDataDir.mkdir()) { - throw new IOException("Unable to create directory" + secondRunDataDir.getAbsolutePath()); - } - } - String[] args2 = {"-location", secondRunDataDir.getAbsolutePath()}; - PreUpgradeTool.main(args2); - - scriptFiles = secondRunDataDir.list(); - assertThat(scriptFiles, is(not(nullValue()))); - assertThat(scriptFiles.length, is(0)); - - } finally { - runStatementOnDriver("drop table if exists TAcid"); - runStatementOnDriver("drop table if exists TAcidPart"); - runStatementOnDriver("drop table if exists TFlat"); - runStatementOnDriver("drop table if exists TFlatText"); - } - } - - private static final String INCLUDE_DATABASE_NAME ="DInclude"; - private static final String EXCLUDE_DATABASE_NAME ="DExclude"; - - @Test - public void testOnlyFilteredDatabasesAreUpgradedWhenRegexIsGiven() throws Exception { - int[][] data = {{1, 2}, {3, 4}, {5, 6}}; - runStatementOnDriver("drop database if exists " + INCLUDE_DATABASE_NAME + " cascade"); - runStatementOnDriver("drop database if exists " + EXCLUDE_DATABASE_NAME + " cascade"); - - try { - runStatementOnDriver("create database " + INCLUDE_DATABASE_NAME); - runStatementOnDriver("use " + INCLUDE_DATABASE_NAME); - runStatementOnDriver("create table " + INCLUDE_TABLE_NAME + " (a int, b int) clustered by (b) " + - "into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); - runStatementOnDriver("insert into " + INCLUDE_TABLE_NAME + makeValuesClause(data)); - runStatementOnDriver("update " + INCLUDE_TABLE_NAME + " set a = 1 where b = 2"); - - runStatementOnDriver("create database " + EXCLUDE_DATABASE_NAME); - runStatementOnDriver("use " + EXCLUDE_DATABASE_NAME); - runStatementOnDriver("create table " + EXCLUDE_DATABASE_NAME + " (a int, b int) clustered by (b) " + - "into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); - runStatementOnDriver("insert into " + EXCLUDE_DATABASE_NAME + makeValuesClause(data)); - runStatementOnDriver("update " + EXCLUDE_DATABASE_NAME + " set a = 1 where b = 2"); - - String[] args = {"-location", getTestDataDir(), "-dbRegex", "*include*"}; - PreUpgradeTool.callback = new PreUpgradeTool.Callback() { - @Override - void onWaitForCompaction() throws MetaException { - runWorker(hiveConf); - } - }; - PreUpgradeTool.pollIntervalMs = 1; - PreUpgradeTool.hiveConf = hiveConf; - PreUpgradeTool.main(args); - - String[] scriptFiles = getScriptFiles(); - assertThat(scriptFiles.length, is(1)); - - List scriptContent = loadScriptContent(new File(getTestDataDir(), scriptFiles[0])); - assertThat(scriptContent.size(), is(1)); - assertThat(scriptContent.get(0), is("ALTER TABLE dinclude.tinclude COMPACT 'major';")); - - } finally { - runStatementOnDriver("drop database if exists " + INCLUDE_DATABASE_NAME + " cascade"); - runStatementOnDriver("drop database if exists " + EXCLUDE_DATABASE_NAME + " cascade"); - } - } - - private static final String INCLUDE_TABLE_NAME ="TInclude"; - private static final String EXCLUDE_TABLE_NAME ="TExclude"; - - @Test - public void testOnlyFilteredTablesAreUpgradedWhenRegexIsGiven() throws Exception { - int[][] data = {{1, 2}, {3, 4}, {5, 6}}; - runStatementOnDriver("drop table if exists " + INCLUDE_TABLE_NAME); - runStatementOnDriver("drop table if exists " + EXCLUDE_TABLE_NAME); - - try { - runStatementOnDriver("create table " + INCLUDE_TABLE_NAME + " (a int, b int) clustered by (b) " + - "into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); - runStatementOnDriver("create table " + EXCLUDE_TABLE_NAME + " (a int, b int) clustered by (b) " + - "into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); - - runStatementOnDriver("insert into " + INCLUDE_TABLE_NAME + makeValuesClause(data)); - runStatementOnDriver("update " + INCLUDE_TABLE_NAME + " set a = 1 where b = 2"); - - runStatementOnDriver("insert into " + EXCLUDE_TABLE_NAME + makeValuesClause(data)); - runStatementOnDriver("update " + EXCLUDE_TABLE_NAME + " set a = 1 where b = 2"); - - String[] args = {"-location", getTestDataDir(), "-tableRegex", "*include*"}; - PreUpgradeTool.callback = new PreUpgradeTool.Callback() { - @Override - void onWaitForCompaction() throws MetaException { - runWorker(hiveConf); - } - }; - PreUpgradeTool.pollIntervalMs = 1; - PreUpgradeTool.hiveConf = hiveConf; - PreUpgradeTool.main(args); - - String[] scriptFiles = getScriptFiles(); - assertThat(scriptFiles.length, is(1)); - - List scriptContent = loadScriptContent(new File(getTestDataDir(), scriptFiles[0])); - assertThat(scriptContent.size(), is(1)); - assertThat(scriptContent.get(0), allOf( - containsString("ALTER TABLE"), - containsString(INCLUDE_TABLE_NAME.toLowerCase()), - containsString("COMPACT"))); - - } finally { - runStatementOnDriver("drop table if exists " + INCLUDE_TABLE_NAME); - runStatementOnDriver("drop table if exists " + EXCLUDE_TABLE_NAME); - } - } - - private String[] getScriptFiles() { - File testDataDir = new File(getTestDataDir()); - String[] scriptFiles = testDataDir.list((dir, name) -> name.startsWith("compacts_") && name.endsWith(".sql")); - assertThat(scriptFiles, is(not(nullValue()))); - return scriptFiles; - } - - private List loadScriptContent(File file) throws IOException { - List content = org.apache.commons.io.FileUtils.readLines(file); - content.removeIf(line -> line.startsWith("--")); - content.removeIf(StringUtils::isBlank); - return content; - } - - @Test - public void testUpgradeExternalTableNoReadPermissionForDatabase() throws Exception { - int[][] data = {{1, 2}, {3, 4}, {5, 6}}; - - runStatementOnDriver("drop database if exists test cascade"); - runStatementOnDriver("drop table if exists TExternal"); - - runStatementOnDriver("create database test"); - runStatementOnDriver("create table test.TExternal (a int, b int) stored as orc tblproperties" + - "('transactional'='false')"); - - //this needs major compaction - runStatementOnDriver("insert into test.TExternal" + makeValuesClause(data)); - - String dbDir = getWarehouseDir() + "/test.db"; - File dbPath = new File(dbDir); - try { - Set perms = PosixFilePermissions.fromString("-w-------"); - Files.setPosixFilePermissions(dbPath.toPath(), perms); - String[] args = {"-location", getTestDataDir(), "-execute"}; - PreUpgradeTool.pollIntervalMs = 1; - PreUpgradeTool.hiveConf = hiveConf; - Exception expected = null; - try { - PreUpgradeTool.main(args); - } catch (Exception e) { - expected = e; - } - - Assert.assertNotNull(expected); - Assert.assertTrue(expected instanceof HiveException); - Assert.assertTrue(expected.getMessage().contains("Pre-upgrade tool requires " + - "read-access to databases and tables to determine if a table has to be compacted.")); - } finally { - Set perms = PosixFilePermissions.fromString("rwxrw----"); - Files.setPosixFilePermissions(dbPath.toPath(), perms); - } - } - - @Test - public void testUpgradeExternalTableNoReadPermissionForTable() throws Exception { - int[][] data = {{1, 2}, {3, 4}, {5, 6}}; - runStatementOnDriver("drop table if exists TExternal"); - - runStatementOnDriver("create table TExternal (a int, b int) stored as orc tblproperties('transactional'='false')"); - - //this needs major compaction - runStatementOnDriver("insert into TExternal" + makeValuesClause(data)); - - String tableDir = getWarehouseDir() + "/texternal"; - File tablePath = new File(tableDir); - try { - Set perms = PosixFilePermissions.fromString("-w-------"); - Files.setPosixFilePermissions(tablePath.toPath(), perms); - String[] args = {"-location", getTestDataDir(), "-execute"}; - PreUpgradeTool.pollIntervalMs = 1; - PreUpgradeTool.hiveConf = hiveConf; - Exception expected = null; - try { - PreUpgradeTool.main(args); - } catch (Exception e) { - expected = e; - } - - Assert.assertNotNull(expected); - Assert.assertTrue(expected instanceof HiveException); - Assert.assertTrue(expected.getMessage().contains("Pre-upgrade tool requires" + - " read-access to databases and tables to determine if a table has to be compacted.")); - } finally { - Set perms = PosixFilePermissions.fromString("rwxrw----"); - Files.setPosixFilePermissions(tablePath.toPath(), perms); - } - } - - @Test - public void testConcurrency() throws Exception { - int numberOfTables = 20; - String tablePrefix = "concurrency_"; - - int[][] data = {{1, 2}, {3, 4}, {5, 6}, {7, 8}, {9, 10}, - {11, 12}, {13, 14}, {15, 16}, {17, 18}, {19, 20}}; - for (int i = 0; i < numberOfTables; i++) { - runStatementOnDriver("drop table if exists " + tablePrefix + i); - } - - try { - for (int i = 0; i < numberOfTables; i++) { - String tableName = tablePrefix + i; - runStatementOnDriver( - "create table " + tableName + " (a int, b int) " + - "clustered by (b) " + - "into 10 buckets " + - "stored as orc TBLPROPERTIES ('transactional'='true')"); - runStatementOnDriver("insert into " + tableName + makeValuesClause(data)); - } - - String[] args = {"-location", getTestDataDir(), "-execute"}; - PreUpgradeTool.callback = new PreUpgradeTool.Callback() { - @Override - void onWaitForCompaction() throws MetaException { - runWorker(hiveConf); - } - }; - PreUpgradeTool.pollIntervalMs = 1; - PreUpgradeTool.hiveConf = hiveConf; - PreUpgradeTool.main(args); - - } finally { - for (int i = 0; i < numberOfTables; i++) { - runStatementOnDriver("drop table if exists " + tablePrefix + i); - } - } - } - - private static void runWorker(HiveConf hiveConf) throws MetaException { - AtomicBoolean stop = new AtomicBoolean(true); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setHiveConf(hiveConf); - AtomicBoolean looped = new AtomicBoolean(); - t.init(stop, looped); - t.run(); - } - - private static String makeValuesClause(int[][] rows) { - assert rows.length > 0; - StringBuilder sb = new StringBuilder(" values"); - for(int[] row : rows) { - assert row.length > 0; - if(row.length > 1) { - sb.append("("); - } - for(int value : row) { - sb.append(value).append(","); - } - sb.setLength(sb.length() - 1);//remove trailing comma - if(row.length > 1) { - sb.append(")"); - } - sb.append(","); - } - sb.setLength(sb.length() - 1);//remove trailing comma - return sb.toString(); - } - - private List runStatementOnDriver(String stmt) throws Exception { - CommandProcessorResponse cpr = d.run(stmt); - if(cpr.getResponseCode() != 0) { - throw new RuntimeException(stmt + " failed: " + cpr); - } - List rs = new ArrayList(); - d.getResults(rs); - return rs; - } - @Before - public void setUp() throws Exception { - setUpInternal(); - } - private void initHiveConf() { - hiveConf = new HiveConf(this.getClass()); - } - @Rule - public TestName testName = new TestName(); - private HiveConf hiveConf; - private Driver d; - private void setUpInternal() throws Exception { - initHiveConf(); - TxnDbUtil.cleanDb();//todo: api changed in 3.0 - FileUtils.deleteDirectory(new File(getTestDataDir())); - - Path workDir = new Path(System.getProperty("test.tmp.dir", - "target" + File.separator + "test" + File.separator + "tmp")); - hiveConf.set("mapred.local.dir", workDir + File.separator + this.getClass().getSimpleName() - + File.separator + "mapred" + File.separator + "local"); - hiveConf.set("mapred.system.dir", workDir + File.separator + this.getClass().getSimpleName() - + File.separator + "mapred" + File.separator + "system"); - hiveConf.set("mapreduce.jobtracker.staging.root.dir", workDir + File.separator + this.getClass().getSimpleName() - + File.separator + "mapred" + File.separator + "staging"); - hiveConf.set("mapred.temp.dir", workDir + File.separator + this.getClass().getSimpleName() - + File.separator + "mapred" + File.separator + "temp"); - hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); - hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, getWarehouseDir()); - hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName()); - hiveConf - .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, - "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); - hiveConf - .setVar(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS, - "org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener"); - hiveConf - .setVar(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER, - "org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider"); - hiveConf.setBoolVar(HiveConf.ConfVars.MERGE_CARDINALITY_VIOLATION_CHECK, true); - hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSCOLAUTOGATHER, false); - TxnDbUtil.setConfValues(hiveConf); - TxnDbUtil.prepDb();//todo: api changed in 3.0 - File f = new File(getWarehouseDir()); - if (f.exists()) { - FileUtil.fullyDelete(f); - } - if (!(new File(getWarehouseDir()).mkdirs())) { - throw new RuntimeException("Could not create " + getWarehouseDir()); - } - SessionState ss = SessionState.start(hiveConf); - ss.applyAuthorizationPolicy(); - d = new Driver(new QueryState(hiveConf), null); - d.setMaxRows(10000); - } - private String getWarehouseDir() { - return getTestDataDir() + "/warehouse"; - } - @After - public void tearDown() throws Exception { - if (d != null) { - d.close(); - d.destroy(); - d = null; - } - } - -} diff --git a/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestRunOptions.java b/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestRunOptions.java deleted file mode 100644 index 8005b5cbc271..000000000000 --- a/upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestRunOptions.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.upgrade.acid; - -import static org.apache.hadoop.hive.upgrade.acid.PreUpgradeTool.createCommandLineOptions; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; - -import org.apache.commons.cli.GnuParser; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -public class TestRunOptions { - - @Rule - public ExpectedException expectedEx = ExpectedException.none(); - - @Test - public void testTablePoolSizeIs5WhenSpecified() throws Exception { - String[] args = {"-tablePoolSize", "5"}; - RunOptions runOptions = RunOptions.fromCommandLine(new GnuParser().parse(createCommandLineOptions(), args)); - assertThat(runOptions.getTablePoolSize(), is(5)); - } - - @Test - public void testExceptionIsThrownWhenTablePoolSizeIsNotANumber() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage("Please specify a positive integer option value for tablePoolSize"); - - String[] args = {"-tablePoolSize", "notANumber"}; - RunOptions.fromCommandLine(new GnuParser().parse(createCommandLineOptions(), args)); - } - - @Test - public void testExceptionIsThrownWhenTablePoolSizeIsLessThan1() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage("Please specify a positive integer option value for tablePoolSize"); - - String[] args = {"-tablePoolSize", "0"}; - RunOptions.fromCommandLine(new GnuParser().parse(createCommandLineOptions(), args)); - } - - @Test - public void testExceptionIsThrownWhenTablePoolSizeIsNotInteger() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage("Please specify a positive integer option value for tablePoolSize"); - - String[] args = {"-tablePoolSize", "0.5"}; - RunOptions.fromCommandLine(new GnuParser().parse(createCommandLineOptions(), args)); - } -} From 564d7e54d2360488611da39d0e5f027a2d574fc1 Mon Sep 17 00:00:00 2001 From: AnmolSun <124231245+AnmolSun@users.noreply.github.com> Date: Wed, 24 Jan 2024 16:04:29 +0530 Subject: [PATCH 153/179] HIVE-27993 : Netty4 ShuffleHandler: should use 1 boss thread (#5006) (AnmolSun reviewed by Laszlo Bodor, Zhang Butao) --- .../apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java b/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java index 70f568ee9041..9893210bd93c 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/shufflehandler/ShuffleHandler.java @@ -299,7 +299,7 @@ public void operationComplete(ChannelFuture future) throws Exception { final String BOSS_THREAD_NAME_PREFIX = "ShuffleHandler Netty Boss #"; AtomicInteger bossThreadCounter = new AtomicInteger(0); - bossGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { + bossGroup = new NioEventLoopGroup(1, new ThreadFactory() { @Override public Thread newThread(Runnable r) { return new Thread(r, BOSS_THREAD_NAME_PREFIX + bossThreadCounter.incrementAndGet()); From b418e3c9f479ba8e7d31e6470306111002ffa809 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Thu, 25 Jan 2024 12:18:19 +0200 Subject: [PATCH 154/179] HIVE-28030: LLAP util code refactor (Denys Kuzmenko, reviewed by Ayush Saxena) Closes #5030 --- .../apache/hadoop/hive/llap/security/LlapSignerImpl.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java index a7fc398892ff..047e17686b7a 100644 --- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hive.llap.security; import java.io.IOException; -import java.util.Arrays; +import java.security.MessageDigest; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; @@ -58,7 +58,9 @@ public SignedMessage serializeAndSign(Signable message) throws IOException { public void checkSignature(byte[] message, byte[] signature, int keyId) throws SecurityException { byte[] expectedSignature = secretManager.signWithKey(message, keyId); - if (Arrays.equals(signature, expectedSignature)) return; + if (MessageDigest.isEqual(signature, expectedSignature)) { + return; + } throw new SecurityException("Message signature does not match"); } From b75a59779dcf9c5ef74890c8916dbe9a2c13aef4 Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Thu, 25 Jan 2024 18:38:53 +0800 Subject: [PATCH 155/179] HIVE-27914: Fix the missing partitions judgement in drop_partitions_req (#4905)(Wechar Yu, reviewed by Butao Zhang) --- .../hadoop/hive/ql/metadata/TestHive.java | 43 +++++++++++++++++++ .../hadoop/hive/metastore/HMSHandler.java | 16 ++++--- 2 files changed, 54 insertions(+), 5 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java index f823b324b0c5..bb7f754fc509 100755 --- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -51,7 +52,10 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.events.InsertEvent; +import org.apache.hadoop.hive.ql.ddl.table.partition.PartitionUtils; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; @@ -682,6 +686,45 @@ public void testDropPartitionsWithPurge() throws Exception { } } + @Test + public void testDropMissingPartitionsByFilter() throws Throwable { + String dbName = Warehouse.DEFAULT_DATABASE_NAME; + String tableName = "table_for_testDropMissingPartitionsByFilter"; + + Table table = createPartitionedTable(dbName, tableName); + for (int i = 10; i <= 12; i++) { + Map partitionSpec = new ImmutableMap.Builder() + .put("ds", "20231129") + .put("hr", String.valueOf(i)) + .build(); + hm.createPartition(table, partitionSpec); + } + + List partitions = hm.getPartitions(table); + assertEquals(3, partitions.size()); + + // drop partitions by filter with missing predicate + try { + List> partExprs = new ArrayList<>(); + ExprNodeColumnDesc column = new ExprNodeColumnDesc( + TypeInfoFactory.stringTypeInfo, "ds", null, true); + List values = Arrays.asList("20231130", "20231129"); + for (int i = 0; i < values.size(); i++) { + ExprNodeGenericFuncDesc expr = PartitionUtils.makeBinaryPredicate( + "=", column, new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, values.get(i))); + partExprs.add(Pair.of(i, SerializationUtilities.serializeObjectWithTypeInformation(expr))); + } + hm.dropPartitions(dbName, tableName, partExprs, PartitionDropOptions.instance()); + fail("Expected exception"); + } catch (HiveException e) { + // expected + assertEquals("Some partitions to drop are missing", e.getCause().getMessage()); + assertEquals(3, hm.getPartitions(table).size()); + } finally { + cleanUpTableQuietly(dbName, tableName); + } + } + /** * Test that tables set up with auto-purge skip trash-directory when tables/partitions are dropped. * @throws Throwable diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index a4367d5716ab..3573ed23a7af 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -5183,8 +5183,8 @@ public DropPartitionsResult drop_partitions_req( mustPurge = isMustPurge(envContext, tbl); tableDataShouldBeDeleted = checkTableDataShouldBeDeleted(tbl, deleteData); writeId = getWriteId(envContext); - - int minCount = 0; + + boolean hasMissingParts = false; RequestPartsSpec spec = request.getParts(); List partNames = null; @@ -5192,7 +5192,6 @@ public DropPartitionsResult drop_partitions_req( // Dropping by expressions. parts = new ArrayList<>(spec.getExprs().size()); for (DropPartitionsExpr expr : spec.getExprs()) { - ++minCount; // At least one partition per expression, if not ifExists List result = new ArrayList<>(); boolean hasUnknown = ms.getPartitionsByExpr(catName, dbName, tblName, result, new GetPartitionsArgs.GetPartitionsArgsBuilder() @@ -5213,20 +5212,27 @@ public DropPartitionsResult drop_partitions_req( } } } + if (result.isEmpty()) { + hasMissingParts = true; + if (!ifExists) { + // fail-fast for missing partition expr + break; + } + } parts.addAll(result); } } else if (spec.isSetNames()) { partNames = spec.getNames(); - minCount = partNames.size(); parts = ms.getPartitionsByNames(catName, dbName, tblName, new GetPartitionsArgs.GetPartitionsArgsBuilder() .partNames(partNames).skipColumnSchemaForPartition(request.isSkipColumnSchemaForPartition()) .build()); + hasMissingParts = (parts.size() != partNames.size()); } else { throw new MetaException("Partition spec is not set"); } - if ((parts.size() < minCount) && !ifExists) { + if (hasMissingParts && !ifExists) { throw new NoSuchObjectException("Some partitions to drop are missing"); } From 94f318959ca6ec5f815e6326241bd2269ce91b80 Mon Sep 17 00:00:00 2001 From: Simhadri Govindappa Date: Fri, 26 Jan 2024 15:32:53 +0530 Subject: [PATCH 156/179] HIVE-28020: Iceberg: Upgrade iceberg version to 1.4.3 (#5023)(Simhadri Govindappa, reviewed by Attila Turoczy, Butao Zhang) --- iceberg/patched-iceberg-core/pom.xml | 1 - .../org/apache/iceberg/PartitionsTable.java | 345 ------------------ iceberg/pom.xml | 2 +- 3 files changed, 1 insertion(+), 347 deletions(-) delete mode 100644 iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java diff --git a/iceberg/patched-iceberg-core/pom.xml b/iceberg/patched-iceberg-core/pom.xml index 9095ff93a26c..b0f6b7e96ac9 100644 --- a/iceberg/patched-iceberg-core/pom.xml +++ b/iceberg/patched-iceberg-core/pom.xml @@ -83,7 +83,6 @@ **/HadoopInputFile.class **/SerializableTable.class - **/PartitionsTable.class diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java deleted file mode 100644 index 904270353d69..000000000000 --- a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ /dev/null @@ -1,345 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg; - -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.LoadingCache; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import org.apache.iceberg.expressions.ManifestEvaluator; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ParallelIterable; -import org.apache.iceberg.util.PartitionUtil; -import org.apache.iceberg.util.StructLikeMap; - -/** A {@link Table} implementation that exposes a table's partitions as rows. */ -public class PartitionsTable extends BaseMetadataTable { - - private final Schema schema; - - private final boolean unpartitionedTable; - - PartitionsTable(Table table) { - this(table, table.name() + ".partitions"); - } - - PartitionsTable(Table table, String name) { - super(table, name); - - this.schema = - new Schema( - Types.NestedField.required(1, "partition", Partitioning.partitionType(table)), - Types.NestedField.required(4, "spec_id", Types.IntegerType.get()), - Types.NestedField.required( - 2, "record_count", Types.LongType.get(), "Count of records in data files"), - Types.NestedField.required( - 3, "file_count", Types.IntegerType.get(), "Count of data files"), - Types.NestedField.required( - 11, - "total_data_file_size_in_bytes", - Types.LongType.get(), - "Total size in bytes of data files"), - Types.NestedField.required( - 5, - "position_delete_record_count", - Types.LongType.get(), - "Count of records in position delete files"), - Types.NestedField.required( - 6, - "position_delete_file_count", - Types.IntegerType.get(), - "Count of position delete files"), - Types.NestedField.required( - 7, - "equality_delete_record_count", - Types.LongType.get(), - "Count of records in equality delete files"), - Types.NestedField.required( - 8, - "equality_delete_file_count", - Types.IntegerType.get(), - "Count of equality delete files"), - Types.NestedField.optional( - 9, - "last_updated_at", - Types.TimestampType.withZone(), - "Commit time of snapshot that last updated this partition"), - Types.NestedField.optional( - 10, - "last_updated_snapshot_id", - Types.LongType.get(), - "Id of snapshot that last updated this partition")); - this.unpartitionedTable = Partitioning.partitionType(table).fields().isEmpty(); - } - - @Override - public TableScan newScan() { - return new PartitionsScan(table()); - } - - @Override - public Schema schema() { - if (unpartitionedTable) { - return schema.select( - "record_count", - "file_count", - "total_data_file_size_in_bytes", - "position_delete_record_count", - "position_delete_file_count", - "equality_delete_record_count", - "equality_delete_file_count", - "last_updated_at", - "last_updated_snapshot_id"); - } - return schema; - } - - @Override - MetadataTableType metadataTableType() { - return MetadataTableType.PARTITIONS; - } - - private DataTask task(StaticTableScan scan) { - Iterable partitions = partitions(table(), scan); - if (unpartitionedTable) { - // the table is unpartitioned, partitions contains only the root partition - return StaticDataTask.of( - io().newInputFile(table().operations().current().metadataFileLocation()), - schema(), - scan.schema(), - partitions, - root -> - StaticDataTask.Row.of( - root.dataRecordCount, - root.dataFileCount, - root.dataFileSizeInBytes, - root.posDeleteRecordCount, - root.posDeleteFileCount, - root.eqDeleteRecordCount, - root.eqDeleteFileCount, - root.lastUpdatedAt, - root.lastUpdatedSnapshotId)); - } else { - return StaticDataTask.of( - io().newInputFile(table().operations().current().metadataFileLocation()), - schema(), - scan.schema(), - partitions, - PartitionsTable::convertPartition); - } - } - - private static StaticDataTask.Row convertPartition(Partition partition) { - return StaticDataTask.Row.of( - partition.partitionData, - partition.specId, - partition.dataRecordCount, - partition.dataFileCount, - partition.dataFileSizeInBytes, - partition.posDeleteRecordCount, - partition.posDeleteFileCount, - partition.eqDeleteRecordCount, - partition.eqDeleteFileCount, - partition.lastUpdatedAt, - partition.lastUpdatedSnapshotId); - } - - private static Iterable partitions(Table table, StaticTableScan scan) { - Types.StructType partitionType = Partitioning.partitionType(table); - PartitionMap partitions = new PartitionMap(partitionType); - try (CloseableIterable>> entries = planEntries(scan)) { - for (ManifestEntry> entry : entries) { - Snapshot snapshot = table.snapshot(entry.snapshotId()); - ContentFile file = entry.file(); - StructLike partition = - PartitionUtil.coercePartition( - partitionType, table.specs().get(file.specId()), file.partition()); - partitions.get(partition).update(file, snapshot); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - - return partitions.all(); - } - - @VisibleForTesting - static CloseableIterable> planEntries(StaticTableScan scan) { - Table table = scan.table(); - - CloseableIterable filteredManifests = - filteredManifests(scan, table, scan.snapshot().allManifests(table.io())); - - Iterable>> tasks = - CloseableIterable.transform(filteredManifests, manifest -> readEntries(manifest, scan)); - - return new ParallelIterable<>(tasks, scan.planExecutor()); - } - - private static CloseableIterable> readEntries( - ManifestFile manifest, StaticTableScan scan) { - Table table = scan.table(); - return CloseableIterable.transform( - ManifestFiles.open(manifest, table.io(), table.specs()) - .caseSensitive(scan.isCaseSensitive()) - .select(scanColumns(manifest.content())) // don't select stats columns - .liveEntries(), - t -> - (ManifestEntry>) - // defensive copy of manifest entry without stats columns - t.copyWithoutStats()); - } - - private static List scanColumns(ManifestContent content) { - switch (content) { - case DATA: - return BaseScan.SCAN_COLUMNS; - case DELETES: - return BaseScan.DELETE_SCAN_COLUMNS; - default: - throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content); - } - } - - private static CloseableIterable filteredManifests( - StaticTableScan scan, Table table, List manifestFilesList) { - CloseableIterable manifestFiles = - CloseableIterable.withNoopClose(manifestFilesList); - - LoadingCache evalCache = - Caffeine.newBuilder() - .build( - specId -> { - PartitionSpec spec = table.specs().get(specId); - PartitionSpec transformedSpec = transformSpec(scan.tableSchema(), spec); - return ManifestEvaluator.forRowFilter( - scan.filter(), transformedSpec, scan.isCaseSensitive()); - }); - - return CloseableIterable.filter( - manifestFiles, manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); - } - - private class PartitionsScan extends StaticTableScan { - PartitionsScan(Table table) { - super( - table, - PartitionsTable.this.schema(), - MetadataTableType.PARTITIONS, - PartitionsTable.this::task); - } - } - - static class PartitionMap { - private final StructLikeMap partitions; - private final Types.StructType keyType; - - PartitionMap(Types.StructType type) { - this.partitions = StructLikeMap.create(type); - this.keyType = type; - } - - Partition get(StructLike key) { - Partition partition = partitions.get(key); - if (partition == null) { - partition = new Partition(key, keyType); - partitions.put(key, partition); - } - return partition; - } - - Iterable all() { - return partitions.values(); - } - } - - static class Partition { - private final PartitionData partitionData; - private int specId; - private long dataRecordCount; - private int dataFileCount; - private long dataFileSizeInBytes; - private long posDeleteRecordCount; - private int posDeleteFileCount; - private long eqDeleteRecordCount; - private int eqDeleteFileCount; - private Long lastUpdatedAt; - private Long lastUpdatedSnapshotId; - - Partition(StructLike key, Types.StructType keyType) { - this.partitionData = toPartitionData(key, keyType); - this.specId = 0; - this.dataRecordCount = 0L; - this.dataFileCount = 0; - this.dataFileSizeInBytes = 0L; - this.posDeleteRecordCount = 0L; - this.posDeleteFileCount = 0; - this.eqDeleteRecordCount = 0L; - this.eqDeleteFileCount = 0; - } - - void update(ContentFile file, Snapshot snapshot) { - if (snapshot != null) { - long snapshotCommitTime = snapshot.timestampMillis() * 1000; - if (this.lastUpdatedAt == null || snapshotCommitTime > this.lastUpdatedAt) { - this.lastUpdatedAt = snapshotCommitTime; - this.lastUpdatedSnapshotId = snapshot.snapshotId(); - } - } - - switch (file.content()) { - case DATA: - this.dataRecordCount += file.recordCount(); - this.dataFileCount += 1; - this.specId = file.specId(); - this.dataFileSizeInBytes += file.fileSizeInBytes(); - break; - case POSITION_DELETES: - this.posDeleteRecordCount = file.recordCount(); - this.posDeleteFileCount += 1; - this.specId = file.specId(); - break; - case EQUALITY_DELETES: - this.eqDeleteRecordCount = file.recordCount(); - this.eqDeleteFileCount += 1; - this.specId = file.specId(); - break; - default: - throw new UnsupportedOperationException( - "Unsupported file content type: " + file.content()); - } - } - - /** Needed because StructProjection is not serializable */ - private PartitionData toPartitionData(StructLike key, Types.StructType keyType) { - PartitionData data = new PartitionData(keyType); - for (int i = 0; i < keyType.fields().size(); i++) { - Object val = key.get(i, keyType.fields().get(i).type().typeId().javaClass()); - if (val != null) { - data.set(i, val); - } - } - return data; - } - } -} diff --git a/iceberg/pom.xml b/iceberg/pom.xml index 8a2f3e0ee4a6..ff5260436925 100644 --- a/iceberg/pom.xml +++ b/iceberg/pom.xml @@ -25,7 +25,7 @@ .. . - 1.4.2 + 1.4.3 4.0.3 3.4.4 1.11.3 From 1dc3dc9ede4fa932df6d5a5626f000e720481188 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 30 Jan 2024 14:17:30 +0530 Subject: [PATCH 157/179] HIVE-28017: Add generated protobuf code. (#5018). (Ayush Saxena, reviewed by Butao Zhang, Zhihua Deng, Attila Turoczy) --- .../daemon/rpc/LlapDaemonProtocolProtos.java | 5176 +++++++---------- .../plugin/rpc/LlapPluginProtocolProtos.java | 207 +- .../hive/ql/hooks/proto/HiveHookEvents.java | 372 +- .../hive/ql/io/protobuf/SampleProtos.java | 658 +-- 4 files changed, 2669 insertions(+), 3744 deletions(-) diff --git a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java index b1526cc0771c..d4c7024010d9 100644 --- a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java +++ b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java @@ -269,56 +269,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private UserPayloadProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - bitField0_ |= 0x00000001; - userPayload_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - version_ = input.readInt32(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_UserPayloadProto_descriptor; @@ -391,7 +341,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeInt32(2, version_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -408,7 +358,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt32Size(2, version_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -433,7 +383,7 @@ public boolean equals(final java.lang.Object obj) { if (getVersion() != other.getVersion()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -452,7 +402,7 @@ public int hashCode() { hash = (37 * hash) + VERSION_FIELD_NUMBER; hash = (53 * hash) + getVersion(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -569,18 +519,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -680,7 +625,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasVersion()) { setVersion(other.getVersion()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -695,17 +640,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + userPayload_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + version_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -823,7 +791,18 @@ public UserPayloadProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new UserPayloadProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -919,70 +898,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private EntityDescriptorProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - className_ = bs; - break; - } - case 18: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) != 0)) { - subBuilder = userPayload_.toBuilder(); - } - userPayload_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(userPayload_); - userPayload_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000002; - break; - } - case 26: { - bitField0_ |= 0x00000004; - historyText_ = input.readBytes(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_EntityDescriptorProto_descriptor; @@ -1113,7 +1028,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000004) != 0)) { output.writeBytes(3, historyText_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -1133,7 +1048,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBytesSize(3, historyText_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -1163,7 +1078,7 @@ public boolean equals(final java.lang.Object obj) { if (!getHistoryText() .equals(other.getHistoryText())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -1186,7 +1101,7 @@ public int hashCode() { hash = (37 * hash) + HISTORY_TEXT_FIELD_NUMBER; hash = (53 * hash) + getHistoryText().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -1434,7 +1349,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasHistoryText()) { setHistoryText(other.getHistoryText()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -1449,17 +1364,47 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + className_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getUserPayloadFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + historyText_ = input.readBytes(); + bitField0_ |= 0x00000004; + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -1742,7 +1687,18 @@ public EntityDescriptorProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new EntityDescriptorProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -1837,70 +1793,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private IOSpecProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - connectedVertexName_ = bs; - break; - } - case 18: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) != 0)) { - subBuilder = ioDescriptor_.toBuilder(); - } - ioDescriptor_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(ioDescriptor_); - ioDescriptor_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000002; - break; - } - case 24: { - bitField0_ |= 0x00000004; - physicalEdgeCount_ = input.readInt32(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_IOSpecProto_descriptor; @@ -2031,7 +1923,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000004) != 0)) { output.writeInt32(3, physicalEdgeCount_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -2051,7 +1943,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt32Size(3, physicalEdgeCount_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -2081,7 +1973,7 @@ public boolean equals(final java.lang.Object obj) { if (getPhysicalEdgeCount() != other.getPhysicalEdgeCount()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -2104,7 +1996,7 @@ public int hashCode() { hash = (37 * hash) + PHYSICAL_EDGE_COUNT_FIELD_NUMBER; hash = (53 * hash) + getPhysicalEdgeCount(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -2352,7 +2244,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasPhysicalEdgeCount()) { setPhysicalEdgeCount(other.getPhysicalEdgeCount()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -2367,17 +2259,47 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + connectedVertexName_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getIoDescriptorFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + physicalEdgeCount_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -2657,7 +2579,18 @@ public IOSpecProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new IOSpecProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -2767,77 +2700,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GroupInputSpecProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - groupName_ = bs; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00000002) != 0)) { - groupVertices_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000002; - } - groupVertices_.add(bs); - break; - } - case 26: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) != 0)) { - subBuilder = mergedInputDescriptor_.toBuilder(); - } - mergedInputDescriptor_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(mergedInputDescriptor_); - mergedInputDescriptor_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000002; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000002) != 0)) { - groupVertices_ = groupVertices_.getUnmodifiableView(); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GroupInputSpecProto_descriptor; @@ -2984,7 +2846,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeMessage(3, getMergedInputDescriptor()); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -3008,7 +2870,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(3, getMergedInputDescriptor()); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -3035,7 +2897,7 @@ public boolean equals(final java.lang.Object obj) { if (!getMergedInputDescriptor() .equals(other.getMergedInputDescriptor())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -3058,7 +2920,7 @@ public int hashCode() { hash = (37 * hash) + MERGED_INPUT_DESCRIPTOR_FIELD_NUMBER; hash = (53 * hash) + getMergedInputDescriptor().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -3314,7 +3176,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasMergedInputDescriptor()) { mergeMergedInputDescriptor(other.getMergedInputDescriptor()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -3329,17 +3191,48 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + groupName_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensureGroupVerticesIsMutable(); + groupVertices_.add(bs); + break; + } // case 18 + case 26: { + input.readMessage( + getMergedInputDescriptorFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000004; + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -3689,7 +3582,18 @@ public GroupInputSpecProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GroupInputSpecProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -4015,158 +3919,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SignableVertexSpec( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - user_ = bs; - break; - } - case 16: { - bitField0_ |= 0x00000002; - signatureKeyId_ = input.readInt64(); - break; - } - case 26: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000004) != 0)) { - subBuilder = queryIdentifier_.toBuilder(); - } - queryIdentifier_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(queryIdentifier_); - queryIdentifier_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000004; - break; - } - case 34: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000008; - hiveQueryId_ = bs; - break; - } - case 42: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000010; - dagName_ = bs; - break; - } - case 50: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000020; - vertexName_ = bs; - break; - } - case 56: { - bitField0_ |= 0x00000040; - vertexIndex_ = input.readInt32(); - break; - } - case 66: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000080; - tokenIdentifier_ = bs; - break; - } - case 74: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000100) != 0)) { - subBuilder = processorDescriptor_.toBuilder(); - } - processorDescriptor_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(processorDescriptor_); - processorDescriptor_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000100; - break; - } - case 82: { - if (!((mutable_bitField0_ & 0x00000200) != 0)) { - inputSpecs_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000200; - } - inputSpecs_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, extensionRegistry)); - break; - } - case 90: { - if (!((mutable_bitField0_ & 0x00000400) != 0)) { - outputSpecs_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000400; - } - outputSpecs_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, extensionRegistry)); - break; - } - case 98: { - if (!((mutable_bitField0_ & 0x00000800) != 0)) { - groupedInputSpecs_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000800; - } - groupedInputSpecs_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto.PARSER, extensionRegistry)); - break; - } - case 104: { - bitField0_ |= 0x00000200; - vertexParallelism_ = input.readInt32(); - break; - } - case 112: { - bitField0_ |= 0x00000400; - isExternalSubmission_ = input.readBool(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000200) != 0)) { - inputSpecs_ = java.util.Collections.unmodifiableList(inputSpecs_); - } - if (((mutable_bitField0_ & 0x00000400) != 0)) { - outputSpecs_ = java.util.Collections.unmodifiableList(outputSpecs_); - } - if (((mutable_bitField0_ & 0x00000800) != 0)) { - groupedInputSpecs_ = java.util.Collections.unmodifiableList(groupedInputSpecs_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SignableVertexSpec_descriptor; @@ -4757,7 +4509,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000400) != 0)) { output.writeBool(14, isExternalSubmission_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -4817,7 +4569,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBoolSize(14, isExternalSubmission_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -4893,7 +4645,7 @@ public boolean equals(final java.lang.Object obj) { if (getIsExternalSubmission() != other.getIsExternalSubmission()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -4962,7 +4714,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( getIsExternalSubmission()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -5132,22 +4884,25 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000100); if (inputSpecsBuilder_ == null) { inputSpecs_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000200); } else { + inputSpecs_ = null; inputSpecsBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000200); if (outputSpecsBuilder_ == null) { outputSpecs_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000400); } else { + outputSpecs_ = null; outputSpecsBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000400); if (groupedInputSpecsBuilder_ == null) { groupedInputSpecs_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000800); } else { + groupedInputSpecs_ = null; groupedInputSpecsBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000800); vertexParallelism_ = 0; bitField0_ = (bitField0_ & ~0x00001000); isExternalSubmission_ = false; @@ -5429,7 +5184,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasIsExternalSubmission()) { setIsExternalSubmission(other.getIsExternalSubmission()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -5444,17 +5199,128 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + user_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + signatureKeyId_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 26: { + input.readMessage( + getQueryIdentifierFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + hiveQueryId_ = input.readBytes(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + dagName_ = input.readBytes(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + vertexName_ = input.readBytes(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 56: { + vertexIndex_ = input.readInt32(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 66: { + tokenIdentifier_ = input.readBytes(); + bitField0_ |= 0x00000080; + break; + } // case 66 + case 74: { + input.readMessage( + getProcessorDescriptorFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000100; + break; + } // case 74 + case 82: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, + extensionRegistry); + if (inputSpecsBuilder_ == null) { + ensureInputSpecsIsMutable(); + inputSpecs_.add(m); + } else { + inputSpecsBuilder_.addMessage(m); + } + break; + } // case 82 + case 90: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, + extensionRegistry); + if (outputSpecsBuilder_ == null) { + ensureOutputSpecsIsMutable(); + outputSpecs_.add(m); + } else { + outputSpecsBuilder_.addMessage(m); + } + break; + } // case 90 + case 98: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto.PARSER, + extensionRegistry); + if (groupedInputSpecsBuilder_ == null) { + ensureGroupedInputSpecsIsMutable(); + groupedInputSpecs_.add(m); + } else { + groupedInputSpecsBuilder_.addMessage(m); + } + break; + } // case 98 + case 104: { + vertexParallelism_ = input.readInt32(); + bitField0_ |= 0x00001000; + break; + } // case 104 + case 112: { + isExternalSubmission_ = input.readBool(); + bitField0_ |= 0x00002000; + break; + } // case 112 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -7091,7 +6957,18 @@ public SignableVertexSpec parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SignableVertexSpec(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -7181,64 +7058,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private VertexOrBinary( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = vertex_.toBuilder(); - } - vertex_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(vertex_); - vertex_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 18: { - bitField0_ |= 0x00000002; - vertexBinary_ = input.readBytes(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexOrBinary_descriptor; @@ -7326,7 +7145,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeBytes(2, vertexBinary_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -7343,7 +7162,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBytesSize(2, vertexBinary_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -7368,7 +7187,7 @@ public boolean equals(final java.lang.Object obj) { if (!getVertexBinary() .equals(other.getVertexBinary())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -7387,7 +7206,7 @@ public int hashCode() { hash = (37 * hash) + VERTEXBINARY_FIELD_NUMBER; hash = (53 * hash) + getVertexBinary().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -7628,7 +7447,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasVertexBinary()) { setVertexBinary(other.getVertexBinary()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -7643,17 +7462,42 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getVertexFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + vertexBinary_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -7868,7 +7712,18 @@ public VertexOrBinary parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new VertexOrBinary(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -7985,76 +7840,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private FragmentRuntimeInfo( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - numSelfAndUpstreamTasks_ = input.readInt32(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - numSelfAndUpstreamCompletedTasks_ = input.readInt32(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - withinDagPriority_ = input.readInt32(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - dagStartTime_ = input.readInt64(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - firstAttemptStartTime_ = input.readInt64(); - break; - } - case 48: { - bitField0_ |= 0x00000020; - currentAttemptStartTime_ = input.readInt64(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentRuntimeInfo_descriptor; @@ -8215,7 +8000,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000020) != 0)) { output.writeInt64(6, currentAttemptStartTime_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -8248,7 +8033,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(6, currentAttemptStartTime_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -8293,7 +8078,7 @@ public boolean equals(final java.lang.Object obj) { if (getCurrentAttemptStartTime() != other.getCurrentAttemptStartTime()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -8331,7 +8116,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getCurrentAttemptStartTime()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -8448,18 +8233,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -8595,7 +8375,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasCurrentAttemptStartTime()) { setCurrentAttemptStartTime(other.getCurrentAttemptStartTime()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -8610,17 +8390,60 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + numSelfAndUpstreamTasks_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 16: { + numSelfAndUpstreamCompletedTasks_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 24: { + withinDagPriority_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + dagStartTime_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 40: { + firstAttemptStartTime_ = input.readInt64(); + bitField0_ |= 0x00000010; + break; + } // case 40 + case 48: { + currentAttemptStartTime_ = input.readInt64(); + bitField0_ |= 0x00000020; + break; + } // case 48 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -8891,7 +8714,18 @@ public FragmentRuntimeInfo parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new FragmentRuntimeInfo(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -8982,62 +8816,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private QueryIdentifierProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - applicationIdString_ = bs; - break; - } - case 16: { - bitField0_ |= 0x00000002; - dagIndex_ = input.readInt32(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - appAttemptNumber_ = input.readInt32(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_QueryIdentifierProto_descriptor; @@ -9161,7 +8939,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000004) != 0)) { output.writeInt32(3, appAttemptNumber_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -9181,7 +8959,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt32Size(3, appAttemptNumber_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -9211,7 +8989,7 @@ public boolean equals(final java.lang.Object obj) { if (getAppAttemptNumber() != other.getAppAttemptNumber()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -9234,7 +9012,7 @@ public int hashCode() { hash = (37 * hash) + APP_ATTEMPT_NUMBER_FIELD_NUMBER; hash = (53 * hash) + getAppAttemptNumber(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -9351,18 +9129,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -9473,7 +9246,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasAppAttemptNumber()) { setAppAttemptNumber(other.getAppAttemptNumber()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -9488,17 +9261,45 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + applicationIdString_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + dagIndex_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 16 + case 24: { + appAttemptNumber_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -9697,7 +9498,18 @@ public QueryIdentifierProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new QueryIdentifierProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -9816,68 +9628,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private NotTezEvent( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - bitField0_ |= 0x00000001; - inputEventProtoBytes_ = input.readBytes(); - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - vertexName_ = bs; - break; - } - case 26: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000004; - destInputName_ = bs; - break; - } - case 32: { - bitField0_ |= 0x00000008; - keyId_ = input.readInt32(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_NotTezEvent_descriptor; @@ -10064,7 +9814,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000008) != 0)) { output.writeInt32(4, keyId_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -10087,7 +9837,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt32Size(4, keyId_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -10122,7 +9872,7 @@ public boolean equals(final java.lang.Object obj) { if (getKeyId() != other.getKeyId()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -10149,7 +9899,7 @@ public int hashCode() { hash = (37 * hash) + KEY_ID_FIELD_NUMBER; hash = (53 * hash) + getKeyId(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -10275,18 +10025,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -10408,7 +10153,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasKeyId()) { setKeyId(other.getKeyId()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -10432,17 +10177,50 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + inputEventProtoBytes_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + vertexName_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + destInputName_ = input.readBytes(); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 32: { + keyId_ = input.readInt32(); + bitField0_ |= 0x00000008; + break; + } // case 32 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -10728,7 +10506,18 @@ public NotTezEvent parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new NotTezEvent(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -10994,135 +10783,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SubmitWorkRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = workSpec_.toBuilder(); - } - workSpec_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(workSpec_); - workSpec_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 18: { - bitField0_ |= 0x00000002; - workSpecSignature_ = input.readBytes(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - fragmentNumber_ = input.readInt32(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - attemptNumber_ = input.readInt32(); - break; - } - case 42: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000010; - containerIdString_ = bs; - break; - } - case 50: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000020; - amHost_ = bs; - break; - } - case 56: { - bitField0_ |= 0x00000040; - amPort_ = input.readInt32(); - break; - } - case 66: { - bitField0_ |= 0x00000080; - credentialsBinary_ = input.readBytes(); - break; - } - case 74: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo.Builder subBuilder = null; - if (((bitField0_ & 0x00000100) != 0)) { - subBuilder = fragmentRuntimeInfo_.toBuilder(); - } - fragmentRuntimeInfo_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(fragmentRuntimeInfo_); - fragmentRuntimeInfo_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000100; - break; - } - case 82: { - bitField0_ |= 0x00000200; - initialEventBytes_ = input.readBytes(); - break; - } - case 90: { - bitField0_ |= 0x00000400; - initialEventSignature_ = input.readBytes(); - break; - } - case 96: { - bitField0_ |= 0x00000800; - isGuaranteed_ = input.readBool(); - break; - } - case 106: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00001000; - jwt_ = bs; - break; - } - case 112: { - bitField0_ |= 0x00002000; - isExternalClientRequest_ = input.readBool(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SubmitWorkRequestProto_descriptor; @@ -11588,7 +11248,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00002000) != 0)) { output.writeBool(14, isExternalClientRequest_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -11650,7 +11310,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBoolSize(14, isExternalClientRequest_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -11735,7 +11395,7 @@ public boolean equals(final java.lang.Object obj) { if (getIsExternalClientRequest() != other.getIsExternalClientRequest()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -11804,7 +11464,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( getIsExternalClientRequest()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -12164,7 +11824,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasIsExternalClientRequest()) { setIsExternalClientRequest(other.getIsExternalClientRequest()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -12179,17 +11839,104 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getWorkSpecFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + workSpecSignature_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + fragmentNumber_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + attemptNumber_ = input.readInt32(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 42: { + containerIdString_ = input.readBytes(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + amHost_ = input.readBytes(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 56: { + amPort_ = input.readInt32(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 66: { + credentialsBinary_ = input.readBytes(); + bitField0_ |= 0x00000080; + break; + } // case 66 + case 74: { + input.readMessage( + getFragmentRuntimeInfoFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000100; + break; + } // case 74 + case 82: { + initialEventBytes_ = input.readBytes(); + bitField0_ |= 0x00000200; + break; + } // case 82 + case 90: { + initialEventSignature_ = input.readBytes(); + bitField0_ |= 0x00000400; + break; + } // case 90 + case 96: { + isGuaranteed_ = input.readBool(); + bitField0_ |= 0x00000800; + break; + } // case 96 + case 106: { + jwt_ = input.readBytes(); + bitField0_ |= 0x00001000; + break; + } // case 106 + case 112: { + isExternalClientRequest_ = input.readBool(); + bitField0_ |= 0x00002000; + break; + } // case 112 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -13149,7 +12896,18 @@ public SubmitWorkRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SubmitWorkRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -13245,70 +13003,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private RegisterDagRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - user_ = bs; - break; - } - case 18: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) != 0)) { - subBuilder = queryIdentifier_.toBuilder(); - } - queryIdentifier_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(queryIdentifier_); - queryIdentifier_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000002; - break; - } - case 26: { - bitField0_ |= 0x00000004; - credentialsBinary_ = input.readBytes(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_RegisterDagRequestProto_descriptor; @@ -13443,7 +13137,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000004) != 0)) { output.writeBytes(3, credentialsBinary_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -13463,7 +13157,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBytesSize(3, credentialsBinary_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -13493,7 +13187,7 @@ public boolean equals(final java.lang.Object obj) { if (!getCredentialsBinary() .equals(other.getCredentialsBinary())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -13516,7 +13210,7 @@ public int hashCode() { hash = (37 * hash) + CREDENTIALS_BINARY_FIELD_NUMBER; hash = (53 * hash) + getCredentialsBinary().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -13764,7 +13458,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasCredentialsBinary()) { setCredentialsBinary(other.getCredentialsBinary()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -13782,17 +13476,47 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + user_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + input.readMessage( + getQueryIdentifierFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + credentialsBinary_ = input.readBytes(); + bitField0_ |= 0x00000004; + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -14075,7 +13799,18 @@ public RegisterDagRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new RegisterDagRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -14126,45 +13861,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private RegisterDagResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_RegisterDagResponseProto_descriptor; @@ -14192,7 +13888,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -14201,7 +13897,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -14216,7 +13912,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -14227,7 +13923,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -14344,18 +14040,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -14434,7 +14125,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -14449,17 +14140,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.RegisterDagResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -14495,7 +14199,18 @@ public RegisterDagResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new RegisterDagResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -14576,64 +14291,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SubmitWorkResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - int rawValue = input.readEnum(); - @SuppressWarnings("deprecation") - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmissionStateProto value = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmissionStateProto.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(1, rawValue); - } else { - bitField0_ |= 0x00000001; - submissionState_ = rawValue; - } - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - uniqueNodeId_ = bs; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SubmitWorkResponseProto_descriptor; @@ -14735,7 +14392,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { com.google.protobuf.GeneratedMessageV3.writeString(output, 2, uniqueNodeId_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -14751,7 +14408,7 @@ public int getSerializedSize() { if (((bitField0_ & 0x00000002) != 0)) { size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, uniqueNodeId_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -14775,7 +14432,7 @@ public boolean equals(final java.lang.Object obj) { if (!getUniqueNodeId() .equals(other.getUniqueNodeId())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -14794,7 +14451,7 @@ public int hashCode() { hash = (37 * hash) + UNIQUE_NODE_ID_FIELD_NUMBER; hash = (53 * hash) + getUniqueNodeId().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -14911,18 +14568,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -15024,7 +14676,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc uniqueNodeId_ = other.uniqueNodeId_; onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -15039,17 +14691,47 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + int tmpRaw = input.readEnum(); + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmissionStateProto tmpValue = + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmissionStateProto.forNumber(tmpRaw); + if (tmpValue == null) { + mergeUnknownVarintField(1, tmpRaw); + } else { + submissionState_ = tmpRaw; + bitField0_ |= 0x00000001; + } + break; + } // case 8 + case 18: { + uniqueNodeId_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -15213,7 +14895,18 @@ public SubmitWorkResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SubmitWorkResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -15309,77 +15002,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SourceStateUpdatedRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = queryIdentifier_.toBuilder(); - } - queryIdentifier_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(queryIdentifier_); - queryIdentifier_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - srcName_ = bs; - break; - } - case 24: { - int rawValue = input.readEnum(); - @SuppressWarnings("deprecation") - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto value = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(3, rawValue); - } else { - bitField0_ |= 0x00000004; - state_ = rawValue; - } - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SourceStateUpdatedRequestProto_descriptor; @@ -15510,7 +15132,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000004) != 0)) { output.writeEnum(3, state_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -15530,7 +15152,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeEnumSize(3, state_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -15559,7 +15181,7 @@ public boolean equals(final java.lang.Object obj) { if (hasState()) { if (state_ != other.state_) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -15582,7 +15204,7 @@ public int hashCode() { hash = (37 * hash) + STATE_FIELD_NUMBER; hash = (53 * hash) + state_; } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -15830,7 +15452,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasState()) { setState(other.getState()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -15845,17 +15467,54 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getQueryIdentifierFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + srcName_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + int tmpRaw = input.readEnum(); + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto tmpValue = + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto.forNumber(tmpRaw); + if (tmpValue == null) { + mergeUnknownVarintField(3, tmpRaw); + } else { + state_ = tmpRaw; + bitField0_ |= 0x00000004; + } + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -16139,7 +15798,18 @@ public SourceStateUpdatedRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SourceStateUpdatedRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -16190,45 +15860,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SourceStateUpdatedResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SourceStateUpdatedResponseProto_descriptor; @@ -16256,7 +15887,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -16265,7 +15896,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -16280,7 +15911,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -16291,7 +15922,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -16408,18 +16039,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -16498,7 +16124,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -16513,17 +16139,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -16559,7 +16198,18 @@ public SourceStateUpdatedResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SourceStateUpdatedResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -16636,64 +16286,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private QueryCompleteRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = queryIdentifier_.toBuilder(); - } - queryIdentifier_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(queryIdentifier_); - queryIdentifier_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 16: { - bitField0_ |= 0x00000002; - deleteDelay_ = input.readInt64(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_QueryCompleteRequestProto_descriptor; @@ -16773,7 +16365,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeInt64(2, deleteDelay_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -16790,7 +16382,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(2, deleteDelay_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -16815,7 +16407,7 @@ public boolean equals(final java.lang.Object obj) { if (getDeleteDelay() != other.getDeleteDelay()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -16835,7 +16427,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getDeleteDelay()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -17072,7 +16664,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasDeleteDelay()) { setDeleteDelay(other.getDeleteDelay()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -17087,17 +16679,42 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getQueryIdentifierFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + deleteDelay_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -17293,7 +16910,18 @@ public QueryCompleteRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new QueryCompleteRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -17344,45 +16972,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private QueryCompleteResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_QueryCompleteResponseProto_descriptor; @@ -17410,7 +16999,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -17419,7 +17008,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -17434,7 +17023,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -17445,7 +17034,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -17562,18 +17151,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -17652,7 +17236,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -17667,17 +17251,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -17713,7 +17310,18 @@ public QueryCompleteResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new QueryCompleteResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -17797,65 +17405,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private TerminateFragmentRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = queryIdentifier_.toBuilder(); - } - queryIdentifier_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(queryIdentifier_); - queryIdentifier_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - fragmentIdentifierString_ = bs; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_TerminateFragmentRequestProto_descriptor; @@ -17964,7 +17513,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { com.google.protobuf.GeneratedMessageV3.writeString(output, 2, fragmentIdentifierString_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -17980,7 +17529,7 @@ public int getSerializedSize() { if (((bitField0_ & 0x00000002) != 0)) { size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, fragmentIdentifierString_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -18005,7 +17554,7 @@ public boolean equals(final java.lang.Object obj) { if (!getFragmentIdentifierString() .equals(other.getFragmentIdentifierString())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -18024,7 +17573,7 @@ public int hashCode() { hash = (37 * hash) + FRAGMENT_IDENTIFIER_STRING_FIELD_NUMBER; hash = (53 * hash) + getFragmentIdentifierString().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -18263,7 +17812,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc fragmentIdentifierString_ = other.fragmentIdentifierString_; onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -18278,17 +17827,42 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getQueryIdentifierFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + fragmentIdentifierString_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -18529,7 +18103,18 @@ public TerminateFragmentRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new TerminateFragmentRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -18580,45 +18165,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private TerminateFragmentResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_TerminateFragmentResponseProto_descriptor; @@ -18646,7 +18192,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -18655,7 +18201,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -18670,7 +18216,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -18681,7 +18227,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -18798,18 +18344,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -18888,7 +18429,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -18903,17 +18444,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -18949,7 +18503,18 @@ public TerminateFragmentResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new TerminateFragmentResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -19044,70 +18609,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private UpdateFragmentRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = queryIdentifier_.toBuilder(); - } - queryIdentifier_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryIdentifierProto.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(queryIdentifier_); - queryIdentifier_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - fragmentIdentifierString_ = bs; - break; - } - case 24: { - bitField0_ |= 0x00000004; - isGuaranteed_ = input.readBool(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_UpdateFragmentRequestProto_descriptor; @@ -19238,7 +18739,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000004) != 0)) { output.writeBool(3, isGuaranteed_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -19258,7 +18759,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBoolSize(3, isGuaranteed_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -19288,7 +18789,7 @@ public boolean equals(final java.lang.Object obj) { if (getIsGuaranteed() != other.getIsGuaranteed()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -19312,7 +18813,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( getIsGuaranteed()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -19560,7 +19061,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasIsGuaranteed()) { setIsGuaranteed(other.getIsGuaranteed()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -19575,17 +19076,47 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UpdateFragmentRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getQueryIdentifierFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + fragmentIdentifierString_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + isGuaranteed_ = input.readBool(); + bitField0_ |= 0x00000004; + break; + } // case 24 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UpdateFragmentRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -19865,7 +19396,18 @@ public UpdateFragmentRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new UpdateFragmentRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -19938,56 +19480,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private UpdateFragmentResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - result_ = input.readBool(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - isGuaranteed_ = input.readBool(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_UpdateFragmentResponseProto_descriptor; @@ -20060,7 +19552,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeBool(2, isGuaranteed_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -20077,7 +19569,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBoolSize(2, isGuaranteed_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -20102,7 +19594,7 @@ public boolean equals(final java.lang.Object obj) { if (getIsGuaranteed() != other.getIsGuaranteed()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -20123,7 +19615,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( getIsGuaranteed()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -20240,18 +19732,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UpdateFragmentResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -20351,7 +19838,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasIsGuaranteed()) { setIsGuaranteed(other.getIsGuaranteed()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -20366,17 +19853,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UpdateFragmentResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + result_ = input.readBool(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 16: { + isGuaranteed_ = input.readBool(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UpdateFragmentResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -20491,7 +20001,18 @@ public UpdateFragmentResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new UpdateFragmentResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -20560,52 +20081,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GetTokenRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - appId_ = bs; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GetTokenRequestProto_descriptor; @@ -20685,7 +20160,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000001) != 0)) { com.google.protobuf.GeneratedMessageV3.writeString(output, 1, appId_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -20697,7 +20172,7 @@ public int getSerializedSize() { if (((bitField0_ & 0x00000001) != 0)) { size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, appId_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -20717,7 +20192,7 @@ public boolean equals(final java.lang.Object obj) { if (!getAppId() .equals(other.getAppId())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -20732,7 +20207,7 @@ public int hashCode() { hash = (37 * hash) + APP_ID_FIELD_NUMBER; hash = (53 * hash) + getAppId().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -20849,18 +20324,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -20953,7 +20423,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc appId_ = other.appId_; onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -20968,17 +20438,35 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + appId_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -21099,7 +20587,18 @@ public GetTokenRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GetTokenRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -21162,51 +20661,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GetTokenResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - bitField0_ |= 0x00000001; - token_ = input.readBytes(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GetTokenResponseProto_descriptor; @@ -21257,7 +20711,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000001) != 0)) { output.writeBytes(1, token_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -21270,7 +20724,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBytesSize(1, token_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -21290,7 +20744,7 @@ public boolean equals(final java.lang.Object obj) { if (!getToken() .equals(other.getToken())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -21305,7 +20759,7 @@ public int hashCode() { hash = (37 * hash) + TOKEN_FIELD_NUMBER; hash = (53 * hash) + getToken().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -21422,18 +20876,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -21524,7 +20973,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasToken()) { setToken(other.getToken()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -21539,17 +20988,35 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + token_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -21628,7 +21095,18 @@ public GetTokenResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GetTokenResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -21713,57 +21191,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private LlapOutputSocketInitMessage( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - fragmentId_ = bs; - break; - } - case 18: { - bitField0_ |= 0x00000002; - token_ = input.readBytes(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_LlapOutputSocketInitMessage_descriptor; @@ -21869,7 +21296,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeBytes(2, token_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -21885,7 +21312,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeBytesSize(2, token_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -21910,7 +21337,7 @@ public boolean equals(final java.lang.Object obj) { if (!getToken() .equals(other.getToken())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -21929,7 +21356,7 @@ public int hashCode() { hash = (37 * hash) + TOKEN_FIELD_NUMBER; hash = (53 * hash) + getToken().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -22050,18 +21477,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.LlapOutputSocketInitMessage.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -22163,7 +21585,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasToken()) { setToken(other.getToken()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -22181,17 +21603,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.LlapOutputSocketInitMessage parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + fragmentId_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + token_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.LlapOutputSocketInitMessage) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -22354,7 +21799,18 @@ public LlapOutputSocketInitMessage parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new LlapOutputSocketInitMessage(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -22405,45 +21861,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private PurgeCacheRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_PurgeCacheRequestProto_descriptor; @@ -22471,7 +21888,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -22480,7 +21897,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -22495,7 +21912,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -22506,7 +21923,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -22623,18 +22040,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -22713,7 +22125,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -22728,17 +22140,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -22774,7 +22199,18 @@ public PurgeCacheRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new PurgeCacheRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -22836,51 +22272,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private PurgeCacheResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - purgedMemoryBytes_ = input.readInt64(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_PurgeCacheResponseProto_descriptor; @@ -22931,7 +22322,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000001) != 0)) { output.writeInt64(1, purgedMemoryBytes_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -22944,7 +22335,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(1, purgedMemoryBytes_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -22964,7 +22355,7 @@ public boolean equals(final java.lang.Object obj) { if (getPurgedMemoryBytes() != other.getPurgedMemoryBytes()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -22980,7 +22371,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getPurgedMemoryBytes()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -23097,18 +22488,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -23199,7 +22585,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasPurgedMemoryBytes()) { setPurgedMemoryBytes(other.getPurgedMemoryBytes()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -23214,17 +22600,35 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + purgedMemoryBytes_ = input.readInt64(); + bitField0_ |= 0x00000001; + break; + } // case 8 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.PurgeCacheResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -23300,7 +22704,18 @@ public PurgeCacheResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new PurgeCacheResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -23380,57 +22795,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private MapEntry( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - key_ = bs; - break; - } - case 16: { - bitField0_ |= 0x00000002; - value_ = input.readInt64(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_MapEntry_descriptor; @@ -23532,7 +22896,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeInt64(2, value_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -23548,7 +22912,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(2, value_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -23573,7 +22937,7 @@ public boolean equals(final java.lang.Object obj) { if (getValue() != other.getValue()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -23593,7 +22957,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getValue()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -23710,18 +23074,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.MapEntry.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -23823,7 +23182,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasValue()) { setValue(other.getValue()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -23838,17 +23197,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.MapEntry parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + key_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 16: { + value_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.MapEntry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -24008,7 +23390,18 @@ public MapEntry parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new MapEntry(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -24059,45 +23452,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GetDaemonMetricsRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GetDaemonMetricsRequestProto_descriptor; @@ -24125,7 +23479,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -24134,7 +23488,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -24149,7 +23503,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -24160,7 +23514,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -24277,18 +23631,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -24367,7 +23716,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -24382,17 +23731,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -24428,7 +23790,18 @@ public GetDaemonMetricsRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GetDaemonMetricsRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -24504,58 +23877,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GetDaemonMetricsResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - metrics_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - metrics_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.MapEntry.PARSER, extensionRegistry)); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - metrics_ = java.util.Collections.unmodifiableList(metrics_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GetDaemonMetricsResponseProto_descriptor; @@ -24626,7 +23947,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < metrics_.size(); i++) { output.writeMessage(1, metrics_.get(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -24639,7 +23960,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, metrics_.get(i)); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -24656,7 +23977,7 @@ public boolean equals(final java.lang.Object obj) { if (!getMetricsList() .equals(other.getMetricsList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -24671,7 +23992,7 @@ public int hashCode() { hash = (37 * hash) + METRICS_FIELD_NUMBER; hash = (53 * hash) + getMetricsList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -24788,29 +24109,24 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getMetricsFieldBuilder(); - } + } @java.lang.Override public Builder clear() { super.clear(); if (metricsBuilder_ == null) { metrics_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); } else { + metrics_ = null; metricsBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000001); return this; } @@ -24921,7 +24237,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc } } } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -24936,17 +24252,43 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.MapEntry m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.MapEntry.PARSER, + extensionRegistry); + if (metricsBuilder_ == null) { + ensureMetricsIsMutable(); + metrics_.add(m); + } else { + metricsBuilder_.addMessage(m); + } + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetDaemonMetricsResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -25223,7 +24565,18 @@ public GetDaemonMetricsResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GetDaemonMetricsResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -25296,56 +24649,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SetCapacityRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - executorNum_ = input.readInt32(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - queueSize_ = input.readInt32(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SetCapacityRequestProto_descriptor; @@ -25418,7 +24721,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeInt32(2, queueSize_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -25435,7 +24738,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt32Size(2, queueSize_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -25460,7 +24763,7 @@ public boolean equals(final java.lang.Object obj) { if (getQueueSize() != other.getQueueSize()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -25479,7 +24782,7 @@ public int hashCode() { hash = (37 * hash) + QUEUESIZE_FIELD_NUMBER; hash = (53 * hash) + getQueueSize(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -25596,18 +24899,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -25707,7 +25005,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasQueueSize()) { setQueueSize(other.getQueueSize()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -25722,17 +25020,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + executorNum_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 16: { + queueSize_ = input.readInt32(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -25847,7 +25168,18 @@ public SetCapacityRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SetCapacityRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -25898,45 +25230,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private SetCapacityResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_SetCapacityResponseProto_descriptor; @@ -25964,7 +25257,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -25973,7 +25266,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -25988,7 +25281,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -25999,7 +25292,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -26116,18 +25409,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -26206,7 +25494,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -26221,17 +25509,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SetCapacityResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -26267,7 +25568,18 @@ public SetCapacityResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new SetCapacityResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -26361,67 +25673,9 @@ protected java.lang.Object newInstance( } @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private EvictEntityRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - dbName_ = bs; - break; - } - case 18: { - if (!((mutable_bitField0_ & 0x00000002) != 0)) { - table_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - table_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TableProto.PARSER, extensionRegistry)); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000002) != 0)) { - table_ = java.util.Collections.unmodifiableList(table_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { @@ -26555,7 +25809,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < table_.size(); i++) { output.writeMessage(2, table_.get(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -26571,7 +25825,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, table_.get(i)); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -26593,7 +25847,7 @@ public boolean equals(final java.lang.Object obj) { } if (!getTableList() .equals(other.getTableList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -26612,7 +25866,7 @@ public int hashCode() { hash = (37 * hash) + TABLE_FIELD_NUMBER; hash = (53 * hash) + getTableList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -26733,19 +25987,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EvictEntityRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getTableFieldBuilder(); - } + } @java.lang.Override public Builder clear() { @@ -26754,10 +26002,11 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000001); if (tableBuilder_ == null) { table_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); } else { + table_ = null; tableBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -26879,7 +26128,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc } } } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -26902,17 +26151,48 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EvictEntityRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + dbName_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TableProto m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TableProto.PARSER, + extensionRegistry); + if (tableBuilder_ == null) { + ensureTableIsMutable(); + table_.add(m); + } else { + tableBuilder_.addMessage(m); + } + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EvictEntityRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -27273,7 +26553,18 @@ public EvictEntityRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new EvictEntityRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -27401,76 +26692,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private TableProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - tableName_ = bs; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00000002) != 0)) { - partKey_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000002; - } - partKey_.add(bs); - break; - } - case 26: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00000004) != 0)) { - partVal_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000004; - } - partVal_.add(bs); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000002) != 0)) { - partKey_ = partKey_.getUnmodifiableView(); - } - if (((mutable_bitField0_ & 0x00000004) != 0)) { - partVal_ = partVal_.getUnmodifiableView(); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_TableProto_descriptor; @@ -27630,7 +26851,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < partVal_.size(); i++) { com.google.protobuf.GeneratedMessageV3.writeString(output, 3, partVal_.getRaw(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -27658,7 +26879,7 @@ public int getSerializedSize() { size += dataSize; size += 1 * getPartValList().size(); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -27682,7 +26903,7 @@ public boolean equals(final java.lang.Object obj) { .equals(other.getPartKeyList())) return false; if (!getPartValList() .equals(other.getPartValList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -27705,7 +26926,7 @@ public int hashCode() { hash = (37 * hash) + PART_VAL_FIELD_NUMBER; hash = (53 * hash) + getPartValList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -27829,18 +27050,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TableProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -27967,7 +27183,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc } onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -27985,17 +27201,47 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TableProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + tableName_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensurePartKeyIsMutable(); + partKey_.add(bs); + break; + } // case 18 + case 26: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensurePartValIsMutable(); + partVal_.add(bs); + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TableProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -28334,7 +27580,18 @@ public TableProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new TableProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -28396,51 +27653,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private EvictEntityResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - evictedBytes_ = input.readInt64(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_EvictEntityResponseProto_descriptor; @@ -28495,7 +27707,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000001) != 0)) { output.writeInt64(1, evictedBytes_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -28508,7 +27720,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(1, evictedBytes_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -28528,7 +27740,7 @@ public boolean equals(final java.lang.Object obj) { if (getEvictedBytes() != other.getEvictedBytes()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -28544,7 +27756,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getEvictedBytes()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -28661,18 +27873,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EvictEntityResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -28763,7 +27970,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasEvictedBytes()) { setEvictedBytes(other.getEvictedBytes()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -28781,17 +27988,35 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EvictEntityResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + evictedBytes_ = input.readInt64(); + bitField0_ |= 0x00000001; + break; + } // case 8 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EvictEntityResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -28867,7 +28092,18 @@ public EvictEntityResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new EvictEntityResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -28918,45 +28154,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GetCacheContentRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GetCacheContentRequestProto_descriptor; @@ -28984,7 +28181,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -28993,7 +28190,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -29008,7 +28205,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -29019,7 +28216,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -29136,18 +28333,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -29226,7 +28418,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto other) { if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -29241,17 +28433,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -29287,7 +28492,18 @@ public GetCacheContentRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GetCacheContentRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -29353,59 +28569,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private GetCacheContentResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryList.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = result_.toBuilder(); - } - result_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryList.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(result_); - result_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_GetCacheContentResponseProto_descriptor; @@ -29463,7 +28626,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000001) != 0)) { output.writeMessage(1, getResult()); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -29476,7 +28639,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, getResult()); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -29496,7 +28659,7 @@ public boolean equals(final java.lang.Object obj) { if (!getResult() .equals(other.getResult())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -29511,7 +28674,7 @@ public int hashCode() { hash = (37 * hash) + RESULT_FIELD_NUMBER; hash = (53 * hash) + getResult().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -29739,7 +28902,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasResult()) { mergeResult(other.getResult()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -29754,17 +28917,37 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + input.readMessage( + getResultFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000001; + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetCacheContentResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -29921,7 +29104,18 @@ public GetCacheContentResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new GetCacheContentResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -29997,58 +29191,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private CacheEntryList( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - entries_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - entries_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntry.PARSER, extensionRegistry)); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - entries_ = java.util.Collections.unmodifiableList(entries_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_CacheEntryList_descriptor; @@ -30119,7 +29261,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < entries_.size(); i++) { output.writeMessage(1, entries_.get(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -30132,7 +29274,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, entries_.get(i)); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -30149,7 +29291,7 @@ public boolean equals(final java.lang.Object obj) { if (!getEntriesList() .equals(other.getEntriesList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -30164,7 +29306,7 @@ public int hashCode() { hash = (37 * hash) + ENTRIES_FIELD_NUMBER; hash = (53 * hash) + getEntriesList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -30281,29 +29423,24 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryList.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getEntriesFieldBuilder(); - } + } @java.lang.Override public Builder clear() { super.clear(); if (entriesBuilder_ == null) { entries_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); } else { + entries_ = null; entriesBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000001); return this; } @@ -30414,7 +29551,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc } } } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -30429,17 +29566,43 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryList parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntry m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntry.PARSER, + extensionRegistry); + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.add(m); + } else { + entriesBuilder_.addMessage(m); + } + break; + } // case 10 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryList) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -30716,7 +29879,18 @@ public CacheEntryList parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new CacheEntryList(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -30837,82 +30011,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private CacheEntry( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - bitField0_ |= 0x00000001; - fileKey_ = input.readBytes(); - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - filePath_ = bs; - break; - } - case 26: { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheTag.Builder subBuilder = null; - if (((bitField0_ & 0x00000004) != 0)) { - subBuilder = cacheTag_.toBuilder(); - } - cacheTag_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheTag.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(cacheTag_); - cacheTag_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000004; - break; - } - case 34: { - if (!((mutable_bitField0_ & 0x00000008) != 0)) { - ranges_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000008; - } - ranges_.add( - input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryRange.PARSER, extensionRegistry)); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000008) != 0)) { - ranges_ = java.util.Collections.unmodifiableList(ranges_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_CacheEntry_descriptor; @@ -31086,7 +30184,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < ranges_.size(); i++) { output.writeMessage(4, ranges_.get(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -31110,7 +30208,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(4, ranges_.get(i)); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -31142,7 +30240,7 @@ public boolean equals(final java.lang.Object obj) { } if (!getRangesList() .equals(other.getRangesList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -31169,7 +30267,7 @@ public int hashCode() { hash = (37 * hash) + RANGES_FIELD_NUMBER; hash = (53 * hash) + getRangesList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -31316,10 +30414,11 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000004); if (rangesBuilder_ == null) { ranges_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); } else { + ranges_ = null; rangesBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -31459,7 +30558,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc } } } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -31474,17 +30573,60 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntry parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + fileKey_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + filePath_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 26: { + input.readMessage( + getCacheTagFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000004; + break; + } // case 26 + case 34: { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryRange m = + input.readMessage( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryRange.PARSER, + extensionRegistry); + if (rangesBuilder_ == null) { + ensureRangesIsMutable(); + ranges_.add(m); + } else { + rangesBuilder_.addMessage(m); + } + break; + } // case 34 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -32007,7 +31149,18 @@ public CacheEntry parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new CacheEntry(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -32102,64 +31255,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private CacheTag( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - tableName_ = bs; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00000002) != 0)) { - partitionDesc_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000002; - } - partitionDesc_.add(bs); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000002) != 0)) { - partitionDesc_ = partitionDesc_.getUnmodifiableView(); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_CacheTag_descriptor; @@ -32277,7 +31372,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < partitionDesc_.size(); i++) { com.google.protobuf.GeneratedMessageV3.writeString(output, 2, partitionDesc_.getRaw(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -32297,7 +31392,7 @@ public int getSerializedSize() { size += dataSize; size += 1 * getPartitionDescList().size(); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -32319,7 +31414,7 @@ public boolean equals(final java.lang.Object obj) { } if (!getPartitionDescList() .equals(other.getPartitionDescList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -32338,7 +31433,7 @@ public int hashCode() { hash = (37 * hash) + PARTITION_DESC_FIELD_NUMBER; hash = (53 * hash) + getPartitionDescList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -32455,18 +31550,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheTag.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -32576,7 +31666,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc } onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -32591,17 +31681,41 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheTag parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + tableName_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensurePartitionDescIsMutable(); + partitionDesc_.add(bs); + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheTag) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -32831,7 +31945,18 @@ public CacheTag parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new CacheTag(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -32904,56 +32029,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private CacheEntryRange( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - start_ = input.readInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - end_ = input.readInt64(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_CacheEntryRange_descriptor; @@ -33026,7 +32101,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { output.writeInt64(2, end_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -33043,7 +32118,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(2, end_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -33068,7 +32143,7 @@ public boolean equals(final java.lang.Object obj) { if (getEnd() != other.getEnd()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -33089,7 +32164,7 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getEnd()); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -33206,18 +32281,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryRange.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -33317,7 +32387,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasEnd()) { setEnd(other.getEnd()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -33332,17 +32402,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryRange parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + start_ = input.readInt64(); + bitField0_ |= 0x00000001; + break; + } // case 8 + case 16: { + end_ = input.readInt64(); + bitField0_ |= 0x00000002; + break; + } // case 16 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.CacheEntryRange) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -33457,7 +32550,18 @@ public CacheEntryRange parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new CacheEntryRange(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; diff --git a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/plugin/rpc/LlapPluginProtocolProtos.java b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/plugin/rpc/LlapPluginProtocolProtos.java index adb65cf27c27..08770605f832 100644 --- a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/plugin/rpc/LlapPluginProtocolProtos.java +++ b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/plugin/rpc/LlapPluginProtocolProtos.java @@ -56,51 +56,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private UpdateQueryRequestProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 8: { - bitField0_ |= 0x00000001; - guaranteedTaskCount_ = input.readInt32(); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.internal_static_UpdateQueryRequestProto_descriptor; @@ -151,7 +106,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000001) != 0)) { output.writeInt32(1, guaranteedTaskCount_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -164,7 +119,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt32Size(1, guaranteedTaskCount_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -184,7 +139,7 @@ public boolean equals(final java.lang.Object obj) { if (getGuaranteedTaskCount() != other.getGuaranteedTaskCount()) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -199,7 +154,7 @@ public int hashCode() { hash = (37 * hash) + GUARANTEED_TASK_COUNT_FIELD_NUMBER; hash = (53 * hash) + getGuaranteedTaskCount(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -316,18 +271,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryRequestProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -418,7 +368,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtoc if (other.hasGuaranteedTaskCount()) { setGuaranteedTaskCount(other.getGuaranteedTaskCount()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -433,17 +383,35 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryRequestProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: { + guaranteedTaskCount_ = input.readInt32(); + bitField0_ |= 0x00000001; + break; + } // case 8 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryRequestProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -519,7 +487,18 @@ public UpdateQueryRequestProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new UpdateQueryRequestProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -570,45 +549,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private UpdateQueryResponseProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.internal_static_UpdateQueryResponseProto_descriptor; @@ -636,7 +576,7 @@ public final boolean isInitialized() { @java.lang.Override public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -645,7 +585,7 @@ public int getSerializedSize() { if (size != -1) return size; size = 0; - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -660,7 +600,7 @@ public boolean equals(final java.lang.Object obj) { } org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto other = (org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto) obj; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -671,7 +611,7 @@ public int hashCode() { } int hash = 41; hash = (19 * hash) + getDescriptor().hashCode(); - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -788,18 +728,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -878,7 +813,7 @@ public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto other) { if (other == org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto.getDefaultInstance()) return this; - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -893,17 +828,30 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.llap.plugin.rpc.LlapPluginProtocolProtos.UpdateQueryResponseProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } @java.lang.Override @@ -939,7 +887,18 @@ public UpdateQueryResponseProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new UpdateQueryResponseProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; diff --git a/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java b/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java index 3f6bef80a8a2..2c2248323997 100644 --- a/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java +++ b/ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/hooks/proto/HiveHookEvents.java @@ -81,58 +81,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private MapFieldEntry( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - key_ = bs; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - value_ = bs; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_MapFieldEntry_descriptor; @@ -263,7 +211,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { com.google.protobuf.GeneratedMessageV3.writeString(output, 2, value_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -278,7 +226,7 @@ public int getSerializedSize() { if (((bitField0_ & 0x00000002) != 0)) { size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, value_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -303,7 +251,7 @@ public boolean equals(final java.lang.Object obj) { if (!getValue() .equals(other.getValue())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -322,7 +270,7 @@ public int hashCode() { hash = (37 * hash) + VALUE_FIELD_NUMBER; hash = (53 * hash) + getValue().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -439,18 +387,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -554,7 +497,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.Ma value_ = other.value_; onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -569,17 +512,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + key_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + value_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -784,7 +750,18 @@ public MapFieldEntry parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new MapFieldEntry(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -1049,129 +1026,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private HiveHookEventProto( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - eventType_ = bs; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - hiveQueryId_ = bs; - break; - } - case 24: { - bitField0_ |= 0x00000004; - timestamp_ = input.readInt64(); - break; - } - case 34: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000008; - executionMode_ = bs; - break; - } - case 42: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000010; - requestUser_ = bs; - break; - } - case 50: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000020; - queue_ = bs; - break; - } - case 58: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000040; - user_ = bs; - break; - } - case 66: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000080; - operationId_ = bs; - break; - } - case 74: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00000100) != 0)) { - tablesWritten_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000100; - } - tablesWritten_.add(bs); - break; - } - case 82: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00000200) != 0)) { - tablesRead_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00000200; - } - tablesRead_.add(bs); - break; - } - case 402: { - if (!((mutable_bitField0_ & 0x00000400) != 0)) { - otherInfo_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000400; - } - otherInfo_.add( - input.readMessage(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.PARSER, extensionRegistry)); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000100) != 0)) { - tablesWritten_ = tablesWritten_.getUnmodifiableView(); - } - if (((mutable_bitField0_ & 0x00000200) != 0)) { - tablesRead_ = tablesRead_.getUnmodifiableView(); - } - if (((mutable_bitField0_ & 0x00000400) != 0)) { - otherInfo_ = java.util.Collections.unmodifiableList(otherInfo_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.internal_static_HiveHookEventProto_descriptor; @@ -1698,7 +1552,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < otherInfo_.size(); i++) { output.writeMessage(50, otherInfo_.get(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -1752,7 +1606,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(50, otherInfo_.get(i)); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -1813,7 +1667,7 @@ public boolean equals(final java.lang.Object obj) { .equals(other.getTablesReadList())) return false; if (!getOtherInfoList() .equals(other.getOtherInfoList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -1869,7 +1723,7 @@ public int hashCode() { hash = (37 * hash) + OTHERINFO_FIELD_NUMBER; hash = (53 * hash) + getOtherInfoList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -1986,19 +1840,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - getOtherInfoFieldBuilder(); - } + } @java.lang.Override public Builder clear() { @@ -2025,10 +1873,11 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000200); if (otherInfoBuilder_ == null) { otherInfo_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000400); } else { + otherInfo_ = null; otherInfoBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000400); return this; } @@ -2241,7 +2090,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.Hi } } } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -2256,17 +2105,95 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + eventType_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + hiveQueryId_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + timestamp_ = input.readInt64(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 34: { + executionMode_ = input.readBytes(); + bitField0_ |= 0x00000008; + break; + } // case 34 + case 42: { + requestUser_ = input.readBytes(); + bitField0_ |= 0x00000010; + break; + } // case 42 + case 50: { + queue_ = input.readBytes(); + bitField0_ |= 0x00000020; + break; + } // case 50 + case 58: { + user_ = input.readBytes(); + bitField0_ |= 0x00000040; + break; + } // case 58 + case 66: { + operationId_ = input.readBytes(); + bitField0_ |= 0x00000080; + break; + } // case 66 + case 74: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensureTablesWrittenIsMutable(); + tablesWritten_.add(bs); + break; + } // case 74 + case 82: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensureTablesReadIsMutable(); + tablesRead_.add(bs); + break; + } // case 82 + case 402: { + org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry m = + input.readMessage( + org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.MapFieldEntry.PARSER, + extensionRegistry); + if (otherInfoBuilder_ == null) { + ensureOtherInfoIsMutable(); + otherInfo_.add(m); + } else { + otherInfoBuilder_.addMessage(m); + } + break; + } // case 402 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents.HiveHookEventProto) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -3388,7 +3315,18 @@ public HiveHookEventProto parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new HiveHookEventProto(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; diff --git a/ql/src/gen/protobuf/gen-test/org/apache/hadoop/hive/ql/io/protobuf/SampleProtos.java b/ql/src/gen/protobuf/gen-test/org/apache/hadoop/hive/ql/io/protobuf/SampleProtos.java index 23202ea491c5..d93abd50d15e 100644 --- a/ql/src/gen/protobuf/gen-test/org/apache/hadoop/hive/ql/io/protobuf/SampleProtos.java +++ b/ql/src/gen/protobuf/gen-test/org/apache/hadoop/hive/ql/io/protobuf/SampleProtos.java @@ -81,58 +81,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private MapFieldEntry( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000001; - key_ = bs; - break; - } - case 18: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00000002; - value_ = bs; - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.internal_static_MapFieldEntry_descriptor; @@ -263,7 +211,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000002) != 0)) { com.google.protobuf.GeneratedMessageV3.writeString(output, 2, value_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -278,7 +226,7 @@ public int getSerializedSize() { if (((bitField0_ & 0x00000002) != 0)) { size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, value_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -303,7 +251,7 @@ public boolean equals(final java.lang.Object obj) { if (!getValue() .equals(other.getValue())) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -322,7 +270,7 @@ public int hashCode() { hash = (37 * hash) + VALUE_FIELD_NUMBER; hash = (53 * hash) + getValue().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -439,18 +387,13 @@ public static final class Builder extends // Construct using org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.newBuilder() private Builder() { - maybeForceBuilderInitialization(); + } private Builder( com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessageV3 - .alwaysUseFieldBuilders) { - } + } @java.lang.Override public Builder clear() { @@ -554,7 +497,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapF value_ = other.value_; onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -569,17 +512,40 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + key_ = input.readBytes(); + bitField0_ |= 0x00000001; + break; + } // case 10 + case 18: { + value_ = input.readBytes(); + bitField0_ |= 0x00000002; + break; + } // case 18 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -784,7 +750,18 @@ public MapFieldEntry parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new MapFieldEntry(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -893,95 +870,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private Mesg1( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 10: { - if (!((mutable_bitField0_ & 0x00000001) != 0)) { - anotherMap_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - anotherMap_.add( - input.readMessage(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.PARSER, extensionRegistry)); - break; - } - case 18: { - org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) != 0)) { - subBuilder = noMap_.toBuilder(); - } - noMap_ = input.readMessage(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(noMap_); - noMap_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 24: { - if (!((mutable_bitField0_ & 0x00000004) != 0)) { - intList_ = newIntList(); - mutable_bitField0_ |= 0x00000004; - } - intList_.addInt(input.readInt32()); - break; - } - case 26: { - int length = input.readRawVarint32(); - int limit = input.pushLimit(length); - if (!((mutable_bitField0_ & 0x00000004) != 0) && input.getBytesUntilLimit() > 0) { - intList_ = newIntList(); - mutable_bitField0_ |= 0x00000004; - } - while (input.getBytesUntilLimit() > 0) { - intList_.addInt(input.readInt32()); - } - input.popLimit(limit); - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00000001) != 0)) { - anotherMap_ = java.util.Collections.unmodifiableList(anotherMap_); - } - if (((mutable_bitField0_ & 0x00000004) != 0)) { - intList_.makeImmutable(); // C - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.internal_static_Mesg1_descriptor; @@ -1112,7 +1000,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < intList_.size(); i++) { output.writeInt32(3, intList_.getInt(i)); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -1138,7 +1026,7 @@ public int getSerializedSize() { size += dataSize; size += 1 * getIntListList().size(); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -1162,7 +1050,7 @@ public boolean equals(final java.lang.Object obj) { } if (!getIntListList() .equals(other.getIntListList())) return false; - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -1185,7 +1073,7 @@ public int hashCode() { hash = (37 * hash) + INTLIST_FIELD_NUMBER; hash = (53 * hash) + getIntListList().hashCode(); } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -1322,10 +1210,11 @@ public Builder clear() { super.clear(); if (anotherMapBuilder_ == null) { anotherMap_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); } else { + anotherMap_ = null; anotherMapBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00000001); if (noMapBuilder_ == null) { noMap_ = null; } else { @@ -1472,7 +1361,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg } onChanged(); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -1487,17 +1376,66 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1 parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: { + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry m = + input.readMessage( + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.PARSER, + extensionRegistry); + if (anotherMapBuilder_ == null) { + ensureAnotherMapIsMutable(); + anotherMap_.add(m); + } else { + anotherMapBuilder_.addMessage(m); + } + break; + } // case 10 + case 18: { + input.readMessage( + getNoMapFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00000002; + break; + } // case 18 + case 24: { + int v = input.readInt32(); + ensureIntListIsMutable(); + intList_.addInt(v); + break; + } // case 24 + case 26: { + int length = input.readRawVarint32(); + int limit = input.pushLimit(length); + ensureIntListIsMutable(); + while (input.getBytesUntilLimit() > 0) { + intList_.addInt(input.readInt32()); + } + input.popLimit(limit); + break; + } // case 26 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -1973,7 +1911,18 @@ public Mesg1 parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new Mesg1(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; @@ -2300,183 +2249,6 @@ protected java.lang.Object newInstance( getUnknownFields() { return this.unknownFields; } - private AllTypes( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - this(); - if (extensionRegistry == null) { - throw new java.lang.NullPointerException(); - } - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { - int tag = input.readTag(); - switch (tag) { - case 0: - done = true; - break; - case 9: { - bitField0_ |= 0x00000001; - doubleType_ = input.readDouble(); - break; - } - case 21: { - bitField0_ |= 0x00000002; - floatType_ = input.readFloat(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - int32Type_ = input.readInt32(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - int64Type_ = input.readInt64(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - uint32Type_ = input.readUInt32(); - break; - } - case 48: { - bitField0_ |= 0x00000020; - uint64Type_ = input.readUInt64(); - break; - } - case 56: { - bitField0_ |= 0x00000040; - sint32Type_ = input.readSInt32(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - sint64Type_ = input.readSInt64(); - break; - } - case 77: { - bitField0_ |= 0x00000100; - fixed32Type_ = input.readFixed32(); - break; - } - case 81: { - bitField0_ |= 0x00000200; - fixed64Type_ = input.readFixed64(); - break; - } - case 93: { - bitField0_ |= 0x00000400; - sfixed32Type_ = input.readSFixed32(); - break; - } - case 97: { - bitField0_ |= 0x00000800; - sfixed64Type_ = input.readSFixed64(); - break; - } - case 104: { - bitField0_ |= 0x00001000; - boolType_ = input.readBool(); - break; - } - case 114: { - com.google.protobuf.ByteString bs = input.readBytes(); - bitField0_ |= 0x00002000; - stringType_ = bs; - break; - } - case 122: { - bitField0_ |= 0x00004000; - bytesType_ = input.readBytes(); - break; - } - case 130: { - if (!((mutable_bitField0_ & 0x00008000) != 0)) { - mapType_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00008000; - } - mapType_.add( - input.readMessage(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.PARSER, extensionRegistry)); - break; - } - case 138: { - com.google.protobuf.ByteString bs = input.readBytes(); - if (!((mutable_bitField0_ & 0x00010000) != 0)) { - stringListType_ = new com.google.protobuf.LazyStringArrayList(); - mutable_bitField0_ |= 0x00010000; - } - stringListType_.add(bs); - break; - } - case 146: { - org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1.Builder subBuilder = null; - if (((bitField0_ & 0x00008000) != 0)) { - subBuilder = messageType_.toBuilder(); - } - messageType_ = input.readMessage(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(messageType_); - messageType_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00008000; - break; - } - case 154: { - if (!((mutable_bitField0_ & 0x00040000) != 0)) { - messageListType_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00040000; - } - messageListType_.add( - input.readMessage(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1.PARSER, extensionRegistry)); - break; - } - case 160: { - int rawValue = input.readEnum(); - @SuppressWarnings("deprecation") - org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllTypes.Enum1 value = org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllTypes.Enum1.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(20, rawValue); - } else { - bitField0_ |= 0x00010000; - enumType_ = rawValue; - } - break; - } - default: { - if (!parseUnknownField( - input, unknownFields, extensionRegistry, tag)) { - done = true; - } - break; - } - } - } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (com.google.protobuf.UninitializedMessageException e) { - throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e).setUnfinishedMessage(this); - } finally { - if (((mutable_bitField0_ & 0x00008000) != 0)) { - mapType_ = java.util.Collections.unmodifiableList(mapType_); - } - if (((mutable_bitField0_ & 0x00010000) != 0)) { - stringListType_ = stringListType_.getUnmodifiableView(); - } - if (((mutable_bitField0_ & 0x00040000) != 0)) { - messageListType_ = java.util.Collections.unmodifiableList(messageListType_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { return org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.internal_static_AllTypes_descriptor; @@ -3135,7 +2907,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00010000) != 0)) { output.writeEnum(20, enumType_); } - unknownFields.writeTo(output); + getUnknownFields().writeTo(output); } @java.lang.Override @@ -3227,7 +2999,7 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeEnumSize(20, enumType_); } - size += unknownFields.getSerializedSize(); + size += getUnknownFields().getSerializedSize(); memoizedSize = size; return size; } @@ -3334,7 +3106,7 @@ public boolean equals(final java.lang.Object obj) { if (hasEnumType()) { if (enumType_ != other.enumType_) return false; } - if (!unknownFields.equals(other.unknownFields)) return false; + if (!getUnknownFields().equals(other.getUnknownFields())) return false; return true; } @@ -3433,7 +3205,7 @@ public int hashCode() { hash = (37 * hash) + ENUMTYPE_FIELD_NUMBER; hash = (53 * hash) + enumType_; } - hash = (29 * hash) + unknownFields.hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } @@ -3601,10 +3373,11 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00004000); if (mapTypeBuilder_ == null) { mapType_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00008000); } else { + mapType_ = null; mapTypeBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00008000); stringListType_ = com.google.protobuf.LazyStringArrayList.EMPTY; bitField0_ = (bitField0_ & ~0x00010000); if (messageTypeBuilder_ == null) { @@ -3615,10 +3388,11 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00020000); if (messageListTypeBuilder_ == null) { messageListType_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00040000); } else { + messageListType_ = null; messageListTypeBuilder_.clear(); } + bitField0_ = (bitField0_ & ~0x00040000); enumType_ = 1; bitField0_ = (bitField0_ & ~0x00080000); return this; @@ -3908,7 +3682,7 @@ public Builder mergeFrom(org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllT if (other.hasEnumType()) { setEnumType(other.getEnumType()); } - this.mergeUnknownFields(other.unknownFields); + this.mergeUnknownFields(other.getUnknownFields()); onChanged(); return this; } @@ -3923,17 +3697,156 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllTypes parsedMessage = null; + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 9: { + doubleType_ = input.readDouble(); + bitField0_ |= 0x00000001; + break; + } // case 9 + case 21: { + floatType_ = input.readFloat(); + bitField0_ |= 0x00000002; + break; + } // case 21 + case 24: { + int32Type_ = input.readInt32(); + bitField0_ |= 0x00000004; + break; + } // case 24 + case 32: { + int64Type_ = input.readInt64(); + bitField0_ |= 0x00000008; + break; + } // case 32 + case 40: { + uint32Type_ = input.readUInt32(); + bitField0_ |= 0x00000010; + break; + } // case 40 + case 48: { + uint64Type_ = input.readUInt64(); + bitField0_ |= 0x00000020; + break; + } // case 48 + case 56: { + sint32Type_ = input.readSInt32(); + bitField0_ |= 0x00000040; + break; + } // case 56 + case 64: { + sint64Type_ = input.readSInt64(); + bitField0_ |= 0x00000080; + break; + } // case 64 + case 77: { + fixed32Type_ = input.readFixed32(); + bitField0_ |= 0x00000100; + break; + } // case 77 + case 81: { + fixed64Type_ = input.readFixed64(); + bitField0_ |= 0x00000200; + break; + } // case 81 + case 93: { + sfixed32Type_ = input.readSFixed32(); + bitField0_ |= 0x00000400; + break; + } // case 93 + case 97: { + sfixed64Type_ = input.readSFixed64(); + bitField0_ |= 0x00000800; + break; + } // case 97 + case 104: { + boolType_ = input.readBool(); + bitField0_ |= 0x00001000; + break; + } // case 104 + case 114: { + stringType_ = input.readBytes(); + bitField0_ |= 0x00002000; + break; + } // case 114 + case 122: { + bytesType_ = input.readBytes(); + bitField0_ |= 0x00004000; + break; + } // case 122 + case 130: { + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry m = + input.readMessage( + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.MapFieldEntry.PARSER, + extensionRegistry); + if (mapTypeBuilder_ == null) { + ensureMapTypeIsMutable(); + mapType_.add(m); + } else { + mapTypeBuilder_.addMessage(m); + } + break; + } // case 130 + case 138: { + com.google.protobuf.ByteString bs = input.readBytes(); + ensureStringListTypeIsMutable(); + stringListType_.add(bs); + break; + } // case 138 + case 146: { + input.readMessage( + getMessageTypeFieldBuilder().getBuilder(), + extensionRegistry); + bitField0_ |= 0x00020000; + break; + } // case 146 + case 154: { + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1 m = + input.readMessage( + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.Mesg1.PARSER, + extensionRegistry); + if (messageListTypeBuilder_ == null) { + ensureMessageListTypeIsMutable(); + messageListType_.add(m); + } else { + messageListTypeBuilder_.addMessage(m); + } + break; + } // case 154 + case 160: { + int tmpRaw = input.readEnum(); + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllTypes.Enum1 tmpValue = + org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllTypes.Enum1.forNumber(tmpRaw); + if (tmpValue == null) { + mergeUnknownVarintField(20, tmpRaw); + } else { + enumType_ = tmpRaw; + bitField0_ |= 0x00080000; + } + break; + } // case 160 + default: { + if (!super.parseUnknownField(input, extensionRegistry, tag)) { + done = true; // was an endgroup tag + } + break; + } // default: + } // switch (tag) + } // while (!done) } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hive.ql.io.protobuf.SampleProtos.AllTypes) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } + onChanged(); + } // finally return this; } private int bitField0_; @@ -5355,7 +5268,18 @@ public AllTypes parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new AllTypes(input, extensionRegistry); + Builder builder = newBuilder(); + try { + builder.mergeFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(builder.buildPartial()); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(builder.buildPartial()); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e) + .setUnfinishedMessage(builder.buildPartial()); + } + return builder.buildPartial(); } }; From e08a60029f93e52182908b61fff32e8659e78daa Mon Sep 17 00:00:00 2001 From: AnmolSun <124231245+AnmolSun@users.noreply.github.com> Date: Wed, 31 Jan 2024 14:04:30 +0530 Subject: [PATCH 158/179] HIVE-28000: Fix scenarios where 'not in' gives incorrect results due to type coercion (Anmol Sundaram reviewed by Krisztian Kasa, Steve Carlin, Attila Turoczy) --- .../ql/parse/type/TypeCheckProcFactory.java | 19 ++--- .../queries/clientpositive/not_in_scenarios.q | 8 ++ .../clientpositive/llap/in_coercion.q.out | 2 + .../llap/not_in_scenarios.q.out | 77 +++++++++++++++++++ .../llap/orc_llap_counters.q.out | 10 +-- 5 files changed, 102 insertions(+), 14 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/not_in_scenarios.q create mode 100644 ql/src/test/results/clientpositive/llap/not_in_scenarios.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java index 318d0c87792c..f8b50ca6c297 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java @@ -998,19 +998,12 @@ protected T getXpathOrFuncExprNodeDesc(ASTNode node, // different value type. The reason is that Hive and Calcite treat // types in IN clauses differently and it is practically impossible // to find some correct implementation unless this is done. - boolean hasNullValue = false; ListMultimap expressions = ArrayListMultimap.create(); for (int i = 1; i < children.size(); i++) { T columnDesc = children.get(0); T valueDesc = interpretNode(columnDesc, children.get(i)); - if (valueDesc == null) { - // Keep original - TypeInfo targetType = exprFactory.getTypeInfo(children.get(i)); - if (!expressions.containsKey(targetType)) { - expressions.put(targetType, columnDesc); - } - expressions.put(targetType, children.get(i)); - } else { + if (valueDesc != null) { + // Only add to the expression map if types can be coerced TypeInfo targetType = exprFactory.getTypeInfo(valueDesc); if (!expressions.containsKey(targetType)) { expressions.put(targetType, columnDesc); @@ -1018,6 +1011,14 @@ protected T getXpathOrFuncExprNodeDesc(ASTNode node, expressions.put(targetType, valueDesc); } } + if(expressions.isEmpty()) { + // We will only hit this when none of the operands inside the "in" clause can be type-coerced + // That would imply that the result of "in" is a boolean "false" + // This should not impact those cases where the "in" clause is used on a boolean column and + // there is no operand in the "in" clause that cannot be type-coerced into boolean because + // in case of boolean, Hive does not allow such use cases and throws an error + return exprFactory.createBooleanConstantExpr("false"); + } children.clear(); List newExprs = new ArrayList<>(); diff --git a/ql/src/test/queries/clientpositive/not_in_scenarios.q b/ql/src/test/queries/clientpositive/not_in_scenarios.q new file mode 100644 index 000000000000..6d614bd2c02c --- /dev/null +++ b/ql/src/test/queries/clientpositive/not_in_scenarios.q @@ -0,0 +1,8 @@ +CREATE TABLE my_tbl (id int); +insert into my_tbl values (100),(200),(300); +select * from my_tbl where id not in ('ABC', 'DEF'); +select * from my_tbl where id not in ('ABC', 'DEF', '123'); +select * from my_tbl where id not in ('ABC', 'DEF', '100'); +select * from my_tbl where id not in (100, 'ABC', 200); +select * from my_tbl where id is not null or id in ("ABC"); +select * from my_tbl where id is not null and id in ("ABC"); \ No newline at end of file diff --git a/ql/src/test/results/clientpositive/llap/in_coercion.q.out b/ql/src/test/results/clientpositive/llap/in_coercion.q.out index fe784ef6ec82..95fc24039751 100644 --- a/ql/src/test/results/clientpositive/llap/in_coercion.q.out +++ b/ql/src/test/results/clientpositive/llap/in_coercion.q.out @@ -31,10 +31,12 @@ POSTHOOK: Input: default@src_table NULL PREHOOK: query: select count(*) from src_table where key in (355.8) PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Input: default@src_table #### A masked pattern was here #### POSTHOOK: query: select count(*) from src_table where key in (355.8) POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Input: default@src_table #### A masked pattern was here #### 0 diff --git a/ql/src/test/results/clientpositive/llap/not_in_scenarios.q.out b/ql/src/test/results/clientpositive/llap/not_in_scenarios.q.out new file mode 100644 index 000000000000..3ca9fa6edd6a --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/not_in_scenarios.q.out @@ -0,0 +1,77 @@ +PREHOOK: query: CREATE TABLE my_tbl (id int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@my_tbl +POSTHOOK: query: CREATE TABLE my_tbl (id int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@my_tbl +PREHOOK: query: insert into my_tbl values (100),(200),(300) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@my_tbl +POSTHOOK: query: insert into my_tbl values (100),(200),(300) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@my_tbl +POSTHOOK: Lineage: my_tbl.id SCRIPT [] +PREHOOK: query: select * from my_tbl where id not in ('ABC', 'DEF') +PREHOOK: type: QUERY +PREHOOK: Input: default@my_tbl +#### A masked pattern was here #### +POSTHOOK: query: select * from my_tbl where id not in ('ABC', 'DEF') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@my_tbl +#### A masked pattern was here #### +100 +200 +300 +PREHOOK: query: select * from my_tbl where id not in ('ABC', 'DEF', '123') +PREHOOK: type: QUERY +PREHOOK: Input: default@my_tbl +#### A masked pattern was here #### +POSTHOOK: query: select * from my_tbl where id not in ('ABC', 'DEF', '123') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@my_tbl +#### A masked pattern was here #### +100 +200 +300 +PREHOOK: query: select * from my_tbl where id not in ('ABC', 'DEF', '100') +PREHOOK: type: QUERY +PREHOOK: Input: default@my_tbl +#### A masked pattern was here #### +POSTHOOK: query: select * from my_tbl where id not in ('ABC', 'DEF', '100') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@my_tbl +#### A masked pattern was here #### +200 +300 +PREHOOK: query: select * from my_tbl where id not in (100, 'ABC', 200) +PREHOOK: type: QUERY +PREHOOK: Input: default@my_tbl +#### A masked pattern was here #### +POSTHOOK: query: select * from my_tbl where id not in (100, 'ABC', 200) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@my_tbl +#### A masked pattern was here #### +300 +PREHOOK: query: select * from my_tbl where id is not null or id in ("ABC") +PREHOOK: type: QUERY +PREHOOK: Input: default@my_tbl +#### A masked pattern was here #### +POSTHOOK: query: select * from my_tbl where id is not null or id in ("ABC") +POSTHOOK: type: QUERY +POSTHOOK: Input: default@my_tbl +#### A masked pattern was here #### +100 +200 +300 +PREHOOK: query: select * from my_tbl where id is not null and id in ("ABC") +PREHOOK: type: QUERY +PREHOOK: Input: default@my_tbl +#### A masked pattern was here #### +POSTHOOK: query: select * from my_tbl where id is not null and id in ("ABC") +POSTHOOK: type: QUERY +POSTHOOK: Input: default@my_tbl +#### A masked pattern was here #### diff --git a/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out b/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out index 7d28a27438fd..49210d080bb1 100644 --- a/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out +++ b/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out @@ -922,12 +922,12 @@ Stage-1 HIVE COUNTERS: RECORDS_OUT_0: 1 RECORDS_OUT_INTERMEDIATE_Map_1: 0 RECORDS_OUT_INTERMEDIATE_Reducer_2: 0 - RECORDS_OUT_OPERATOR_FIL_7: 0 - RECORDS_OUT_OPERATOR_FS_9: 1 - RECORDS_OUT_OPERATOR_GBY_8: 1 + RECORDS_OUT_OPERATOR_FIL_8: 0 + RECORDS_OUT_OPERATOR_FS_12: 1 + RECORDS_OUT_OPERATOR_GBY_11: 1 RECORDS_OUT_OPERATOR_MAP_0: 0 - RECORDS_OUT_OPERATOR_RS_3: 0 - RECORDS_OUT_OPERATOR_SEL_2: 0 + RECORDS_OUT_OPERATOR_RS_10: 0 + RECORDS_OUT_OPERATOR_SEL_9: 0 RECORDS_OUT_OPERATOR_TS_0: 0 TOTAL_TABLE_ROWS_WRITTEN: 0 Stage-1 LLAP IO COUNTERS: From 6ab9b14c2c45e139a30ab78d41cefe5f1ab64d22 Mon Sep 17 00:00:00 2001 From: Butao Zhang Date: Thu, 1 Feb 2024 14:27:43 +0800 Subject: [PATCH 159/179] HIVE-27984: Support backward compatibility of hms thrift struct about column stats (#4984)(Butao Zhang, reviewed by okumin, Zhihua Deng) --- .../TestCachedStoreUpdateUsingEvents.java | 18 +-- .../hive/ql/exec/ColumnStatsUpdateTask.java | 2 +- .../hive/ql/stats/ColStatsProcessor.java | 2 +- .../thrift/gen-cpp/hive_metastore_types.cpp | 51 ++++----- .../gen/thrift/gen-cpp/hive_metastore_types.h | 105 +++++++++--------- .../hive/metastore/api/ColumnStatistics.java | 5 +- .../api/GetPartitionsByNamesRequest.java | 5 +- .../hive/metastore/api/GetTableRequest.java | 5 +- .../metastore/api/PartitionsStatsRequest.java | 60 +++++----- .../api/SetPartitionsStatsRequest.java | 60 +++++----- .../hive/metastore/api/TableStatsRequest.java | 62 ++++++----- .../gen-php/metastore/ColumnStatistics.php | 2 +- .../metastore/GetPartitionsByNamesRequest.php | 2 +- .../gen-php/metastore/GetTableRequest.php | 2 +- .../metastore/PartitionsStatsRequest.php | 4 +- .../metastore/SetPartitionsStatsRequest.php | 4 +- .../gen-php/metastore/TableStatsRequest.php | 4 +- .../thrift/gen-py/hive_metastore/ttypes.py | 30 ++--- .../gen/thrift/gen-rb/hive_metastore_types.rb | 15 +-- .../hive/metastore/HiveMetaStoreClient.java | 18 +-- .../src/main/thrift/hive_metastore.thrift | 12 +- .../HiveMetaStoreClientPreCatalog.java | 22 ++-- 22 files changed, 262 insertions(+), 228 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStoreUpdateUsingEvents.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStoreUpdateUsingEvents.java index dd08d8aa1095..1ad1349b10be 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStoreUpdateUsingEvents.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStoreUpdateUsingEvents.java @@ -585,7 +585,7 @@ private void updateTableColStats(String dbName, String tblName, String[] colName colStats.setStatsObj(getStatsObjects(dbName, tblName, colName, highValue, avgColLen)); colStats.setEngine(Constants.HIVE_ENGINE); - SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats)); setTblColStat.setWriteId(writeId); setTblColStat.setValidWriteIdList(validWriteIds); @@ -627,7 +627,7 @@ private void updatePartColStats(String dbName, String tblName, boolean isTxnTabl colStats.setStatsObj(getStatsObjects(dbName, tblName, colName, highValue, avgColLen)); colStats.setEngine(Constants.HIVE_ENGINE); - SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats)); setTblColStat.setWriteId(writeId); setTblColStat.setValidWriteIdList(validWriteIds); @@ -940,7 +940,7 @@ public void testTableColumnStatisticsTxnTableMultiAbort() throws Throwable { colStats.setStatsObj(getStatsObjects(dbName, tblName, colName, highValue, avgColLen)); colStats.setEngine(Constants.HIVE_ENGINE); - SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats)); setTblColStat.setWriteId(writeId); setTblColStat.setValidWriteIdList(validWriteIds); @@ -1007,7 +1007,7 @@ public void testTableColumnStatisticsTxnTableOpenTxn() throws Throwable { colStats.setStatsObj(getStatsObjects(dbName, tblName, colName, highValue, avgColLen)); colStats.setEngine(Constants.HIVE_ENGINE); - SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats)); setTblColStat.setWriteId(writeId); setTblColStat.setValidWriteIdList(validWriteIds); @@ -1056,7 +1056,7 @@ private void verifyAggrStat(String dbName, String tblName, String[] colName, Lis // This will update the cache for non txn table. PartitionsStatsRequest request = new PartitionsStatsRequest(dbName, tblName, - Collections.singletonList(colName[0]), partitions, Constants.HIVE_ENGINE); + Collections.singletonList(colName[0]), partitions); request.setCatName(DEFAULT_CATALOG_NAME); request.setValidWriteIdList(validWriteIds); AggrStats aggrStatsCached = hmsHandler.get_aggr_stats_for(request); @@ -1123,7 +1123,7 @@ public void testAggrStatTxnTable() throws Throwable { colStats.setStatsObj(getStatsObjects(dbName, tblName, colName, 5, 20)); colStats.setEngine(Constants.HIVE_ENGINE); - SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats)); setTblColStat.setWriteId(writeId); setTblColStat.setValidWriteIdList(validWriteIds); hmsHandler.update_partition_column_statistics_req(setTblColStat); @@ -1136,7 +1136,7 @@ public void testAggrStatTxnTable() throws Throwable { // keep the txn open and verify that the stats got is not compliant. PartitionsStatsRequest request = new PartitionsStatsRequest(dbName, tblName, - Collections.singletonList(colName[0]), partitions, Constants.HIVE_ENGINE); + Collections.singletonList(colName[0]), partitions); request.setCatName(DEFAULT_CATALOG_NAME); request.setValidWriteIdList(validWriteIds); AggrStats aggrStatsCached = hmsHandler.get_aggr_stats_for(request); @@ -1175,7 +1175,7 @@ public void testAggrStatAbortTxn() throws Throwable { colStats.setStatsObj(getStatsObjects(dbName, tblName, colName, 5, 20)); colStats.setEngine(Constants.HIVE_ENGINE); - SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + SetPartitionsStatsRequest setTblColStat = new SetPartitionsStatsRequest(Collections.singletonList(colStats)); setTblColStat.setWriteId(writeId); setTblColStat.setValidWriteIdList(validWriteIds); hmsHandler.update_partition_column_statistics_req(setTblColStat); @@ -1191,7 +1191,7 @@ public void testAggrStatAbortTxn() throws Throwable { // keep the txn open and verify that the stats got is not compliant. PartitionsStatsRequest request = new PartitionsStatsRequest(dbName, tblName, - Collections.singletonList(colName[0]), partitions, Constants.HIVE_ENGINE); + Collections.singletonList(colName[0]), partitions); request.setCatName(DEFAULT_CATALOG_NAME); request.setValidWriteIdList(validWriteIds); AggrStats aggrStatsCached = hmsHandler.get_aggr_stats_for(request); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java index d8492a16f175..8b6c8d6b1bd4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java @@ -317,7 +317,7 @@ private ColumnStatisticsDesc getColumnStatsDesc(String dbName, private int persistColumnStats(Hive db) throws HiveException, MetaException, IOException { ColumnStatistics colStats = constructColumnStatsFromInput(); SetPartitionsStatsRequest request = - new SetPartitionsStatsRequest(Collections.singletonList(colStats), Constants.HIVE_ENGINE); + new SetPartitionsStatsRequest(Collections.singletonList(colStats)); // Set writeId and validWriteId list for replicated statistics. getColStats() will return // non-null value only during replication. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java index e6f945bc864f..285175414d62 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java @@ -211,7 +211,7 @@ public int persistColumnStats(Hive db, Table tbl) throws HiveException, MetaExce if (colStats.isEmpty()) { continue; } - SetPartitionsStatsRequest request = new SetPartitionsStatsRequest(colStats, Constants.HIVE_ENGINE); + SetPartitionsStatsRequest request = new SetPartitionsStatsRequest(colStats); request.setNeedMerge(colStatDesc.isNeedMerge()); if (txnMgr != null) { request.setWriteId(writeId); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index 2a7a8d65b5e4..1ec6bcee3f19 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -13933,6 +13933,7 @@ __isset.validWriteIdList = true; void SetPartitionsStatsRequest::__set_engine(const std::string& val) { this->engine = val; +__isset.engine = true; } std::ostream& operator<<(std::ostream& out, const SetPartitionsStatsRequest& obj) { @@ -13954,7 +13955,6 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* using ::apache::thrift::protocol::TProtocolException; bool isset_colStats = false; - bool isset_engine = false; while (true) { @@ -14011,7 +14011,7 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* case 5: if (ftype == ::apache::thrift::protocol::T_STRING) { xfer += iprot->readString(this->engine); - isset_engine = true; + this->__isset.engine = true; } else { xfer += iprot->skip(ftype); } @@ -14027,8 +14027,6 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* if (!isset_colStats) throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_engine) - throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } @@ -14064,10 +14062,11 @@ uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* xfer += oprot->writeString(this->validWriteIdList); xfer += oprot->writeFieldEnd(); } - xfer += oprot->writeFieldBegin("engine", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->engine); - xfer += oprot->writeFieldEnd(); - + if (this->__isset.engine) { + xfer += oprot->writeFieldBegin("engine", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->engine); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -14107,7 +14106,7 @@ void SetPartitionsStatsRequest::printTo(std::ostream& out) const { out << ", " << "needMerge="; (__isset.needMerge ? (out << to_string(needMerge)) : (out << "")); out << ", " << "writeId="; (__isset.writeId ? (out << to_string(writeId)) : (out << "")); out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "")); - out << ", " << "engine=" << to_string(engine); + out << ", " << "engine="; (__isset.engine ? (out << to_string(engine)) : (out << "")); out << ")"; } @@ -18221,6 +18220,7 @@ __isset.validWriteIdList = true; void TableStatsRequest::__set_engine(const std::string& val) { this->engine = val; +__isset.engine = true; } void TableStatsRequest::__set_id(const int64_t val) { @@ -18249,7 +18249,6 @@ uint32_t TableStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) { bool isset_dbName = false; bool isset_tblName = false; bool isset_colNames = false; - bool isset_engine = false; while (true) { @@ -18314,7 +18313,7 @@ uint32_t TableStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) { case 6: if (ftype == ::apache::thrift::protocol::T_STRING) { xfer += iprot->readString(this->engine); - isset_engine = true; + this->__isset.engine = true; } else { xfer += iprot->skip(ftype); } @@ -18342,8 +18341,6 @@ uint32_t TableStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) { throw TProtocolException(TProtocolException::INVALID_DATA); if (!isset_colNames) throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_engine) - throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } @@ -18382,10 +18379,11 @@ uint32_t TableStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) xfer += oprot->writeString(this->validWriteIdList); xfer += oprot->writeFieldEnd(); } - xfer += oprot->writeFieldBegin("engine", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->engine); - xfer += oprot->writeFieldEnd(); - + if (this->__isset.engine) { + xfer += oprot->writeFieldBegin("engine", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->engine); + xfer += oprot->writeFieldEnd(); + } if (this->__isset.id) { xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 7); xfer += oprot->writeI64(this->id); @@ -18437,7 +18435,7 @@ void TableStatsRequest::printTo(std::ostream& out) const { out << ", " << "colNames=" << to_string(colNames); out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "")); out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "")); - out << ", " << "engine=" << to_string(engine); + out << ", " << "engine="; (__isset.engine ? (out << to_string(engine)) : (out << "")); out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); out << ")"; } @@ -18475,6 +18473,7 @@ __isset.validWriteIdList = true; void PartitionsStatsRequest::__set_engine(const std::string& val) { this->engine = val; +__isset.engine = true; } std::ostream& operator<<(std::ostream& out, const PartitionsStatsRequest& obj) { @@ -18499,7 +18498,6 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr bool isset_tblName = false; bool isset_colNames = false; bool isset_partNames = false; - bool isset_engine = false; while (true) { @@ -18584,7 +18582,7 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr case 7: if (ftype == ::apache::thrift::protocol::T_STRING) { xfer += iprot->readString(this->engine); - isset_engine = true; + this->__isset.engine = true; } else { xfer += iprot->skip(ftype); } @@ -18606,8 +18604,6 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr throw TProtocolException(TProtocolException::INVALID_DATA); if (!isset_partNames) throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_engine) - throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } @@ -18658,10 +18654,11 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op xfer += oprot->writeString(this->validWriteIdList); xfer += oprot->writeFieldEnd(); } - xfer += oprot->writeFieldBegin("engine", ::apache::thrift::protocol::T_STRING, 7); - xfer += oprot->writeString(this->engine); - xfer += oprot->writeFieldEnd(); - + if (this->__isset.engine) { + xfer += oprot->writeFieldBegin("engine", ::apache::thrift::protocol::T_STRING, 7); + xfer += oprot->writeString(this->engine); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -18709,7 +18706,7 @@ void PartitionsStatsRequest::printTo(std::ostream& out) const { out << ", " << "partNames=" << to_string(partNames); out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "")); out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "")); - out << ", " << "engine=" << to_string(engine); + out << ", " << "engine="; (__isset.engine ? (out << to_string(engine)) : (out << "")); out << ")"; } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h index 4e0a567345b8..2a65c40f531a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -5012,7 +5012,7 @@ void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b); std::ostream& operator<<(std::ostream& out, const ColumnStatisticsDesc& obj); typedef struct _ColumnStatistics__isset { - _ColumnStatistics__isset() : isStatsCompliant(false), engine(false) {} + _ColumnStatistics__isset() : isStatsCompliant(false), engine(true) {} bool isStatsCompliant :1; bool engine :1; } _ColumnStatistics__isset; @@ -5022,9 +5022,8 @@ class ColumnStatistics : public virtual ::apache::thrift::TBase { ColumnStatistics(const ColumnStatistics&); ColumnStatistics& operator=(const ColumnStatistics&); - ColumnStatistics() noexcept - : isStatsCompliant(0), - engine() { + ColumnStatistics() : isStatsCompliant(0), + engine("hive") { } virtual ~ColumnStatistics() noexcept; @@ -5938,10 +5937,11 @@ void swap(AggrStats &a, AggrStats &b); std::ostream& operator<<(std::ostream& out, const AggrStats& obj); typedef struct _SetPartitionsStatsRequest__isset { - _SetPartitionsStatsRequest__isset() : needMerge(false), writeId(true), validWriteIdList(false) {} + _SetPartitionsStatsRequest__isset() : needMerge(false), writeId(true), validWriteIdList(false), engine(true) {} bool needMerge :1; bool writeId :1; bool validWriteIdList :1; + bool engine :1; } _SetPartitionsStatsRequest__isset; class SetPartitionsStatsRequest : public virtual ::apache::thrift::TBase { @@ -5949,11 +5949,10 @@ class SetPartitionsStatsRequest : public virtual ::apache::thrift::TBase { SetPartitionsStatsRequest(const SetPartitionsStatsRequest&); SetPartitionsStatsRequest& operator=(const SetPartitionsStatsRequest&); - SetPartitionsStatsRequest() noexcept - : needMerge(0), - writeId(-1LL), - validWriteIdList(), - engine() { + SetPartitionsStatsRequest() : needMerge(0), + writeId(-1LL), + validWriteIdList(), + engine("hive") { } virtual ~SetPartitionsStatsRequest() noexcept; @@ -5991,7 +5990,9 @@ class SetPartitionsStatsRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList)) return false; - if (!(engine == rhs.engine)) + if (__isset.engine != rhs.__isset.engine) + return false; + else if (__isset.engine && !(engine == rhs.engine)) return false; return true; } @@ -7473,9 +7474,10 @@ void swap(PartitionsStatsResult &a, PartitionsStatsResult &b); std::ostream& operator<<(std::ostream& out, const PartitionsStatsResult& obj); typedef struct _TableStatsRequest__isset { - _TableStatsRequest__isset() : catName(false), validWriteIdList(false), id(true) {} + _TableStatsRequest__isset() : catName(false), validWriteIdList(false), engine(true), id(true) {} bool catName :1; bool validWriteIdList :1; + bool engine :1; bool id :1; } _TableStatsRequest__isset; @@ -7484,13 +7486,12 @@ class TableStatsRequest : public virtual ::apache::thrift::TBase { TableStatsRequest(const TableStatsRequest&); TableStatsRequest& operator=(const TableStatsRequest&); - TableStatsRequest() noexcept - : dbName(), - tblName(), - catName(), - validWriteIdList(), - engine(), - id(-1LL) { + TableStatsRequest() : dbName(), + tblName(), + catName(), + validWriteIdList(), + engine("hive"), + id(-1LL) { } virtual ~TableStatsRequest() noexcept; @@ -7534,7 +7535,9 @@ class TableStatsRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList)) return false; - if (!(engine == rhs.engine)) + if (__isset.engine != rhs.__isset.engine) + return false; + else if (__isset.engine && !(engine == rhs.engine)) return false; if (__isset.id != rhs.__isset.id) return false; @@ -7559,9 +7562,10 @@ void swap(TableStatsRequest &a, TableStatsRequest &b); std::ostream& operator<<(std::ostream& out, const TableStatsRequest& obj); typedef struct _PartitionsStatsRequest__isset { - _PartitionsStatsRequest__isset() : catName(false), validWriteIdList(false) {} + _PartitionsStatsRequest__isset() : catName(false), validWriteIdList(false), engine(true) {} bool catName :1; bool validWriteIdList :1; + bool engine :1; } _PartitionsStatsRequest__isset; class PartitionsStatsRequest : public virtual ::apache::thrift::TBase { @@ -7569,12 +7573,11 @@ class PartitionsStatsRequest : public virtual ::apache::thrift::TBase { PartitionsStatsRequest(const PartitionsStatsRequest&); PartitionsStatsRequest& operator=(const PartitionsStatsRequest&); - PartitionsStatsRequest() noexcept - : dbName(), - tblName(), - catName(), - validWriteIdList(), - engine() { + PartitionsStatsRequest() : dbName(), + tblName(), + catName(), + validWriteIdList(), + engine("hive") { } virtual ~PartitionsStatsRequest() noexcept; @@ -7620,7 +7623,9 @@ class PartitionsStatsRequest : public virtual ::apache::thrift::TBase { return false; else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList)) return false; - if (!(engine == rhs.engine)) + if (__isset.engine != rhs.__isset.engine) + return false; + else if (__isset.engine && !(engine == rhs.engine)) return false; return true; } @@ -8254,7 +8259,7 @@ void swap(PartitionValuesResponse &a, PartitionValuesResponse &b); std::ostream& operator<<(std::ostream& out, const PartitionValuesResponse& obj); typedef struct _GetPartitionsByNamesRequest__isset { - _GetPartitionsByNamesRequest__isset() : names(false), get_col_stats(false), processorCapabilities(false), processorIdentifier(false), engine(false), validWriteIdList(false), getFileMetadata(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} + _GetPartitionsByNamesRequest__isset() : names(false), get_col_stats(false), processorCapabilities(false), processorIdentifier(false), engine(true), validWriteIdList(false), getFileMetadata(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {} bool names :1; bool get_col_stats :1; bool processorCapabilities :1; @@ -8273,18 +8278,17 @@ class GetPartitionsByNamesRequest : public virtual ::apache::thrift::TBase { GetPartitionsByNamesRequest(const GetPartitionsByNamesRequest&); GetPartitionsByNamesRequest& operator=(const GetPartitionsByNamesRequest&); - GetPartitionsByNamesRequest() noexcept - : db_name(), - tbl_name(), - get_col_stats(0), - processorIdentifier(), - engine(), - validWriteIdList(), - getFileMetadata(0), - id(-1LL), - skipColumnSchemaForPartition(0), - includeParamKeyPattern(), - excludeParamKeyPattern() { + GetPartitionsByNamesRequest() : db_name(), + tbl_name(), + get_col_stats(0), + processorIdentifier(), + engine("hive"), + validWriteIdList(), + getFileMetadata(0), + id(-1LL), + skipColumnSchemaForPartition(0), + includeParamKeyPattern(), + excludeParamKeyPattern() { } virtual ~GetPartitionsByNamesRequest() noexcept; @@ -13888,7 +13892,7 @@ void swap(GetProjectionsSpec &a, GetProjectionsSpec &b); std::ostream& operator<<(std::ostream& out, const GetProjectionsSpec& obj); typedef struct _GetTableRequest__isset { - _GetTableRequest__isset() : capabilities(false), catName(false), validWriteIdList(false), getColumnStats(false), processorCapabilities(false), processorIdentifier(false), engine(false), id(true) {} + _GetTableRequest__isset() : capabilities(false), catName(false), validWriteIdList(false), getColumnStats(false), processorCapabilities(false), processorIdentifier(false), engine(true), id(true) {} bool capabilities :1; bool catName :1; bool validWriteIdList :1; @@ -13904,15 +13908,14 @@ class GetTableRequest : public virtual ::apache::thrift::TBase { GetTableRequest(const GetTableRequest&); GetTableRequest& operator=(const GetTableRequest&); - GetTableRequest() noexcept - : dbName(), - tblName(), - catName(), - validWriteIdList(), - getColumnStats(0), - processorIdentifier(), - engine(), - id(-1LL) { + GetTableRequest() : dbName(), + tblName(), + catName(), + validWriteIdList(), + getColumnStats(0), + processorIdentifier(), + engine("hive"), + id(-1LL) { } virtual ~GetTableRequest() noexcept; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java index 782586acece1..3fffbce5d105 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java @@ -114,6 +114,8 @@ public java.lang.String getFieldName() { } public ColumnStatistics() { + this.engine = "hive"; + } public ColumnStatistics( @@ -156,7 +158,8 @@ public void clear() { this.statsObj = null; setIsStatsCompliantIsSet(false); this.isStatsCompliant = false; - this.engine = null; + this.engine = "hive"; + } @org.apache.thrift.annotation.Nullable diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java index 2ecd9aa0c1a5..2654f09d4f4e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java @@ -181,6 +181,8 @@ public java.lang.String getFieldName() { } public GetPartitionsByNamesRequest() { + this.engine = "hive"; + this.id = -1L; } @@ -247,7 +249,8 @@ public void clear() { this.get_col_stats = false; this.processorCapabilities = null; this.processorIdentifier = null; - this.engine = null; + this.engine = "hive"; + this.validWriteIdList = null; setGetFileMetadataIsSet(false); this.getFileMetadata = false; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java index 50d04e78653f..a333674af92f 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java @@ -157,6 +157,8 @@ public java.lang.String getFieldName() { } public GetTableRequest() { + this.engine = "hive"; + this.id = -1L; } @@ -219,7 +221,8 @@ public void clear() { this.getColumnStats = false; this.processorCapabilities = null; this.processorIdentifier = null; - this.engine = null; + this.engine = "hive"; + this.id = -1L; } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java index bdddf8844bdd..7ef14ac7706e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java @@ -28,7 +28,7 @@ private @org.apache.thrift.annotation.Nullable java.util.List partNames; // required private @org.apache.thrift.annotation.Nullable java.lang.String catName; // optional private @org.apache.thrift.annotation.Nullable java.lang.String validWriteIdList; // optional - private @org.apache.thrift.annotation.Nullable java.lang.String engine; // required + private @org.apache.thrift.annotation.Nullable java.lang.String engine; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -109,7 +109,7 @@ public java.lang.String getFieldName() { } // isset id assignments - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.VALID_WRITE_ID_LIST}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.VALID_WRITE_ID_LIST,_Fields.ENGINE}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -127,28 +127,28 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ENGINE, new org.apache.thrift.meta_data.FieldMetaData("engine", org.apache.thrift.TFieldRequirementType.REQUIRED, + tmpMap.put(_Fields.ENGINE, new org.apache.thrift.meta_data.FieldMetaData("engine", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionsStatsRequest.class, metaDataMap); } public PartitionsStatsRequest() { + this.engine = "hive"; + } public PartitionsStatsRequest( java.lang.String dbName, java.lang.String tblName, java.util.List colNames, - java.util.List partNames, - java.lang.String engine) + java.util.List partNames) { this(); this.dbName = dbName; this.tblName = tblName; this.colNames = colNames; this.partNames = partNames; - this.engine = engine; } /** @@ -192,7 +192,8 @@ public void clear() { this.partNames = null; this.catName = null; this.validWriteIdList = null; - this.engine = null; + this.engine = "hive"; + } @org.apache.thrift.annotation.Nullable @@ -773,14 +774,16 @@ public java.lang.String toString() { } first = false; } - if (!first) sb.append(", "); - sb.append("engine:"); - if (this.engine == null) { - sb.append("null"); - } else { - sb.append(this.engine); + if (isSetEngine()) { + if (!first) sb.append(", "); + sb.append("engine:"); + if (this.engine == null) { + sb.append("null"); + } else { + sb.append(this.engine); + } + first = false; } - first = false; sb.append(")"); return sb.toString(); } @@ -803,10 +806,6 @@ public void validate() throws org.apache.thrift.TException { throw new org.apache.thrift.protocol.TProtocolException("Required field 'partNames' is unset! Struct:" + toString()); } - if (!isSetEngine()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'engine' is unset! Struct:" + toString()); - } - // check for sub-struct validity } @@ -982,9 +981,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, PartitionsStatsReq } } if (struct.engine != null) { - oprot.writeFieldBegin(ENGINE_FIELD_DESC); - oprot.writeString(struct.engine); - oprot.writeFieldEnd(); + if (struct.isSetEngine()) { + oprot.writeFieldBegin(ENGINE_FIELD_DESC); + oprot.writeString(struct.engine); + oprot.writeFieldEnd(); + } } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -1019,7 +1020,6 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsRequ oprot.writeString(_iter627); } } - oprot.writeString(struct.engine); java.util.BitSet optionals = new java.util.BitSet(); if (struct.isSetCatName()) { optionals.set(0); @@ -1027,13 +1027,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsRequ if (struct.isSetValidWriteIdList()) { optionals.set(1); } - oprot.writeBitSet(optionals, 2); + if (struct.isSetEngine()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } if (struct.isSetValidWriteIdList()) { oprot.writeString(struct.validWriteIdList); } + if (struct.isSetEngine()) { + oprot.writeString(struct.engine); + } } @Override @@ -1065,9 +1071,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsReque } } struct.setPartNamesIsSet(true); - struct.engine = iprot.readString(); - struct.setEngineIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(2); + java.util.BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); @@ -1076,6 +1080,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsReque struct.validWriteIdList = iprot.readString(); struct.setValidWriteIdListIsSet(true); } + if (incoming.get(2)) { + struct.engine = iprot.readString(); + struct.setEngineIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java index 1c373d92da2e..c10e726c6e70 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java @@ -24,7 +24,7 @@ private boolean needMerge; // optional private long writeId; // optional private @org.apache.thrift.annotation.Nullable java.lang.String validWriteIdList; // optional - private @org.apache.thrift.annotation.Nullable java.lang.String engine; // required + private @org.apache.thrift.annotation.Nullable java.lang.String engine; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -102,7 +102,7 @@ public java.lang.String getFieldName() { private static final int __NEEDMERGE_ISSET_ID = 0; private static final int __WRITEID_ISSET_ID = 1; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.NEED_MERGE,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST}; + private static final _Fields optionals[] = {_Fields.NEED_MERGE,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST,_Fields.ENGINE}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -115,7 +115,7 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ENGINE, new org.apache.thrift.meta_data.FieldMetaData("engine", org.apache.thrift.TFieldRequirementType.REQUIRED, + tmpMap.put(_Fields.ENGINE, new org.apache.thrift.meta_data.FieldMetaData("engine", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = java.util.Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SetPartitionsStatsRequest.class, metaDataMap); @@ -124,15 +124,15 @@ public java.lang.String getFieldName() { public SetPartitionsStatsRequest() { this.writeId = -1L; + this.engine = "hive"; + } public SetPartitionsStatsRequest( - java.util.List colStats, - java.lang.String engine) + java.util.List colStats) { this(); this.colStats = colStats; - this.engine = engine; } /** @@ -169,7 +169,8 @@ public void clear() { this.writeId = -1L; this.validWriteIdList = null; - this.engine = null; + this.engine = "hive"; + } public int getColStatsSize() { @@ -588,14 +589,16 @@ public java.lang.String toString() { } first = false; } - if (!first) sb.append(", "); - sb.append("engine:"); - if (this.engine == null) { - sb.append("null"); - } else { - sb.append(this.engine); + if (isSetEngine()) { + if (!first) sb.append(", "); + sb.append("engine:"); + if (this.engine == null) { + sb.append("null"); + } else { + sb.append(this.engine); + } + first = false; } - first = false; sb.append(")"); return sb.toString(); } @@ -606,10 +609,6 @@ public void validate() throws org.apache.thrift.TException { throw new org.apache.thrift.protocol.TProtocolException("Required field 'colStats' is unset! Struct:" + toString()); } - if (!isSetEngine()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'engine' is unset! Struct:" + toString()); - } - // check for sub-struct validity } @@ -743,9 +742,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SetPartitionsStats } } if (struct.engine != null) { - oprot.writeFieldBegin(ENGINE_FIELD_DESC); - oprot.writeString(struct.engine); - oprot.writeFieldEnd(); + if (struct.isSetEngine()) { + oprot.writeFieldBegin(ENGINE_FIELD_DESC); + oprot.writeString(struct.engine); + oprot.writeFieldEnd(); + } } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -771,7 +772,6 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsR _iter450.write(oprot); } } - oprot.writeString(struct.engine); java.util.BitSet optionals = new java.util.BitSet(); if (struct.isSetNeedMerge()) { optionals.set(0); @@ -782,7 +782,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsR if (struct.isSetValidWriteIdList()) { optionals.set(2); } - oprot.writeBitSet(optionals, 3); + if (struct.isSetEngine()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); if (struct.isSetNeedMerge()) { oprot.writeBool(struct.needMerge); } @@ -792,6 +795,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsR if (struct.isSetValidWriteIdList()) { oprot.writeString(struct.validWriteIdList); } + if (struct.isSetEngine()) { + oprot.writeString(struct.engine); + } } @Override @@ -809,9 +815,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsRe } } struct.setColStatsIsSet(true); - struct.engine = iprot.readString(); - struct.setEngineIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(3); + java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { struct.needMerge = iprot.readBool(); struct.setNeedMergeIsSet(true); @@ -824,6 +828,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsRe struct.validWriteIdList = iprot.readString(); struct.setValidWriteIdListIsSet(true); } + if (incoming.get(3)) { + struct.engine = iprot.readString(); + struct.setEngineIsSet(true); + } } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java index ea6626b78feb..685fe9d6982e 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java @@ -27,7 +27,7 @@ private @org.apache.thrift.annotation.Nullable java.util.List colNames; // required private @org.apache.thrift.annotation.Nullable java.lang.String catName; // optional private @org.apache.thrift.annotation.Nullable java.lang.String validWriteIdList; // optional - private @org.apache.thrift.annotation.Nullable java.lang.String engine; // required + private @org.apache.thrift.annotation.Nullable java.lang.String engine; // optional private long id; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ @@ -111,7 +111,7 @@ public java.lang.String getFieldName() { // isset id assignments private static final int __ID_ISSET_ID = 0; private byte __isset_bitfield = 0; - private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.VALID_WRITE_ID_LIST,_Fields.ID}; + private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.VALID_WRITE_ID_LIST,_Fields.ENGINE,_Fields.ID}; public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -126,7 +126,7 @@ public java.lang.String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - tmpMap.put(_Fields.ENGINE, new org.apache.thrift.meta_data.FieldMetaData("engine", org.apache.thrift.TFieldRequirementType.REQUIRED, + tmpMap.put(_Fields.ENGINE, new org.apache.thrift.meta_data.FieldMetaData("engine", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); @@ -135,6 +135,8 @@ public java.lang.String getFieldName() { } public TableStatsRequest() { + this.engine = "hive"; + this.id = -1L; } @@ -142,14 +144,12 @@ public TableStatsRequest() { public TableStatsRequest( java.lang.String dbName, java.lang.String tblName, - java.util.List colNames, - java.lang.String engine) + java.util.List colNames) { this(); this.dbName = dbName; this.tblName = tblName; this.colNames = colNames; - this.engine = engine; } /** @@ -190,7 +190,8 @@ public void clear() { this.colNames = null; this.catName = null; this.validWriteIdList = null; - this.engine = null; + this.engine = "hive"; + this.id = -1L; } @@ -747,14 +748,16 @@ public java.lang.String toString() { } first = false; } - if (!first) sb.append(", "); - sb.append("engine:"); - if (this.engine == null) { - sb.append("null"); - } else { - sb.append(this.engine); + if (isSetEngine()) { + if (!first) sb.append(", "); + sb.append("engine:"); + if (this.engine == null) { + sb.append("null"); + } else { + sb.append(this.engine); + } + first = false; } - first = false; if (isSetId()) { if (!first) sb.append(", "); sb.append("id:"); @@ -779,10 +782,6 @@ public void validate() throws org.apache.thrift.TException { throw new org.apache.thrift.protocol.TProtocolException("Required field 'colNames' is unset! Struct:" + toString()); } - if (!isSetEngine()) { - throw new org.apache.thrift.protocol.TProtocolException("Required field 'engine' is unset! Struct:" + toString()); - } - // check for sub-struct validity } @@ -938,9 +937,11 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TableStatsRequest } } if (struct.engine != null) { - oprot.writeFieldBegin(ENGINE_FIELD_DESC); - oprot.writeString(struct.engine); - oprot.writeFieldEnd(); + if (struct.isSetEngine()) { + oprot.writeFieldBegin(ENGINE_FIELD_DESC); + oprot.writeString(struct.engine); + oprot.writeFieldEnd(); + } } if (struct.isSetId()) { oprot.writeFieldBegin(ID_FIELD_DESC); @@ -973,7 +974,6 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TableStatsRequest s oprot.writeString(_iter614); } } - oprot.writeString(struct.engine); java.util.BitSet optionals = new java.util.BitSet(); if (struct.isSetCatName()) { optionals.set(0); @@ -981,16 +981,22 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TableStatsRequest s if (struct.isSetValidWriteIdList()) { optionals.set(1); } - if (struct.isSetId()) { + if (struct.isSetEngine()) { optionals.set(2); } - oprot.writeBitSet(optionals, 3); + if (struct.isSetId()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); if (struct.isSetCatName()) { oprot.writeString(struct.catName); } if (struct.isSetValidWriteIdList()) { oprot.writeString(struct.validWriteIdList); } + if (struct.isSetEngine()) { + oprot.writeString(struct.engine); + } if (struct.isSetId()) { oprot.writeI64(struct.id); } @@ -1014,9 +1020,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsRequest st } } struct.setColNamesIsSet(true); - struct.engine = iprot.readString(); - struct.setEngineIsSet(true); - java.util.BitSet incoming = iprot.readBitSet(3); + java.util.BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { struct.catName = iprot.readString(); struct.setCatNameIsSet(true); @@ -1026,6 +1030,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsRequest st struct.setValidWriteIdListIsSet(true); } if (incoming.get(2)) { + struct.engine = iprot.readString(); + struct.setEngineIsSet(true); + } + if (incoming.get(3)) { struct.id = iprot.readI64(); struct.setIdIsSet(true); } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ColumnStatistics.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ColumnStatistics.php index 0387862ec870..e5f9553ef95a 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ColumnStatistics.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ColumnStatistics.php @@ -64,7 +64,7 @@ class ColumnStatistics /** * @var string */ - public $engine = null; + public $engine = "hive"; public function __construct($vals = null) { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php index c06aa5ac27e9..e21e5fa5ad98 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetPartitionsByNamesRequest.php @@ -123,7 +123,7 @@ class GetPartitionsByNamesRequest /** * @var string */ - public $engine = null; + public $engine = "hive"; /** * @var string */ diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php index 3c56331bb0f5..217f5a377b29 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/GetTableRequest.php @@ -113,7 +113,7 @@ class GetTableRequest /** * @var string */ - public $engine = null; + public $engine = "hive"; /** * @var int */ diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsStatsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsStatsRequest.php index c7b3f053ae72..2c039313bd5b 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsStatsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/PartitionsStatsRequest.php @@ -61,7 +61,7 @@ class PartitionsStatsRequest ), 7 => array( 'var' => 'engine', - 'isRequired' => true, + 'isRequired' => false, 'type' => TType::STRING, ), ); @@ -93,7 +93,7 @@ class PartitionsStatsRequest /** * @var string */ - public $engine = null; + public $engine = "hive"; public function __construct($vals = null) { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SetPartitionsStatsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SetPartitionsStatsRequest.php index dae3c80aa277..ca7bad0cfb24 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SetPartitionsStatsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/SetPartitionsStatsRequest.php @@ -48,7 +48,7 @@ class SetPartitionsStatsRequest ), 5 => array( 'var' => 'engine', - 'isRequired' => true, + 'isRequired' => false, 'type' => TType::STRING, ), ); @@ -72,7 +72,7 @@ class SetPartitionsStatsRequest /** * @var string */ - public $engine = null; + public $engine = "hive"; public function __construct($vals = null) { diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/TableStatsRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/TableStatsRequest.php index 00c26651ff46..00d3ebb33d94 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/TableStatsRequest.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/TableStatsRequest.php @@ -52,7 +52,7 @@ class TableStatsRequest ), 6 => array( 'var' => 'engine', - 'isRequired' => true, + 'isRequired' => false, 'type' => TType::STRING, ), 7 => array( @@ -85,7 +85,7 @@ class TableStatsRequest /** * @var string */ - public $engine = null; + public $engine = "hive"; /** * @var int */ diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py index c7fe5de1aa0d..7f3c0e949ffc 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -6478,7 +6478,7 @@ class ColumnStatistics(object): """ - def __init__(self, statsDesc=None, statsObj=None, isStatsCompliant=None, engine=None,): + def __init__(self, statsDesc=None, statsObj=None, isStatsCompliant=None, engine="hive",): self.statsDesc = statsDesc self.statsObj = statsObj self.isStatsCompliant = isStatsCompliant @@ -7955,7 +7955,7 @@ class SetPartitionsStatsRequest(object): """ - def __init__(self, colStats=None, needMerge=None, writeId=-1, validWriteIdList=None, engine=None,): + def __init__(self, colStats=None, needMerge=None, writeId=-1, validWriteIdList=None, engine="hive",): self.colStats = colStats self.needMerge = needMerge self.writeId = writeId @@ -8041,8 +8041,6 @@ def write(self, oprot): def validate(self): if self.colStats is None: raise TProtocolException(message='Required field colStats is unset!') - if self.engine is None: - raise TProtocolException(message='Required field engine is unset!') return def __repr__(self): @@ -10466,7 +10464,7 @@ class TableStatsRequest(object): """ - def __init__(self, dbName=None, tblName=None, colNames=None, catName=None, validWriteIdList=None, engine=None, id=-1,): + def __init__(self, dbName=None, tblName=None, colNames=None, catName=None, validWriteIdList=None, engine="hive", id=-1,): self.dbName = dbName self.tblName = tblName self.colNames = colNames @@ -10575,8 +10573,6 @@ def validate(self): raise TProtocolException(message='Required field tblName is unset!') if self.colNames is None: raise TProtocolException(message='Required field colNames is unset!') - if self.engine is None: - raise TProtocolException(message='Required field engine is unset!') return def __repr__(self): @@ -10605,7 +10601,7 @@ class PartitionsStatsRequest(object): """ - def __init__(self, dbName=None, tblName=None, colNames=None, partNames=None, catName=None, validWriteIdList=None, engine=None,): + def __init__(self, dbName=None, tblName=None, colNames=None, partNames=None, catName=None, validWriteIdList=None, engine="hive",): self.dbName = dbName self.tblName = tblName self.colNames = colNames @@ -10724,8 +10720,6 @@ def validate(self): raise TProtocolException(message='Required field colNames is unset!') if self.partNames is None: raise TProtocolException(message='Required field partNames is unset!') - if self.engine is None: - raise TProtocolException(message='Required field engine is unset!') return def __repr__(self): @@ -11728,7 +11722,7 @@ class GetPartitionsByNamesRequest(object): """ - def __init__(self, db_name=None, tbl_name=None, names=None, get_col_stats=None, processorCapabilities=None, processorIdentifier=None, engine=None, validWriteIdList=None, getFileMetadata=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): + def __init__(self, db_name=None, tbl_name=None, names=None, get_col_stats=None, processorCapabilities=None, processorIdentifier=None, engine="hive", validWriteIdList=None, getFileMetadata=None, id=-1, skipColumnSchemaForPartition=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,): self.db_name = db_name self.tbl_name = tbl_name self.names = names @@ -20259,7 +20253,7 @@ class GetTableRequest(object): """ - def __init__(self, dbName=None, tblName=None, capabilities=None, catName=None, validWriteIdList=None, getColumnStats=None, processorCapabilities=None, processorIdentifier=None, engine=None, id=-1,): + def __init__(self, dbName=None, tblName=None, capabilities=None, catName=None, validWriteIdList=None, getColumnStats=None, processorCapabilities=None, processorIdentifier=None, engine="hive", id=-1,): self.dbName = dbName self.tblName = tblName self.capabilities = capabilities @@ -31727,7 +31721,7 @@ def __ne__(self, other): (1, TType.STRUCT, 'statsDesc', [ColumnStatisticsDesc, None], None, ), # 1 (2, TType.LIST, 'statsObj', (TType.STRUCT, [ColumnStatisticsObj, None], False), None, ), # 2 (3, TType.BOOL, 'isStatsCompliant', None, None, ), # 3 - (4, TType.STRING, 'engine', 'UTF8', None, ), # 4 + (4, TType.STRING, 'engine', 'UTF8', "hive", ), # 4 ) all_structs.append(FileMetadata) FileMetadata.thrift_spec = ( @@ -31845,7 +31839,7 @@ def __ne__(self, other): (2, TType.BOOL, 'needMerge', None, None, ), # 2 (3, TType.I64, 'writeId', None, -1, ), # 3 (4, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 4 - (5, TType.STRING, 'engine', 'UTF8', None, ), # 5 + (5, TType.STRING, 'engine', 'UTF8', "hive", ), # 5 ) all_structs.append(SetPartitionsStatsResponse) SetPartitionsStatsResponse.thrift_spec = ( @@ -32044,7 +32038,7 @@ def __ne__(self, other): (3, TType.LIST, 'colNames', (TType.STRING, 'UTF8', False), None, ), # 3 (4, TType.STRING, 'catName', 'UTF8', None, ), # 4 (5, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 5 - (6, TType.STRING, 'engine', 'UTF8', None, ), # 6 + (6, TType.STRING, 'engine', 'UTF8', "hive", ), # 6 (7, TType.I64, 'id', None, -1, ), # 7 ) all_structs.append(PartitionsStatsRequest) @@ -32056,7 +32050,7 @@ def __ne__(self, other): (4, TType.LIST, 'partNames', (TType.STRING, 'UTF8', False), None, ), # 4 (5, TType.STRING, 'catName', 'UTF8', None, ), # 5 (6, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 6 - (7, TType.STRING, 'engine', 'UTF8', None, ), # 7 + (7, TType.STRING, 'engine', 'UTF8', "hive", ), # 7 ) all_structs.append(AddPartitionsResult) AddPartitionsResult.thrift_spec = ( @@ -32142,7 +32136,7 @@ def __ne__(self, other): (4, TType.BOOL, 'get_col_stats', None, None, ), # 4 (5, TType.LIST, 'processorCapabilities', (TType.STRING, 'UTF8', False), None, ), # 5 (6, TType.STRING, 'processorIdentifier', 'UTF8', None, ), # 6 - (7, TType.STRING, 'engine', 'UTF8', None, ), # 7 + (7, TType.STRING, 'engine', 'UTF8', "hive", ), # 7 (8, TType.STRING, 'validWriteIdList', 'UTF8', None, ), # 8 (9, TType.BOOL, 'getFileMetadata', None, None, ), # 9 (10, TType.I64, 'id', None, -1, ), # 10 @@ -32834,7 +32828,7 @@ def __ne__(self, other): (7, TType.BOOL, 'getColumnStats', None, None, ), # 7 (8, TType.LIST, 'processorCapabilities', (TType.STRING, 'UTF8', False), None, ), # 8 (9, TType.STRING, 'processorIdentifier', 'UTF8', None, ), # 9 - (10, TType.STRING, 'engine', 'UTF8', None, ), # 10 + (10, TType.STRING, 'engine', 'UTF8', "hive", ), # 10 (11, TType.I64, 'id', None, -1, ), # 11 ) all_structs.append(GetTableResult) diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb index 634f58351bb5..b5fcabee2504 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -2342,7 +2342,7 @@ class ColumnStatistics STATSDESC => {:type => ::Thrift::Types::STRUCT, :name => 'statsDesc', :class => ::ColumnStatisticsDesc}, STATSOBJ => {:type => ::Thrift::Types::LIST, :name => 'statsObj', :element => {:type => ::Thrift::Types::STRUCT, :class => ::ColumnStatisticsObj}}, ISSTATSCOMPLIANT => {:type => ::Thrift::Types::BOOL, :name => 'isStatsCompliant', :optional => true}, - ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :optional => true} + ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :default => %q"hive", :optional => true} } def struct_fields; FIELDS; end @@ -2656,14 +2656,13 @@ class SetPartitionsStatsRequest NEEDMERGE => {:type => ::Thrift::Types::BOOL, :name => 'needMerge', :optional => true}, WRITEID => {:type => ::Thrift::Types::I64, :name => 'writeId', :default => -1, :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, - ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine'} + ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :default => %q"hive", :optional => true} } def struct_fields; FIELDS; end def validate raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field colStats is unset!') unless @colStats - raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field engine is unset!') unless @engine end ::Thrift::Struct.generate_accessors self @@ -3274,7 +3273,7 @@ class TableStatsRequest COLNAMES => {:type => ::Thrift::Types::LIST, :name => 'colNames', :element => {:type => ::Thrift::Types::STRING}}, CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, - ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine'}, + ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :default => %q"hive", :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true} } @@ -3284,7 +3283,6 @@ def validate raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tblName is unset!') unless @tblName raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field colNames is unset!') unless @colNames - raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field engine is unset!') unless @engine end ::Thrift::Struct.generate_accessors self @@ -3307,7 +3305,7 @@ class PartitionsStatsRequest PARTNAMES => {:type => ::Thrift::Types::LIST, :name => 'partNames', :element => {:type => ::Thrift::Types::STRING}}, CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, - ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine'} + ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :default => %q"hive", :optional => true} } def struct_fields; FIELDS; end @@ -3317,7 +3315,6 @@ def validate raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tblName is unset!') unless @tblName raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field colNames is unset!') unless @colNames raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field partNames is unset!') unless @partNames - raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field engine is unset!') unless @engine end ::Thrift::Struct.generate_accessors self @@ -3574,7 +3571,7 @@ class GetPartitionsByNamesRequest GET_COL_STATS => {:type => ::Thrift::Types::BOOL, :name => 'get_col_stats', :optional => true}, PROCESSORCAPABILITIES => {:type => ::Thrift::Types::LIST, :name => 'processorCapabilities', :element => {:type => ::Thrift::Types::STRING}, :optional => true}, PROCESSORIDENTIFIER => {:type => ::Thrift::Types::STRING, :name => 'processorIdentifier', :optional => true}, - ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :optional => true}, + ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :default => %q"hive", :optional => true}, VALIDWRITEIDLIST => {:type => ::Thrift::Types::STRING, :name => 'validWriteIdList', :optional => true}, GETFILEMETADATA => {:type => ::Thrift::Types::BOOL, :name => 'getFileMetadata', :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true}, @@ -5705,7 +5702,7 @@ class GetTableRequest GETCOLUMNSTATS => {:type => ::Thrift::Types::BOOL, :name => 'getColumnStats', :optional => true}, PROCESSORCAPABILITIES => {:type => ::Thrift::Types::LIST, :name => 'processorCapabilities', :element => {:type => ::Thrift::Types::STRING}, :optional => true}, PROCESSORIDENTIFIER => {:type => ::Thrift::Types::STRING, :name => 'processorIdentifier', :optional => true}, - ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :optional => true}, + ENGINE => {:type => ::Thrift::Types::STRING, :name => 'engine', :default => %q"hive", :optional => true}, ID => {:type => ::Thrift::Types::I64, :name => 'id', :default => -1, :optional => true} } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index 9b7cbb82df67..7d484bf44cd7 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -1272,7 +1272,8 @@ public Map> getPartitionColumnStatistics( List colNames, String engine, String validWriteIdList) throws NoSuchObjectException, MetaException, TException { PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, tableName, colNames, - partNames == null ? new ArrayList() : partNames, engine); + partNames == null ? new ArrayList() : partNames); + rqst.setEngine(engine); rqst.setCatName(catName); rqst.setValidWriteIdList(validWriteIdList); return client.get_partitions_statistics_req(rqst).getPartStats(); @@ -1297,7 +1298,8 @@ public AggrStats getAggrColStatsFor(String catName, String dbName, String tblNam LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side."); return new AggrStats(new ArrayList<>(), 0); // Nothing to aggregate } - PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames, engine); + PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames); + req.setEngine(engine); req.setCatName(catName); req.setValidWriteIdList(writeIdList); @@ -3446,7 +3448,8 @@ public List getTableColumnStatistics(String catName, String if (colNames.isEmpty()) { return Collections.emptyList(); } - TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames, engine); + TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames); + rqst.setEngine(engine); rqst.setCatName(catName); rqst.setEngine(engine); return getTableColumnStatisticsInternal(rqst).getTableStats(); @@ -3479,7 +3482,7 @@ public List getTableColumnStatistics(String catName, String if (colNames.isEmpty()) { return Collections.emptyList(); } - TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames, engine); + TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames); rqst.setEngine(engine); rqst.setCatName(catName); rqst.setValidWriteIdList(validWriteIdList); @@ -3505,8 +3508,8 @@ public Map> getPartitionColumnStatistics( public Map> getPartitionColumnStatistics( String catName, String dbName, String tableName, List partNames, List colNames, String engine) throws TException { - PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, tableName, colNames, - partNames, engine); + PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, tableName, colNames, partNames); + rqst.setEngine(engine); rqst.setCatName(catName); rqst.setValidWriteIdList(getValidWriteIdList(dbName, tableName)); return client.get_partitions_statistics_req(rqst).getPartStats(); @@ -4659,7 +4662,8 @@ public AggrStats getAggrColStatsFor(String catName, String dbName, String tblNam LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side."); return new AggrStats(new ArrayList<>(), 0); // Nothing to aggregate } - PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames, engine); + PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames); + req.setEngine(engine); req.setCatName(catName); req.setValidWriteIdList(getValidWriteIdList(dbName, tblName)); diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift index 422f23fdfffc..cbe1d5c96b38 100644 --- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift +++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift @@ -600,7 +600,7 @@ struct ColumnStatistics { 2: required list statsObj, 3: optional bool isStatsCompliant, // Are the stats isolation-level-compliant with the // the calling query? -4: optional string engine +4: optional string engine = "hive" } // FileMetadata represents the table-level (in case of unpartitioned) or partition-level @@ -725,7 +725,7 @@ struct SetPartitionsStatsRequest { 2: optional bool needMerge, //stats need to be merged with the existing stats 3: optional i64 writeId=-1, // writeId for the current query that updates the stats 4: optional string validWriteIdList, // valid write id list for the table for which this struct is being sent -5: required string engine //engine creating the current request +5: optional string engine = "hive" //engine creating the current request } struct SetPartitionsStatsResponse { @@ -901,7 +901,7 @@ struct TableStatsRequest { 3: required list colNames 4: optional string catName, 5: optional string validWriteIdList, // valid write id list for the table for which this struct is being sent - 6: required string engine, //engine creating the current request + 6: optional string engine = "hive", //engine creating the current request 7: optional i64 id=-1 // table id } @@ -912,7 +912,7 @@ struct PartitionsStatsRequest { 4: required list partNames, 5: optional string catName, 6: optional string validWriteIdList, // valid write id list for the table for which this struct is being sent - 7: required string engine //engine creating the current request + 7: optional string engine = "hive" //engine creating the current request } // Return type for add_partitions_req @@ -993,7 +993,7 @@ struct GetPartitionsByNamesRequest { 4: optional bool get_col_stats, 5: optional list processorCapabilities, 6: optional string processorIdentifier, - 7: optional string engine, + 7: optional string engine = "hive", 8: optional string validWriteIdList, 9: optional bool getFileMetadata, 10: optional i64 id=-1, // table id @@ -1710,7 +1710,7 @@ struct GetTableRequest { 7: optional bool getColumnStats, 8: optional list processorCapabilities, 9: optional string processorIdentifier, - 10: optional string engine, + 10: optional string engine = "hive", 11: optional i64 id=-1 // table id } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java index 96695ee1eaaf..07b4f25c3bc1 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java @@ -1918,15 +1918,17 @@ public void flushCache() { public List getTableColumnStatistics(String dbName, String tableName, List colNames, String engine) throws NoSuchObjectException, MetaException, TException, InvalidInputException, InvalidObjectException { - return client.get_table_statistics_req( - new TableStatsRequest(dbName, tableName, colNames, engine)).getTableStats(); + TableStatsRequest tsr = new TableStatsRequest(dbName, tableName, colNames); + tsr.setEngine(engine); + return client.get_table_statistics_req(new TableStatsRequest(tsr)).getTableStats(); } @Override public List getTableColumnStatistics( String dbName, String tableName, List colNames, String engine, String validWriteIdList) throws NoSuchObjectException, MetaException, TException { - TableStatsRequest tsr = new TableStatsRequest(dbName, tableName, colNames, engine); + TableStatsRequest tsr = new TableStatsRequest(dbName, tableName, colNames); + tsr.setEngine(engine); tsr.setValidWriteIdList(validWriteIdList); return client.get_table_statistics_req(tsr).getTableStats(); @@ -1937,8 +1939,9 @@ public List getTableColumnStatistics( public Map> getPartitionColumnStatistics( String dbName, String tableName, List partNames, List colNames, String engine) throws NoSuchObjectException, MetaException, TException { - return client.get_partitions_statistics_req( - new PartitionsStatsRequest(dbName, tableName, colNames, partNames, engine)).getPartStats(); + PartitionsStatsRequest psr = new PartitionsStatsRequest(dbName, tableName, colNames, partNames); + psr.setEngine(engine); + return client.get_partitions_statistics_req(new PartitionsStatsRequest(psr)).getPartStats(); } @Override @@ -1946,7 +1949,8 @@ public Map> getPartitionColumnStatistics( String dbName, String tableName, List partNames, List colNames, String engine, String validWriteIdList) throws NoSuchObjectException, MetaException, TException { - PartitionsStatsRequest psr = new PartitionsStatsRequest(dbName, tableName, colNames, partNames, engine); + PartitionsStatsRequest psr = new PartitionsStatsRequest(dbName, tableName, colNames, partNames); + psr.setEngine(engine); psr.setValidWriteIdList(validWriteIdList); return client.get_partitions_statistics_req( psr).getPartStats(); @@ -2945,7 +2949,8 @@ public AggrStats getAggrColStatsFor(String dbName, String tblName, LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side."); return new AggrStats(new ArrayList<>(),0); // Nothing to aggregate } - PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames, engine); + PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames); + req.setEngine(engine); return client.get_aggr_stats_for(req); } @@ -2958,7 +2963,8 @@ public AggrStats getAggrColStatsFor( LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side."); return new AggrStats(new ArrayList<>(),0); // Nothing to aggregate } - PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partName, engine); + PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partName); + req.setEngine(engine); req.setValidWriteIdList(writeIdList); return client.get_aggr_stats_for(req); } From 00d0b6d7b94e3db00d671542efa3c3cf1ad14714 Mon Sep 17 00:00:00 2001 From: Simhadri Govindappa Date: Thu, 1 Feb 2024 15:30:38 +0530 Subject: [PATCH 160/179] HIVE-27938: Iceberg: Fix java.lang.ClassCastException during vectorized reads on partition columns (#5048) (Simhadri Govindappa reviewed by Sourabh Badhya) --- .../HiveIdentityPartitionConverters.java | 69 +++++++++ .../mr/mapreduce/IcebergInputFormat.java | 5 +- .../iceberg_partition_vectorized_read.q | 24 +++ .../iceberg_partition_vectorized_read.q.out | 139 ++++++++++++++++++ 4 files changed, 235 insertions(+), 2 deletions(-) create mode 100644 iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/HiveIdentityPartitionConverters.java create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_partition_vectorized_read.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/iceberg_partition_vectorized_read.q.out diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/HiveIdentityPartitionConverters.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/HiveIdentityPartitionConverters.java new file mode 100644 index 000000000000..6c51de9dabbe --- /dev/null +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/HiveIdentityPartitionConverters.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.mapreduce; + +import java.math.BigDecimal; +import org.apache.avro.generic.GenericData; +import org.apache.hadoop.hive.common.type.Date; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.Timestamp; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; + +public class HiveIdentityPartitionConverters { + + private HiveIdentityPartitionConverters() { + } + + public static Object convertConstant(Type type, Object value) { + if (value == null) { + return null; + } + + switch (type.typeId()) { + case STRING: + return value.toString(); + case TIME: + return DateTimeUtil.timeFromMicros((Long) value); + case DATE: + return Date.ofEpochDay((Integer) value); + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + return DateTimeUtil.timestamptzFromMicros((Long) value).toOffsetTime(); + } else { + return new Timestamp(DateTimeUtil.timestampFromMicros((Long) value)); + } + case DECIMAL: + if (value.getClass().isAssignableFrom(BigDecimal.class)) { + return HiveDecimal.create((BigDecimal) value); + } + return value; + case FIXED: + if (value instanceof GenericData.Fixed) { + return ((GenericData.Fixed) value).bytes(); + } + return value; + default: + } + return value; + } + +} diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 3ec1a3b3b7a5..754d78e4d933 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -391,7 +391,7 @@ private CloseableIterable openVectorized(FileScanTask task, Schema readSchema "Vectorized read is unsupported for Hive 2 integration."); Path path = new Path(task.file().path().toString()); - Map idToConstant = constantsMap(task, IdentityPartitionConverters::convertConstant); + Map idToConstant = constantsMap(task, HiveIdentityPartitionConverters::convertConstant); Expression residual = HiveIcebergInputFormat.residualForTask(task, context.getConfiguration()); // TODO: We have to take care of the EncryptionManager when LLAP and vectorization is used @@ -544,7 +544,8 @@ private CloseableIterable newOrcIterable(InputFile inputFile, FileScanTask ta Types.StructType partitionType = Partitioning.partitionType(table); return PartitionUtil.constantsMap(task, partitionType, converter); } else if (projectsIdentityPartitionColumns) { - return PartitionUtil.constantsMap(task, converter); + Types.StructType partitionType = Partitioning.partitionType(table); + return PartitionUtil.constantsMap(task, partitionType, converter); } else { return Collections.emptyMap(); } diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_partition_vectorized_read.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_partition_vectorized_read.q new file mode 100644 index 000000000000..506f69488712 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_partition_vectorized_read.q @@ -0,0 +1,24 @@ +set hive.vectorized.execution.enabled=true; + +CREATE EXTERNAL TABLE ice_date (`col1` int, `day` date, `calday` date) PARTITIONED BY SPEC (calday) stored by +iceberg tblproperties('format-version'='2'); +insert into ice_date values(1, '2020-11-20', '2020-11-20'), (1, '2020-11-20', '2020-11-20'); +select * from ice_date; +select count(calday) from ice_date; +select distinct(calday) from ice_date; + + +CREATE EXTERNAL TABLE ice_timestamp (`col1` int, `day` date, `times` timestamp) PARTITIONED BY SPEC (times) stored +by iceberg tblproperties('format-version'='2'); +insert into ice_timestamp values(1, '2020-11-20', '2020-11-20'), (1, '2020-11-20', '2020-11-20'); +select * from ice_timestamp; +select count(times) from ice_timestamp; +select distinct(times) from ice_timestamp; + + +CREATE EXTERNAL TABLE ice_decimal (`col1` int, `decimalA` decimal(5,2), `decimalC` decimal(5,2)) PARTITIONED BY SPEC +(decimalC) stored by iceberg tblproperties('format-version'='2'); +insert into ice_decimal values(1, 122.91, 102.21), (1, 12.32, 200.12); +select * from ice_decimal; +select distinct(decimalc) from ice_decimal; +select count(decimala) from ice_decimal where decimala=122.91; diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_partition_vectorized_read.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_partition_vectorized_read.q.out new file mode 100644 index 000000000000..3cc643380fd2 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_partition_vectorized_read.q.out @@ -0,0 +1,139 @@ +PREHOOK: query: CREATE EXTERNAL TABLE ice_date (`col1` int, `day` date, `calday` date) PARTITIONED BY SPEC (calday) stored by +iceberg tblproperties('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_date +POSTHOOK: query: CREATE EXTERNAL TABLE ice_date (`col1` int, `day` date, `calday` date) PARTITIONED BY SPEC (calday) stored by +iceberg tblproperties('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_date +PREHOOK: query: insert into ice_date values(1, '2020-11-20', '2020-11-20'), (1, '2020-11-20', '2020-11-20') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_date +POSTHOOK: query: insert into ice_date values(1, '2020-11-20', '2020-11-20'), (1, '2020-11-20', '2020-11-20') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_date +PREHOOK: query: select * from ice_date +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 2020-11-20 2020-11-20 +1 2020-11-20 2020-11-20 +PREHOOK: query: select count(calday) from ice_date +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(calday) from ice_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 +PREHOOK: query: select distinct(calday) from ice_date +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_date +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select distinct(calday) from ice_date +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_date +POSTHOOK: Output: hdfs://### HDFS PATH ### +2020-11-20 +PREHOOK: query: CREATE EXTERNAL TABLE ice_timestamp (`col1` int, `day` date, `times` timestamp) PARTITIONED BY SPEC (times) stored +by iceberg tblproperties('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_timestamp +POSTHOOK: query: CREATE EXTERNAL TABLE ice_timestamp (`col1` int, `day` date, `times` timestamp) PARTITIONED BY SPEC (times) stored +by iceberg tblproperties('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_timestamp +PREHOOK: query: insert into ice_timestamp values(1, '2020-11-20', '2020-11-20'), (1, '2020-11-20', '2020-11-20') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_timestamp +POSTHOOK: query: insert into ice_timestamp values(1, '2020-11-20', '2020-11-20'), (1, '2020-11-20', '2020-11-20') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_timestamp +PREHOOK: query: select * from ice_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 2020-11-20 2020-11-20 00:00:00 +1 2020-11-20 2020-11-20 00:00:00 +PREHOOK: query: select count(times) from ice_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(times) from ice_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +2 +PREHOOK: query: select distinct(times) from ice_timestamp +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_timestamp +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select distinct(times) from ice_timestamp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_timestamp +POSTHOOK: Output: hdfs://### HDFS PATH ### +2020-11-20 00:00:00 +PREHOOK: query: CREATE EXTERNAL TABLE ice_decimal (`col1` int, `decimalA` decimal(5,2), `decimalC` decimal(5,2)) PARTITIONED BY SPEC +(decimalC) stored by iceberg tblproperties('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_decimal +POSTHOOK: query: CREATE EXTERNAL TABLE ice_decimal (`col1` int, `decimalA` decimal(5,2), `decimalC` decimal(5,2)) PARTITIONED BY SPEC +(decimalC) stored by iceberg tblproperties('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_decimal +PREHOOK: query: insert into ice_decimal values(1, 122.91, 102.21), (1, 12.32, 200.12) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_decimal +POSTHOOK: query: insert into ice_decimal values(1, 122.91, 102.21), (1, 12.32, 200.12) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_decimal +PREHOOK: query: select * from ice_decimal +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_decimal +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from ice_decimal +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_decimal +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 122.91 102.21 +1 12.32 200.12 +PREHOOK: query: select distinct(decimalc) from ice_decimal +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_decimal +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select distinct(decimalc) from ice_decimal +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_decimal +POSTHOOK: Output: hdfs://### HDFS PATH ### +102.21 +200.12 +PREHOOK: query: select count(decimala) from ice_decimal where decimala=122.91 +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_decimal +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(decimala) from ice_decimal where decimala=122.91 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_decimal +POSTHOOK: Output: hdfs://### HDFS PATH ### +1 From e892ce7212b2f0c9e55092d89076b129c86c8172 Mon Sep 17 00:00:00 2001 From: veghlaci05 Date: Thu, 1 Feb 2024 11:02:15 +0100 Subject: [PATCH 161/179] HIVE-27481: Addendum: Fix post-refactor issues (Laszlo Vegh, reviewed by Attila Turoczy, Denys Kuzmenko) Closes #5010 --- .../TestMaterializedViewRebuild.java | 33 +++++++++++++++---- .../txn/jdbc/MultiDataSourceJdbcResource.java | 29 ++++++---------- .../txn/jdbc/functions/OnRenameFunction.java | 28 ++++++++-------- .../ReleaseMaterializationRebuildLocks.java | 18 +++++----- .../queries/LatestTxnIdInConflictHandler.java | 10 ++++-- 5 files changed, 70 insertions(+), 48 deletions(-) diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMaterializedViewRebuild.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMaterializedViewRebuild.java index a0bf2608bfbe..d38e6695cb49 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMaterializedViewRebuild.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMaterializedViewRebuild.java @@ -17,20 +17,27 @@ */ package org.apache.hadoop.hive.ql.txn.compactor; -import java.util.Arrays; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.metastore.api.CompactionType; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; -import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.executeStatementOnDriver; import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.execSelectAndDumpData; +import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.executeStatementOnDriver; import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.executeStatementOnDriverSilently; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.when; public class TestMaterializedViewRebuild extends CompactorOnTezTest { @@ -182,4 +189,18 @@ private void assertResult(List expected, List actual) { Assert.assertEquals(expected, actual); } + @Test + public void testMaterializationLockCleaned() throws Exception { + TxnStore txnHandler = TxnUtils.getTxnStore(conf); + OpenTxnsResponse response = txnHandler.openTxns(new OpenTxnRequest(1, "user", "host")); + txnHandler.lockMaterializationRebuild("default", TABLE1, response.getTxn_ids().get(0)); + + //Mimic the lock can be cleaned up + ValidTxnList validTxnList = Mockito.mock(ValidReadTxnList.class); + when(validTxnList.isTxnValid(anyLong())).thenReturn(true); + + long removedCnt = txnHandler.cleanupMaterializationRebuildLocks(validTxnList, 10); + Assert.assertEquals(1, removedCnt); + } + } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java index 101172c74077..7ab42c1336d8 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/MultiDataSourceJdbcResource.java @@ -173,9 +173,9 @@ public DatabaseProduct getDatabaseProduct() { * @throws MetaException Forwarded from {@link ParameterizedCommand#getParameterizedQueryString(DatabaseProduct)} or * thrown if the update count was rejected by the {@link ParameterizedCommand#resultPolicy()} method */ - public Integer execute(ParameterizedCommand command) throws MetaException { + public int execute(ParameterizedCommand command) throws MetaException { if (!shouldExecute(command)) { - return null; + return -1; } try { return execute(command.getParameterizedQueryString(getDatabaseProduct()), @@ -191,32 +191,23 @@ public Integer execute(ParameterizedCommand command) throws MetaException { * call using the query string obtained from {@link ParameterizedBatchCommand#getParameterizedQueryString(DatabaseProduct)}, * the parameters obtained from {@link ParameterizedBatchCommand#getQueryParameters()}, and the * {@link org.springframework.jdbc.core.PreparedStatementSetter} obtained from - * {@link ParameterizedBatchCommand#getPreparedStatementSetter()} methods. The batchSize is coming fomr the - * {@link Configuration} object. After the execution, this method validates the resulted number of affected rows using the - * {@link ParameterizedBatchCommand#resultPolicy()} function for each element in the batch. + * {@link ParameterizedBatchCommand#getPreparedStatementSetter()} methods. The batchSize is coming from the + * {@link Configuration} object. * * @param command The {@link ParameterizedBatchCommand} to execute. - * @return Returns an integer array,containing the number of affected rows for each element in the batch. */ - public int execute(ParameterizedBatchCommand command) throws MetaException { + public int[][] execute(ParameterizedBatchCommand command) throws MetaException { if (!shouldExecute(command)) { - return 0; + return null; } try { int maxBatchSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.JDBC_MAX_BATCH_SIZE); - int[][] result = getJdbcTemplate().getJdbcTemplate().batchUpdate( + return getJdbcTemplate().getJdbcTemplate().batchUpdate( command.getParameterizedQueryString(databaseProduct), command.getQueryParameters(), maxBatchSize, command.getPreparedStatementSetter() - ); - - Function resultPolicy = command.resultPolicy(); - if (resultPolicy != null && !Arrays.stream(result).allMatch(inner -> Arrays.stream(inner).allMatch(resultPolicy::apply))) { - LOG.error("The update count was rejected in at least one of the result array. Rolling back."); - throw new MetaException("The update count was rejected in at least one of the result array. Rolling back."); - } - return Arrays.stream(result).reduce(0, (acc, i) -> acc + Arrays.stream(i).sum(), Integer::sum); + ); } catch (Exception e) { handleError(command, e); throw e; @@ -300,7 +291,7 @@ public int execute(InClauseBatchCommand command) throws MetaException { * @throws MetaException Forwarded from {@link ParameterizedCommand#getParameterizedQueryString(DatabaseProduct)} or * thrown if the update count was rejected by the {@link ParameterizedCommand#resultPolicy()} method */ - public Integer execute(String query, SqlParameterSource params, + public int execute(String query, SqlParameterSource params, Function resultPolicy) throws MetaException { LOG.debug("Going to execute command <{}>", query); int count = getJdbcTemplate().update(query, params); @@ -322,7 +313,7 @@ public Integer execute(String query, SqlParameterSource params, * @return Returns with the object(s) constructed from the result of the executed query. * @throws MetaException Forwarded from {@link ParameterizedCommand#getParameterizedQueryString(DatabaseProduct)}. */ - public Result execute(QueryHandler queryHandler) throws MetaException { + public T execute(QueryHandler queryHandler) throws MetaException { String queryStr = queryHandler.getParameterizedQueryString(getDatabaseProduct()); LOG.debug("Going to execute query <{}>", queryStr); SqlParameterSource params = queryHandler.getQueryParameters(); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java index 69173857f43e..1167ee4f42a2 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/OnRenameFunction.java @@ -26,6 +26,8 @@ import org.springframework.dao.DataAccessException; import org.springframework.jdbc.core.namedparam.MapSqlParameterSource; +import java.sql.Types; + public class OnRenameFunction implements TransactionalFunction { private static final Logger LOG = LoggerFactory.getLogger(OnRenameFunction.class); @@ -105,12 +107,12 @@ public class OnRenameFunction implements TransactionalFunction { public OnRenameFunction(String oldCatName, String oldDbName, String oldTabName, String oldPartName, String newCatName, String newDbName, String newTabName, String newPartName) { this.oldCatName = oldCatName; - this.oldDbName = oldDbName; - this.oldTabName = oldTabName; + this.oldDbName = StringUtils.lowerCase(oldDbName); + this.oldTabName = StringUtils.lowerCase(oldTabName); this.oldPartName = oldPartName; this.newCatName = newCatName; - this.newDbName = newDbName; - this.newTabName = newTabName; + this.newDbName = StringUtils.lowerCase(newDbName); + this.newTabName = StringUtils.lowerCase(newTabName); this.newPartName = newPartName; } @@ -121,23 +123,23 @@ public Void execute(MultiDataSourceJdbcResource jdbcResource) throws MetaExcepti oldCatName + "," + oldDbName + "," + oldTabName + "," + oldPartName + "," + newCatName + "," + newDbName + "," + newTabName + "," + newPartName + ")"; - if(newPartName != null) { + if (newPartName != null) { assert oldPartName != null && oldTabName != null && oldDbName != null && oldCatName != null : callSig; } - if(newTabName != null) { + if (newTabName != null) { assert oldTabName != null && oldDbName != null && oldCatName != null : callSig; } - if(newDbName != null) { + if (newDbName != null) { assert oldDbName != null && oldCatName != null : callSig; } MapSqlParameterSource paramSource = new MapSqlParameterSource() - .addValue("oldDbName", StringUtils.lowerCase(oldDbName)) - .addValue("newDbName", StringUtils.lowerCase(newDbName)) - .addValue("oldTableName", StringUtils.lowerCase(oldTabName)) - .addValue("newTableName", StringUtils.lowerCase(newTabName)) - .addValue("oldPartName", oldPartName) - .addValue("newPartName", newPartName); + .addValue("oldDbName", oldDbName, Types.VARCHAR) + .addValue("newDbName", newDbName, Types.VARCHAR) + .addValue("oldTableName", oldTabName, Types.VARCHAR) + .addValue("newTableName", newTabName, Types.VARCHAR) + .addValue("oldPartName", oldPartName, Types.VARCHAR) + .addValue("newPartName", newPartName, Types.VARCHAR); try { for (String command : UPDATE_COMMANNDS) { jdbcResource.getJdbcTemplate().update(command, paramSource); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java index 0ddebae5c588..86f1af05e6ad 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/functions/ReleaseMaterializationRebuildLocks.java @@ -53,14 +53,16 @@ public Long execute(MultiDataSourceJdbcResource jdbcResource) throws MetaExcepti LOG.debug("Going to execute query <{}>", selectQ); jdbcResource.getJdbcTemplate().query(selectQ, rs -> { - long lastHeartbeat = rs.getLong(2); - if (lastHeartbeat < timeoutTime) { - // The heartbeat has timeout, double check whether we can remove it - long txnId = rs.getLong(1); - if (validTxnList.isTxnValid(txnId) || validTxnList.isTxnAborted(txnId)) { - // Txn was committed (but notification was not received) or it was aborted. - // Either case, we can clean it up - txnIds.add(txnId); + if (rs.next()) { + long lastHeartbeat = rs.getLong(2); + if (lastHeartbeat < timeoutTime) { + // The heartbeat has timeout, double check whether we can remove it + long txnId = rs.getLong(1); + if (validTxnList.isTxnValid(txnId) || validTxnList.isTxnAborted(txnId)) { + // Txn was committed (but notification was not received) or it was aborted. + // Either case, we can clean it up + txnIds.add(txnId); + } } } return null; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java index 33c7719cc711..f63748ca9662 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/LatestTxnIdInConflictHandler.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hive.metastore.txn.jdbc.queries; -import com.sun.tools.javac.util.List; +import com.google.common.collect.ImmutableList; import org.apache.hadoop.hive.metastore.DatabaseProduct; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.txn.entities.OperationType; @@ -30,8 +30,14 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Types; +import java.util.List; public class LatestTxnIdInConflictHandler implements QueryHandler { + + private static final List OPERATION_TYPES = ImmutableList.of( + OperationType.UPDATE.getSqlConst(), + OperationType.DELETE.getSqlConst() + ); private final long txnId; @@ -67,7 +73,7 @@ public String getParameterizedQueryString(DatabaseProduct databaseProduct) throw public SqlParameterSource getQueryParameters() { return new MapSqlParameterSource() .addValue("txnId", txnId) - .addValue("types", List.of(OperationType.UPDATE.getSqlConst(), OperationType.DELETE.getSqlConst()), Types.CHAR) + .addValue("types", OPERATION_TYPES, Types.CHAR) .addValue("wsType", OperationType.INSERT.getSqlConst(), Types.CHAR); } From f3439697343f3d5e1f1d007d8c878a6eb821713b Mon Sep 17 00:00:00 2001 From: InvisibleProgrammer Date: Fri, 2 Feb 2024 09:32:26 +0100 Subject: [PATCH 162/179] HIVE-27637: Compare highest write ID of compaction records when trying to perform abort cleanup (#4740) (Zsolt Miskolczi reviewed by Attila Turoczy, Sourabh Badhya) --- .../txn/TestCompactionTxnHandler.java | 44 +++++++++++++++++++ .../handler/TestAbortedTxnCleaner.java | 17 ++++++- .../queries/ReadyToCleanAbortHandler.java | 12 ++++- 3 files changed, 70 insertions(+), 3 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java index d26f3774af70..cd02eb1ba3eb 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java @@ -1056,6 +1056,50 @@ public void testFindReadyToCleanAborts_limitFetchSize() throws Exception { assertEquals(1, potentials.size()); } + @Test + public void testFindReadyToCleanAborts() throws Exception { + long txnId = openTxn(); + + List components = new ArrayList<>(); + components.add(createLockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb", "mytable", "mypartition=myvalue", DataOperationType.UPDATE)); + components.add(createLockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb", "yourtable", "mypartition=myvalue", DataOperationType.UPDATE)); + + allocateTableWriteIds("mydb", "mytable", txnId); + allocateTableWriteIds("mydb", "yourtable", txnId); + + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnId); + LockResponse res = txnHandler.lock(req); + assertSame(res.getState(), LockState.ACQUIRED); + + txnHandler.abortTxn(new AbortTxnRequest((txnId))); + + txnId = openTxn(); + components = new ArrayList<>(); + components.add(createLockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb", "mytable", "mypartition=myvalue", DataOperationType.UPDATE)); + allocateTableWriteIds("mydb", "mytable", txnId); + + req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnId); + res = txnHandler.lock(req); + assertSame(res.getState(), LockState.ACQUIRED); + + CompactionRequest rqst = new CompactionRequest("mydb", "mytable", CompactionType.MINOR); + rqst.setPartitionname("mypartition=myvalue"); + txnHandler.compact(rqst); + + CompactionInfo ci = txnHandler.findNextToCompact(aFindNextCompactRequest("fred", WORKER_VERSION)); + assertNotNull(ci); + ci.highestWriteId = 41; + txnHandler.updateCompactorState(ci, 0); + + List potentials = txnHandler.findReadyToCleanAborts(1, 0); + assertEquals(1, potentials.size()); + CompactionInfo potentialToCleanAbort = potentials.get(0); + assertEquals("mydb", potentialToCleanAbort.dbname); + assertEquals("yourtable", potentialToCleanAbort.tableName); + } + private static FindNextCompactRequest aFindNextCompactRequest(String workerId, String workerVersion) { FindNextCompactRequest request = new FindNextCompactRequest(); request.setWorkerId(workerId); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 8f6814d4890f..1d31aae94343 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -44,7 +44,9 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -282,8 +284,21 @@ public void testCleaningOfAbortedDirectoriesBelowBase() throws Exception { cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); cleaner.run(); - Mockito.verify(mockedFSRemover, Mockito.times(1)).clean(any(CleanupRequest.class)); + Mockito.verifyNoInteractions(mockedFSRemover); Mockito.verify(mockedTaskHandler, Mockito.times(1)).getTasks(); + String compactionQueuePresence = "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + + " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\" IS NULL"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, compactionQueuePresence)); + + directories = getDirectories(conf, t, null); + // Both base and delta files are present since the cleaner skips them as there is a newer write. + Assert.assertEquals(5, directories.size()); + Assert.assertEquals(1, directories.stream().filter(dir -> dir.getName().startsWith(AcidUtils.BASE_PREFIX)).count()); + + // Run compaction and clean up + startInitiator(); + startWorker(); + startCleaner(); directories = getDirectories(conf, t, null); // The table is already compacted, so we must see 1 base delta diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java index 4940d384095b..eebe29dc441a 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java @@ -52,7 +52,7 @@ public class ReadyToCleanAbortHandler implements QueryHandler \"res1\".\"MAX_ABORTED_WRITE_ID\"" + + " AND \"cq\".\"CQ_STATE\" " + + " IN ('i', 'w', 'r'))"; private final long abortedTimeThreshold; private final int abortedThreshold; From 0a133b3ec3867c734c7ad93776c89f7f7d1ffe99 Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Thu, 1 Feb 2024 15:05:45 +0100 Subject: [PATCH 163/179] HIVE-28053: Incorrect shading configuration for beeline jar-with-dependencies (Stamatis Zampetakis reviewed by Alessandro Solimando, Attila Turoczy) 1. Remove descriptorRefs cause it is not valid configuration parameter for maven-shade-plugin and has no effect on the build. 2. Set correct mainClass in the MANIFEST file making the jar runnable without extra options and avoid errors due to missing jmh class. Close apache/hive#5056 --- beeline/pom.xml | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/beeline/pom.xml b/beeline/pom.xml index 71da41a32da8..7024486ecccf 100644 --- a/beeline/pom.xml +++ b/beeline/pom.xml @@ -243,14 +243,11 @@ shade - - jar-with-dependencies - jar-with-dependencies - org.openjdk.jmh.Main + org.apache.hive.beeline.BeeLine From 0f8190fefa56513a042566ef87b852e8343610aa Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Fri, 2 Feb 2024 18:59:36 +0530 Subject: [PATCH 164/179] Revert "HIVE-27637: Compare highest write ID of compaction records when trying to perform abort cleanup (Zsolt Miskolczi reviewed by Attila Turoczy, Sourabh Badhya)" (#5058) This reverts commit f3439697343f3d5e1f1d007d8c878a6eb821713b. --- .../txn/TestCompactionTxnHandler.java | 44 ------------------- .../handler/TestAbortedTxnCleaner.java | 17 +------ .../queries/ReadyToCleanAbortHandler.java | 12 +---- 3 files changed, 3 insertions(+), 70 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java index cd02eb1ba3eb..d26f3774af70 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java @@ -1056,50 +1056,6 @@ public void testFindReadyToCleanAborts_limitFetchSize() throws Exception { assertEquals(1, potentials.size()); } - @Test - public void testFindReadyToCleanAborts() throws Exception { - long txnId = openTxn(); - - List components = new ArrayList<>(); - components.add(createLockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb", "mytable", "mypartition=myvalue", DataOperationType.UPDATE)); - components.add(createLockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb", "yourtable", "mypartition=myvalue", DataOperationType.UPDATE)); - - allocateTableWriteIds("mydb", "mytable", txnId); - allocateTableWriteIds("mydb", "yourtable", txnId); - - LockRequest req = new LockRequest(components, "me", "localhost"); - req.setTxnid(txnId); - LockResponse res = txnHandler.lock(req); - assertSame(res.getState(), LockState.ACQUIRED); - - txnHandler.abortTxn(new AbortTxnRequest((txnId))); - - txnId = openTxn(); - components = new ArrayList<>(); - components.add(createLockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb", "mytable", "mypartition=myvalue", DataOperationType.UPDATE)); - allocateTableWriteIds("mydb", "mytable", txnId); - - req = new LockRequest(components, "me", "localhost"); - req.setTxnid(txnId); - res = txnHandler.lock(req); - assertSame(res.getState(), LockState.ACQUIRED); - - CompactionRequest rqst = new CompactionRequest("mydb", "mytable", CompactionType.MINOR); - rqst.setPartitionname("mypartition=myvalue"); - txnHandler.compact(rqst); - - CompactionInfo ci = txnHandler.findNextToCompact(aFindNextCompactRequest("fred", WORKER_VERSION)); - assertNotNull(ci); - ci.highestWriteId = 41; - txnHandler.updateCompactorState(ci, 0); - - List potentials = txnHandler.findReadyToCleanAborts(1, 0); - assertEquals(1, potentials.size()); - CompactionInfo potentialToCleanAbort = potentials.get(0); - assertEquals("mydb", potentialToCleanAbort.dbname); - assertEquals("yourtable", potentialToCleanAbort.tableName); - } - private static FindNextCompactRequest aFindNextCompactRequest(String workerId, String workerVersion) { FindNextCompactRequest request = new FindNextCompactRequest(); request.setWorkerId(workerId); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 1d31aae94343..8f6814d4890f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -44,9 +44,7 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -284,21 +282,8 @@ public void testCleaningOfAbortedDirectoriesBelowBase() throws Exception { cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); cleaner.run(); - Mockito.verifyNoInteractions(mockedFSRemover); + Mockito.verify(mockedFSRemover, Mockito.times(1)).clean(any(CleanupRequest.class)); Mockito.verify(mockedTaskHandler, Mockito.times(1)).getTasks(); - String compactionQueuePresence = "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + - " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\" IS NULL"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, compactionQueuePresence)); - - directories = getDirectories(conf, t, null); - // Both base and delta files are present since the cleaner skips them as there is a newer write. - Assert.assertEquals(5, directories.size()); - Assert.assertEquals(1, directories.stream().filter(dir -> dir.getName().startsWith(AcidUtils.BASE_PREFIX)).count()); - - // Run compaction and clean up - startInitiator(); - startWorker(); - startCleaner(); directories = getDirectories(conf, t, null); // The table is already compacted, so we must see 1 base delta diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java index eebe29dc441a..4940d384095b 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/jdbc/queries/ReadyToCleanAbortHandler.java @@ -52,7 +52,7 @@ public class ReadyToCleanAbortHandler implements QueryHandler \"res1\".\"MAX_ABORTED_WRITE_ID\"" + - " AND \"cq\".\"CQ_STATE\" " + - " IN ('i', 'w', 'r'))"; + " WHERE \"res3\".\"RETRY_RECORD_CHECK\" <= 0 OR \"res3\".\"RETRY_RECORD_CHECK\" IS NULL"; private final long abortedTimeThreshold; private final int abortedThreshold; From 8b735924b4d3bbd1d066d7f3508314b95ee55e2d Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Mon, 5 Feb 2024 15:32:38 +0800 Subject: [PATCH 165/179] HIVE-28038: Disable fallback to jdo for DeadlineException (#5040)(Wechar Yu, reviewed by Butao Zhang) --- .../hive/metastore/DatabaseProduct.java | 10 +-- .../hive/metastore/TestObjectStore.java | 66 ++++++++++--------- 2 files changed, 40 insertions(+), 36 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java index ea3faf09113d..642057bd69a8 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Stream; -import java.util.stream.Collectors; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.conf.Configurable; @@ -52,9 +51,10 @@ * */ public class DatabaseProduct implements Configurable { static final private Logger LOG = LoggerFactory.getLogger(DatabaseProduct.class.getName()); - private static final Class[] unrecoverableSqlExceptions = new Class[]{ - // TODO: collect more unrecoverable SQLExceptions - SQLIntegrityConstraintViolationException.class + private static final Class[] unrecoverableExceptions = new Class[]{ + // TODO: collect more unrecoverable Exceptions + SQLIntegrityConstraintViolationException.class, + DeadlineException.class }; public enum DbType {DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER, CUSTOM, UNDEFINED}; @@ -164,7 +164,7 @@ private static DbType getDbType(String productName) { } public static boolean isRecoverableException(Throwable t) { - return Stream.of(unrecoverableSqlExceptions) + return Stream.of(unrecoverableExceptions) .allMatch(ex -> ExceptionUtils.indexOfType(t, ex) < 0); } diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java index 380863b716c6..1da814dd0927 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java @@ -1635,40 +1635,44 @@ protected Object getJdoResult(ObjectStore.GetHelper ctx) throws MetaExce @Test public void testNoJdoForUnrecoverableException() throws Exception { - objectStore.openTransaction(); - AtomicBoolean runDirectSql = new AtomicBoolean(false); - AtomicBoolean runJdo = new AtomicBoolean(false); - try { - objectStore.new GetHelper(DEFAULT_CATALOG_NAME, DB1, TABLE1, true, true) { - @Override - protected String describeResult() { - return "test not run jdo for unrecoverable exception"; - } + Exception[] unrecoverableExceptions = new Exception[] { + new SQLIntegrityConstraintViolationException("Unrecoverable ex"), + new DeadlineException("unrecoverable ex")}; + for (Exception unrecoverableException : unrecoverableExceptions) { + objectStore.openTransaction(); + AtomicBoolean runDirectSql = new AtomicBoolean(false); + AtomicBoolean runJdo = new AtomicBoolean(false); + try { + objectStore.new GetHelper(DEFAULT_CATALOG_NAME, DB1, TABLE1, true, true) { + @Override + protected String describeResult() { + return "test not run jdo for unrecoverable exception"; + } - @Override - protected Object getSqlResult(ObjectStore.GetHelper ctx) throws MetaException { - runDirectSql.set(true); - SQLIntegrityConstraintViolationException ex = new SQLIntegrityConstraintViolationException("Unrecoverable ex"); - MetaException me = new MetaException("Throwing unrecoverable exception to test not run jdo."); - me.initCause(ex); - throw me; - } + @Override + protected Object getSqlResult(ObjectStore.GetHelper ctx) throws MetaException { + runDirectSql.set(true); + MetaException me = new MetaException("Throwing unrecoverable exception to test not run jdo."); + me.initCause(unrecoverableException); + throw me; + } - @Override - protected Object getJdoResult(ObjectStore.GetHelper ctx) throws MetaException, NoSuchObjectException { - runJdo.set(true); - SQLIntegrityConstraintViolationException ex = new SQLIntegrityConstraintViolationException("Unrecoverable ex"); - MetaException me = new MetaException("Throwing unrecoverable exception to test not run jdo."); - me.initCause(ex); - throw me; - } - }.run(false); - } catch (MetaException ex) { - // expected + @Override + protected Object getJdoResult(ObjectStore.GetHelper ctx) throws MetaException, NoSuchObjectException { + runJdo.set(true); + SQLIntegrityConstraintViolationException ex = new SQLIntegrityConstraintViolationException("Unrecoverable ex"); + MetaException me = new MetaException("Throwing unrecoverable exception to test not run jdo."); + me.initCause(ex); + throw me; + } + }.run(false); + } catch (MetaException ex) { + // expected + } + objectStore.commitTransaction(); + Assert.assertEquals(true, runDirectSql.get()); + Assert.assertEquals(false, runJdo.get()); } - objectStore.commitTransaction(); - Assert.assertEquals(true, runDirectSql.get()); - Assert.assertEquals(false, runJdo.get()); } /** From af545603951c24f6559ed2b2fb0b7286b0c3d5be Mon Sep 17 00:00:00 2001 From: Stamatis Zampetakis Date: Thu, 1 Feb 2024 11:15:44 +0100 Subject: [PATCH 166/179] HIVE-26818: Beeline module misses transitive dependencies due to shading (Stamatis Zampetakis reviewed by Attila Turoczy, Ayush Saxena, Kokila N) Due to shading, the dependecy-reduced-pom.xml file is installed in the local maven repository (~/.m2/repository/org/apache/hive/hive-beeline/4.0.0-SNAPSHOT/) for beeline. The latter indicates that the module doesn't have any transitive dependencies. If we were publishing the shaded jar that would be true but we publish the regular jar. At this point, modules which include hive-beeline as a maven dependency are broken leading to build problems and classpath errors. 1. Do not generate the dependency-reduced-pom.xml and publish the regular pom.xml file along with the main jar file. 2. Remove some redundant explicit dependency declarations that were added to workaround the missing dependency problem from the beeline module. Close apache/hive#5055 --- beeline/pom.xml | 1 + itests/hive-unit/pom.xml | 6 ------ itests/util/pom.xml | 4 ---- 3 files changed, 1 insertion(+), 10 deletions(-) diff --git a/beeline/pom.xml b/beeline/pom.xml index 7024486ecccf..a308ef8b6e2b 100644 --- a/beeline/pom.xml +++ b/beeline/pom.xml @@ -243,6 +243,7 @@ shade + false jar-with-dependencies diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index fd64a7853171..7c3b2333b9eb 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -494,12 +494,6 @@ postgresql test - - net.sf.supercsv - super-csv - ${super-csv.version} - test - diff --git a/itests/util/pom.xml b/itests/util/pom.xml index 97ebb9c83aa2..556879b30aba 100644 --- a/itests/util/pom.xml +++ b/itests/util/pom.xml @@ -103,10 +103,6 @@ hive-exec tests - - org.apache.hive - hive-jdbc - org.apache.hive hive-standalone-metastore-common From 4b01a607091581ac9bdb372f8b47c1efca4d4bb4 Mon Sep 17 00:00:00 2001 From: Wechar Yu Date: Tue, 6 Feb 2024 17:15:18 +0800 Subject: [PATCH 167/179] HIVE-27958: Refactor DirectSqlUpdatePart class (Wechar Yu, reviewed by Attila Turoczy, Denys Kuzmenko) Closes #5003 --- .../hive/metastore/DatabaseProduct.java | 23 +++ .../hive/metastore/DirectSqlUpdatePart.java | 192 ++++++------------ .../metastore/txn/retry/SqlRetryHandler.java | 27 +-- 3 files changed, 87 insertions(+), 155 deletions(-) diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java index 642057bd69a8..b2b20503d249 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; import org.apache.commons.lang3.exception.ExceptionUtils; @@ -57,6 +58,11 @@ public class DatabaseProduct implements Configurable { DeadlineException.class }; + /** + * Derby specific concurrency control + */ + private static final ReentrantLock derbyLock = new ReentrantLock(true); + public enum DbType {DERBY, MYSQL, POSTGRES, ORACLE, SQLSERVER, CUSTOM, UNDEFINED}; public DbType dbType; @@ -776,4 +782,21 @@ public Configuration getConf() { public void setConf(Configuration c) { myConf = c; } + + /** + * lockInternal() and {@link #unlockInternal()} are used to serialize those operations that require + * Select ... For Update to sequence operations properly. In practice that means when running + * with Derby database. See more notes at class level. + */ + public void lockInternal() { + if (isDERBY()) { + derbyLock.lock(); + } + } + + public void unlockInternal() { + if (isDERBY()) { + derbyLock.unlock(); + } + } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java index 67c293ee64f4..441ce26ac6d7 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdatePart.java @@ -67,7 +67,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE; @@ -92,8 +91,6 @@ class DirectSqlUpdatePart { private final int maxBatchSize; private final SQLGenerator sqlGenerator; - private static final ReentrantLock derbyLock = new ReentrantLock(true); - public DirectSqlUpdatePart(PersistenceManager pm, Configuration conf, DatabaseProduct dbType, int batchSize) { this.pm = pm; @@ -103,23 +100,6 @@ public DirectSqlUpdatePart(PersistenceManager pm, Configuration conf, sqlGenerator = new SQLGenerator(dbType, conf); } - /** - * {@link #lockInternal()} and {@link #unlockInternal()} are used to serialize those operations that require - * Select ... For Update to sequence operations properly. In practice that means when running - * with Derby database. See more notes at class level. - */ - private void lockInternal() { - if(dbType.isDERBY()) { - derbyLock.lock(); - } - } - - private void unlockInternal() { - if(dbType.isDERBY()) { - derbyLock.unlock(); - } - } - void rollbackDBConn(Connection dbConn) { try { if (dbConn != null && !dbConn.isClosed()) dbConn.rollback(); @@ -138,43 +118,16 @@ void closeDbConn(JDOConnection jdoConn) { } } - void closeStmt(Statement stmt) { - try { - if (stmt != null && !stmt.isClosed()) stmt.close(); - } catch (SQLException e) { - LOG.warn("Failed to close statement ", e); - } - } - - void close(ResultSet rs) { - try { - if (rs != null && !rs.isClosed()) { - rs.close(); - } - } - catch(SQLException ex) { - LOG.warn("Failed to close statement ", ex); - } - } - static String quoteString(String input) { return "'" + input + "'"; } - void close(ResultSet rs, Statement stmt, JDOConnection dbConn) { - close(rs); - closeStmt(stmt); - closeDbConn(dbConn); - } - private void populateInsertUpdateMap(Map statsPartInfoMap, Map updateMap, MapinsertMap, Connection dbConn, Table tbl) throws SQLException, MetaException, NoSuchObjectException { StringBuilder prefix = new StringBuilder(); StringBuilder suffix = new StringBuilder(); - Statement statement = null; - ResultSet rs = null; List queries = new ArrayList<>(); Set selectedParts = new HashSet<>(); @@ -186,16 +139,14 @@ private void populateInsertUpdateMap(Map statsP TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, partIdList, "\"PART_ID\"", true, false); - for (String query : queries) { - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute query " + query); - rs = statement.executeQuery(query); - while (rs.next()) { - selectedParts.add(new PartColNameInfo(rs.getLong(1), rs.getString(2), rs.getString(3))); + try (Statement statement = dbConn.createStatement()) { + for (String query : queries) { + LOG.debug("Execute query: " + query); + try (ResultSet rs = statement.executeQuery(query)) { + while (rs.next()) { + selectedParts.add(new PartColNameInfo(rs.getLong(1), rs.getString(2), rs.getString(3))); + } } - } finally { - close(rs, statement, null); } } @@ -246,13 +197,13 @@ private void updatePartColStatTable(Map(); } } if (!partIds.isEmpty()) { - LOG.debug("Going to execute updates on part: {}", partIds); + LOG.debug("Execute updates on part: {}", partIds); verifyUpdates(pst.executeBatch(), partIds); } } @@ -270,7 +221,6 @@ private void verifyUpdates(int[] numUpdates, List partIds) throws MetaExce private void insertIntoPartColStatTable(Map insertMap, long maxCsId, Connection dbConn) throws SQLException, MetaException, NoSuchObjectException { - PreparedStatement preparedStatement = null; int numRows = 0; String insert = "INSERT INTO \"PART_COL_STATS\" (\"CS_ID\", \"CAT_NAME\", \"DB_NAME\"," + "\"TABLE_NAME\", \"PARTITION_NAME\", \"COLUMN_NAME\", \"COLUMN_TYPE\", \"PART_ID\"," @@ -279,8 +229,7 @@ private void insertIntoPartColStatTable(Map getParamValues(Connection dbConn, List partIdLis List queries = new ArrayList<>(); StringBuilder prefix = new StringBuilder(); StringBuilder suffix = new StringBuilder(); - Statement statement = null; - ResultSet rs = null; prefix.append("select \"PART_ID\", \"PARAM_VALUE\" " + " from \"PARTITION_PARAMS\" where " @@ -343,18 +288,17 @@ private Map getParamValues(Connection dbConn, List partIdLis partIdList, "\"PART_ID\"", true, false); Map partIdToParaMap = new HashMap<>(); - for (String query : queries) { - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute query " + query); - rs = statement.executeQuery(query); - while (rs.next()) { - partIdToParaMap.put(rs.getLong(1), rs.getString(2)); + try (Statement statement = dbConn.createStatement()) { + for (String query : queries) { + LOG.debug("Execute query: " + query); + try (ResultSet rs = statement.executeQuery(query)) { + while (rs.next()) { + partIdToParaMap.put(rs.getLong(1), rs.getString(2)); + } } - } finally { - close(rs, statement, null); } } + return partIdToParaMap; } @@ -367,14 +311,10 @@ private void updateWriteIdForPartitions(Connection dbConn, long writeId, List getPartitionInfo(Connection dbConn, List queries = new ArrayList<>(); StringBuilder prefix = new StringBuilder(); StringBuilder suffix = new StringBuilder(); - Statement statement = null; - ResultSet rs = null; Map partitionInfoMap = new HashMap<>(); List partKeys = partColStatsMap.keySet().stream().map( @@ -516,20 +454,18 @@ private Map getPartitionInfo(Connection dbConn, TxnUtils.buildQueryWithINClauseStrings(conf, queries, prefix, suffix, partKeys, "\"PART_NAME\"", true, false); - for (String query : queries) { - // Select for update makes sure that the partitions are not modified while the stats are getting updated. - query = sqlGenerator.addForUpdateClause(query); - try { - statement = dbConn.createStatement(); - LOG.debug("Going to execute query <" + query + ">"); - rs = statement.executeQuery(query); - while (rs.next()) { - PartitionInfo partitionInfo = new PartitionInfo(rs.getLong(1), - rs.getLong(2), rs.getString(3)); - partitionInfoMap.put(partitionInfo, partColStatsMap.get(rs.getString(3))); + try (Statement statement = dbConn.createStatement()) { + for (String query : queries) { + // Select for update makes sure that the partitions are not modified while the stats are getting updated. + query = sqlGenerator.addForUpdateClause(query); + LOG.debug("Execute query: " + query); + try (ResultSet rs = statement.executeQuery(query)) { + while (rs.next()) { + PartitionInfo partitionInfo = new PartitionInfo(rs.getLong(1), + rs.getLong(2), rs.getString(3)); + partitionInfoMap.put(partitionInfo, partColStatsMap.get(rs.getString(3))); + } } - } finally { - close(rs, statement, null); } } return partitionInfoMap; @@ -556,7 +492,7 @@ public Map> updatePartitionColumnStatistics(Map> updatePartitionColumnStatistics(Map> updatePartitionColumnStatistics(Map threadLocal = new StackThreadLocal<>(); - /** - * Derby specific concurrency control - */ - private static final ReentrantLock derbyLock = new ReentrantLock(true); - private final DatabaseProduct databaseProduct; private final long deadlockRetryInterval; private final long retryInterval; @@ -122,14 +116,14 @@ public Result executeWithRetry(SqlRetryCallProperties properties, SqlRe try { if (properties.isLockInternally()) { - lockInternal(); + databaseProduct.lockInternal(); } threadLocal.set(new Object()); return executeWithRetryInternal(properties, function); } finally { threadLocal.unset(); if (properties.isLockInternally()) { - unlockInternal(); + databaseProduct.unlockInternal(); } } } @@ -269,21 +263,4 @@ static boolean isRetryable(Configuration conf, Exception ex) { } return false; } - - /** - * lockInternal() and {@link #unlockInternal()} are used to serialize those operations that require - * Select ... For Update to sequence operations properly. In practice that means when running - * with Derby database. See more notes at class level. - */ - private void lockInternal() { - if(databaseProduct.isDERBY()) { - derbyLock.lock(); - } - } - private void unlockInternal() { - if(databaseProduct.isDERBY()) { - derbyLock.unlock(); - } - } - } From 57e686e9d67809ea6ef77f7e1109cc1c08fc656b Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 7 Feb 2024 10:42:59 +0530 Subject: [PATCH 168/179] HIVE-28052: Iceberg: Major QB Compaction fails with ClassNotFoundException: org.springframework.core.ErrorCoded (#5061). (Ayush Saxena, reviewed by Butao Zhang, Attila Turoczy) --- pom.xml | 3 ++- ql/pom.xml | 2 +- standalone-metastore/pom.xml | 3 ++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 1317235e40c8..82a9f753d379 100644 --- a/pom.xml +++ b/pom.xml @@ -223,7 +223,8 @@ 1.0.1 1.12.499 2.4.0 - 5.2.24.RELEASE + + 5.3.21 diff --git a/ql/pom.xml b/ql/pom.xml index 30bd3e310359..ec43088346b5 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -26,7 +26,7 @@ .. 0.10.2 - 2.1.0 + 2.3.0 diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml index caba6ea7f230..9e9e47197fce 100644 --- a/standalone-metastore/pom.xml +++ b/standalone-metastore/pom.xml @@ -110,7 +110,8 @@ 9.31 9.4.40.v20210413 1.3.2 - 5.2.24.RELEASE + + 5.3.21 you-must-set-this-to-run-thrift ${basedir}/src/gen/thrift From 56e7aae05016603e6064eb55e9bbe49bfab934f0 Mon Sep 17 00:00:00 2001 From: Soumyakanti Das Date: Wed, 7 Feb 2024 00:12:11 -0800 Subject: [PATCH 169/179] HIVE-28054: SemanticException for join condition in subquery (Soumyakanti Das, reviewed by Krisztian Kasa, Henri Biestro) --- .../ql/parse/type/RexNodeExprFactory.java | 25 ++-- .../queries/clientpositive/subquery_join.q | 17 +++ .../clientpositive/llap/subquery_join.q.out | 115 ++++++++++++++++++ 3 files changed, 145 insertions(+), 12 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/subquery_join.q create mode 100644 ql/src/test/results/clientpositive/llap/subquery_join.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/RexNodeExprFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/RexNodeExprFactory.java index a57203878dda..ee8779188d33 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/RexNodeExprFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/RexNodeExprFactory.java @@ -165,22 +165,23 @@ protected RexNode createColumnRefExpr(ColumnInfo colInfo, List rowR private int getPosition(ColumnInfo colInfo, List rowResolverList) throws SemanticException { - ColumnInfo tmp; - ColumnInfo cInfoToRet = null; int position = 0; - for (RowResolver rr : rowResolverList) { - tmp = rr.get(colInfo.getTabAlias(), colInfo.getAlias()); - if (tmp != null) { - if (cInfoToRet != null) { - throw new CalciteSemanticException("Could not resolve column name"); - } - cInfoToRet = tmp; - position += rr.getPosition(cInfoToRet.getInternalName()); - } else if (cInfoToRet == null) { + + for (RowResolver rr: rowResolverList) { + ColumnInfo tmp = rr.get(colInfo.getTabAlias(), colInfo.getAlias()); + if (tmp == null) { + // if column is not present in the RR, increment position by size of RR position += rr.getColumnInfos().size(); + } else { + // if column is present, increment position by the position of the column in RR + // and return early. + position += rr.getPosition(tmp.getInternalName()); + return position; } } - return position; + + // If we are out of the for loop, then the column is not present in any RR + throw new CalciteSemanticException("Could not resolve column name"); } /** diff --git a/ql/src/test/queries/clientpositive/subquery_join.q b/ql/src/test/queries/clientpositive/subquery_join.q new file mode 100644 index 000000000000..f9dc9795e060 --- /dev/null +++ b/ql/src/test/queries/clientpositive/subquery_join.q @@ -0,0 +1,17 @@ +create table t1 (id int); +create table t2 (id int); + +explain cbo select id, + (select count(*) from t1 join t2 on t1.id=t2.id) + from t2 +order by id; + +explain cbo select id, + (select count(*) from t1 join t2 using (id)) + from t2 +order by id; + +explain cbo select id, + (select count(*) from t1 join t2 where t1.id=t2.id) + from t2 +order by id; diff --git a/ql/src/test/results/clientpositive/llap/subquery_join.q.out b/ql/src/test/results/clientpositive/llap/subquery_join.q.out new file mode 100644 index 000000000000..737a2cab3951 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/subquery_join.q.out @@ -0,0 +1,115 @@ +PREHOOK: query: create table t1 (id int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1 (id int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: create table t2 (id int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t2 +POSTHOOK: query: create table t2 (id int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t2 +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain cbo select id, + (select count(*) from t1 join t2 on t1.id=t2.id) + from t2 +order by id +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: explain cbo select id, + (select count(*) from t1 join t2 on t1.id=t2.id) + from t2 +order by id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +CBO PLAN: +HiveSortLimit(sort0=[$0], dir0=[ASC]) + HiveProject(id=[$0], $f0=[$1]) + HiveJoin(condition=[true], joinType=[left], algorithm=[none], cost=[not available]) + HiveProject(id=[$0]) + HiveTableScan(table=[[default, t2]], table:alias=[t2]) + HiveProject($f0=[$0]) + HiveAggregate(group=[{}], agg#0=[count()]) + HiveJoin(condition=[=($0, $1)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(id=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, t1]], table:alias=[t1]) + HiveProject(id=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, t2]], table:alias=[t2]) + +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain cbo select id, + (select count(*) from t1 join t2 using (id)) + from t2 +order by id +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: explain cbo select id, + (select count(*) from t1 join t2 using (id)) + from t2 +order by id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +CBO PLAN: +HiveSortLimit(sort0=[$0], dir0=[ASC]) + HiveProject(id=[$0], $f0=[$1]) + HiveJoin(condition=[true], joinType=[left], algorithm=[none], cost=[not available]) + HiveProject(id=[$0]) + HiveTableScan(table=[[default, t2]], table:alias=[t2]) + HiveProject($f0=[$0]) + HiveAggregate(group=[{}], agg#0=[count()]) + HiveJoin(condition=[=($0, $1)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(id=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, t1]], table:alias=[t1]) + HiveProject(id=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, t2]], table:alias=[t2]) + +Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +PREHOOK: query: explain cbo select id, + (select count(*) from t1 join t2 where t1.id=t2.id) + from t2 +order by id +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: explain cbo select id, + (select count(*) from t1 join t2 where t1.id=t2.id) + from t2 +order by id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +CBO PLAN: +HiveSortLimit(sort0=[$0], dir0=[ASC]) + HiveProject(id=[$0], $f0=[$1]) + HiveJoin(condition=[true], joinType=[left], algorithm=[none], cost=[not available]) + HiveProject(id=[$0]) + HiveTableScan(table=[[default, t2]], table:alias=[t2]) + HiveProject($f0=[$0]) + HiveAggregate(group=[{}], agg#0=[count()]) + HiveJoin(condition=[=($0, $1)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(id=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, t1]], table:alias=[t1]) + HiveProject(id=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, t2]], table:alias=[t2]) + From 01e262cd3f259a7ecb4b23e789ff1c189ab2de68 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Fri, 9 Feb 2024 12:01:56 +0530 Subject: [PATCH 170/179] HIVE-28057: Iceberg: Branches with non-lowercase characters can't be accessed. (#5068). (Ayush Saxena, reviewed by Denys Kuzmenko) --- .../mr/hive/TestHiveIcebergBranchOperation.java | 14 ++++++++++++++ .../apache/hadoop/hive/ql/metadata/HiveUtils.java | 8 ++++++++ .../hadoop/hive/ql/parse/SemanticAnalyzer.java | 2 +- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java index d892a72b8221..9f88bae3496b 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java @@ -238,4 +238,18 @@ public void testCreateBranchFromTag() throws IOException, InterruptedException { "ALTER TABLE customers CREATE BRANCH %s FOR TAG AS OF %s", branchName2, branchName1))) .isInstanceOf(IllegalArgumentException.class).hasMessageEndingWith("does not exist"); } + + @Test + public void testCreateBranchWithNonLowerCase() throws InterruptedException, IOException { + Table table = + testTables.createTableWithVersions(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 2); + + String branchName = "test_Branch_1"; + Long snapshotId = table.history().get(0).snapshotId(); + shell.executeStatement( + String.format("ALTER TABLE customers CREATE BRANCH %s FOR SYSTEM_VERSION AS OF %d", branchName, snapshotId)); + // Select with non-lower case branch name shouldn't throw exception. + shell.executeStatement(String.format("SELECT * FROM default.customers.branch_%s", branchName)); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java index d1d197ebece3..ae8849b8528d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java @@ -468,4 +468,12 @@ public static Boolean isTableTag(String refName) { Matcher ref = TAG.matcher(refName); return ref.matches(); } + + public static String getLowerCaseTableName(String refName) { + String[] refParts = refName.split("\\."); + if (refParts.length == 3 && SNAPSHOT_REF.matcher(refParts[2]).matches()) { + return (refParts[0] + "." + refParts[1]).toLowerCase() + "." + refParts[2]; + } + return refName.toLowerCase(); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 5b84437ecef4..9bc8e03d1644 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -1137,7 +1137,7 @@ private String processTable(QB qb, ASTNode tabref) throws SemanticException { ASTNode tableTree = (ASTNode) (tabref.getChild(0)); - String tabIdName = getUnescapedName(tableTree).toLowerCase(); + String tabIdName = HiveUtils.getLowerCaseTableName(getUnescapedName(tableTree)); String alias = findSimpleTableName(tabref, aliasIndex); From 40b3bb484046cc9af4dd5004ae18e9dd3f34e82c Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 9 Feb 2024 12:13:11 +0100 Subject: [PATCH 171/179] HIVE-27992: Upgrade to tez 0.10.3 (#4991) (Laszlo Bodor reviewed by Ayush Saxena) --- .../org/apache/hadoop/hive/ql/QOutProcessor.java | 4 ++++ pom.xml | 2 +- ql/src/test/results/clientnegative/broken_pipe.q.out | 8 ++++---- .../results/clientnegative/cachingprintstream.q.out | 12 ++++++------ .../results/clientnegative/script_broken_pipe3.q.out | 8 ++++---- .../test/results/clientnegative/script_error.q.out | 8 ++++---- ql/src/test/results/clientnegative/stack_trace.q.out | 8 ++++---- 7 files changed, 27 insertions(+), 23 deletions(-) diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java index 5d199ce0239c..da872f17cfcc 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java @@ -337,6 +337,10 @@ private final static class PatternReplacementPair { ppm.add(new PatternReplacementPair(Pattern.compile("vertex_[0-9_]+"), "vertex_#ID#")); ppm.add(new PatternReplacementPair(Pattern.compile("task_[0-9_]+"), "task_#ID#")); + // since TEZ-4506, the node is reported with task attempt failures, which needs to be masked + ppm.add(new PatternReplacementPair(Pattern.compile("Error: Node: (.*) : Error while running task"), + "Error: Node: #NODE# : Error while running task")); + ppm.add(new PatternReplacementPair(Pattern.compile("rowcount = [0-9]+(\\.[0-9]+(E[0-9]+)?)?, cumulative cost = \\{.*\\}, id = [0-9]*"), "rowcount = ###Masked###, cumulative cost = ###Masked###, id = ###Masked###")); diff --git a/pom.xml b/pom.xml index 82a9f753d379..b0644f9b3917 100644 --- a/pom.xml +++ b/pom.xml @@ -198,7 +198,7 @@ 1.7.30 4.0.4 4.0.0-beta-2-SNAPSHOT - 0.10.2 + 0.10.3 2.2.0 1.1 1.1.10.4 diff --git a/ql/src/test/results/clientnegative/broken_pipe.q.out b/ql/src/test/results/clientnegative/broken_pipe.q.out index 3b830247418f..6eaf02a92139 100644 --- a/ql/src/test/results/clientnegative/broken_pipe.q.out +++ b/ql/src/test/results/clientnegative/broken_pipe.q.out @@ -3,13 +3,13 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: hdfs://### HDFS PATH ### Status: Failed -Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### @@ -18,13 +18,13 @@ Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An e ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE] [Masked Vertex killed due to OTHER_VERTEX_FAILURE] DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### diff --git a/ql/src/test/results/clientnegative/cachingprintstream.q.out b/ql/src/test/results/clientnegative/cachingprintstream.q.out index db6abc66ecb0..062139fc710a 100644 --- a/ql/src/test/results/clientnegative/cachingprintstream.q.out +++ b/ql/src/test/results/clientnegative/cachingprintstream.q.out @@ -9,7 +9,7 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### Status: Failed -Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -21,7 +21,7 @@ Caused by: java.io.IOException: Cannot run program "FAKE_SCRIPT_SHOULD_NOT_EXIST #### A masked pattern was here #### Caused by: java.io.IOException: error=2, No such file or directory #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -41,7 +41,7 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### Status: Failed -Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -53,7 +53,7 @@ Caused by: java.io.IOException: Cannot run program "FAKE_SCRIPT_SHOULD_NOT_EXIST #### A masked pattern was here #### Caused by: java.io.IOException: error=2, No such file or directory #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -68,7 +68,7 @@ Caused by: java.io.IOException: error=2, No such file or directory ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE] DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0 End cached logs. -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -80,7 +80,7 @@ Caused by: java.io.IOException: Cannot run program "FAKE_SCRIPT_SHOULD_NOT_EXIST #### A masked pattern was here #### Caused by: java.io.IOException: error=2, No such file or directory #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### diff --git a/ql/src/test/results/clientnegative/script_broken_pipe3.q.out b/ql/src/test/results/clientnegative/script_broken_pipe3.q.out index 2ef43286244e..cfe51bcfa223 100644 --- a/ql/src/test/results/clientnegative/script_broken_pipe3.q.out +++ b/ql/src/test/results/clientnegative/script_broken_pipe3.q.out @@ -3,13 +3,13 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### Status: Failed -Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. +Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### @@ -17,13 +17,13 @@ Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An e #### A masked pattern was here #### ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Reducer 2] killed/failed due to:OWN_TASK_FAILURE] DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 2, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### diff --git a/ql/src/test/results/clientnegative/script_error.q.out b/ql/src/test/results/clientnegative/script_error.q.out index 00b1ac2c7814..9c184f60a69b 100644 --- a/ql/src/test/results/clientnegative/script_error.q.out +++ b/ql/src/test/results/clientnegative/script_error.q.out @@ -55,13 +55,13 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### Status: Failed -Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### @@ -69,13 +69,13 @@ Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An e #### A masked pattern was here #### ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE] DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators #### A masked pattern was here #### diff --git a/ql/src/test/results/clientnegative/stack_trace.q.out b/ql/src/test/results/clientnegative/stack_trace.q.out index 73fef35981f0..6d4331cdefe3 100644 --- a/ql/src/test/results/clientnegative/stack_trace.q.out +++ b/ql/src/test/results/clientnegative/stack_trace.q.out @@ -3,7 +3,7 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: hdfs://### HDFS PATH ### Status: Failed -Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -15,7 +15,7 @@ Caused by: java.io.IOException: Cannot run program "script_does_not_exist": erro #### A masked pattern was here #### Caused by: java.io.IOException: error=2, No such file or directory #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -29,7 +29,7 @@ Caused by: java.io.IOException: error=2, No such file or directory #### A masked pattern was here #### ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE] DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### @@ -41,7 +41,7 @@ Caused by: java.io.IOException: Cannot run program "script_does_not_exist": erro #### A masked pattern was here #### Caused by: java.io.IOException: error=2, No such file or directory #### A masked pattern was here #### -], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row +], TaskAttempt 1 failed, info=[Error: Node: #NODE# : Error while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row #### A masked pattern was here #### From 32a79270c33e151826b43eec0daa985b159fc568 Mon Sep 17 00:00:00 2001 From: Dmitriy Fingerman Date: Fri, 9 Feb 2024 08:28:49 -0500 Subject: [PATCH 172/179] HIVE-27850: Iceberg: Addendum: Use fully qualified table name in compaction query (Dmitriy Fingerman, reviewed by Denys Kuzmenko) Closes #5074 --- .../mr/hive/compaction/IcebergMajorQueryCompactor.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java index 96141e50494d..e3dba519dc96 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext; import org.apache.hadoop.hive.ql.txn.compactor.QueryCompactor; +import org.apache.hive.iceberg.org.apache.orc.storage.common.TableName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,12 +40,12 @@ public class IcebergMajorQueryCompactor extends QueryCompactor { @Override public boolean run(CompactorContext context) throws IOException, HiveException, InterruptedException { - String compactTableName = context.getTable().getTableName(); + String compactTableName = TableName.getDbTable(context.getTable().getDbName(), context.getTable().getTableName()); Map tblProperties = context.getTable().getParameters(); LOG.debug("Initiating compaction for the {} table", compactTableName); - String compactionQuery = String.format("insert overwrite table %s select * from %s", - compactTableName, compactTableName); + String compactionQuery = String.format("insert overwrite table %s select * from % Date: Mon, 12 Feb 2024 08:17:42 +0100 Subject: [PATCH 173/179] HIVE-28050: Disable Incremental non aggregated materialized view rebuild in presence of delete operations (Krisztian Kasa, reviewed by Stamatis Zampetakis, Aman Sinha) --- .../AlterMaterializedViewRebuildAnalyzer.java | 109 +--- ...nInsertDeleteIncrementalRewritingRule.java | 212 ------- .../hive/ql/parse/SemanticAnalyzer.java | 3 +- .../materialized_view_create_rewrite_12.q | 36 ++ .../materialized_view_create_rewrite_5.q | 3 + .../materialized_view_create_rewrite_8.q | 3 + .../materialized_view_join_rebuild.q | 33 -- .../materialized_view_repeated_rebuild.q | 37 -- .../materialized_view_create_rewrite_12.q.out | 211 +++++++ .../materialized_view_create_rewrite_5.q.out | 532 ++++++------------ .../materialized_view_create_rewrite_8.q.out | 255 +++------ .../llap/materialized_view_join_rebuild.q.out | 108 ---- .../materialized_view_repeated_rebuild.q.out | 134 ----- 13 files changed, 509 insertions(+), 1167 deletions(-) delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java create mode 100644 ql/src/test/queries/clientpositive/materialized_view_create_rewrite_12.q delete mode 100644 ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q delete mode 100644 ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q create mode 100644 ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_12.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out delete mode 100644 ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java index 3542ac06ab0d..823e574cb40d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/materialized/alter/rebuild/AlterMaterializedViewRebuildAnalyzer.java @@ -44,17 +44,13 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization; import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.metadata.VirtualColumn; -import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; import org.apache.hadoop.hive.ql.optimizer.calcite.HiveTezModelRelMetadataProvider; import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveInBetweenExpandRule; -import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.ColumnPropagationException; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveAggregateInsertDeleteIncrementalRewritingRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveAggregateInsertIncrementalRewritingRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveAggregatePartitionIncrementalRewritingRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveInsertOnlyScanWriteIdRule; -import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveJoinInsertDeleteIncrementalRewritingRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveJoinInsertIncrementalRewritingRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializationRelMetadataProvider; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewRule; @@ -322,8 +318,7 @@ private RelNode applyRecordIncrementalRebuildPlan( } return applyAggregateInsertDeleteIncremental(basePlan, mdProvider, executorProvider); } else { - return applyJoinInsertDeleteIncremental( - basePlan, mdProvider, executorProvider, optCluster, calcitePreMVRewritingPlan); + return calcitePreMVRewritingPlan; } } else { return calcitePreMVRewritingPlan; @@ -373,21 +368,6 @@ private RelNode applyAggregateInsertIncremental( } } - private RelNode applyJoinInsertDeleteIncremental( - RelNode basePlan, RelMetadataProvider mdProvider, RexExecutor executorProvider, RelOptCluster optCluster, - RelNode calcitePreMVRewritingPlan) { - basePlan = applyIncrementalRebuild( - basePlan, mdProvider, executorProvider, HiveJoinInsertDeleteIncrementalRewritingRule.INSTANCE); - mvRebuildMode = MaterializationRebuildMode.JOIN_INSERT_DELETE_REBUILD; - try { - return new HiveJoinInsertDeleteIncrementalRewritingRule.FilterPropagator( - HiveRelFactories.HIVE_BUILDER.create(optCluster, null)).propagate(basePlan); - } catch (ColumnPropagationException ex) { - LOG.warn("Exception while propagating column " + VirtualColumn.ROWISDELETED.getName(), ex); - return calcitePreMVRewritingPlan; - } - } - private RelNode applyJoinInsertIncremental( RelNode basePlan, RelMetadataProvider mdProvider, RexExecutor executorProvider) { mvRebuildMode = MaterializationRebuildMode.JOIN_INSERT_REBUILD; @@ -479,9 +459,6 @@ protected ASTNode fixUpAfterCbo(ASTNode originalAst, ASTNode newAst, CalcitePlan case AGGREGATE_INSERT_DELETE_REBUILD: fixUpASTAggregateInsertDeleteIncrementalRebuild(fixedAST, getMaterializedViewASTBuilder()); return fixedAST; - case JOIN_INSERT_DELETE_REBUILD: - fixUpASTJoinInsertDeleteIncrementalRebuild(fixedAST, getMaterializedViewASTBuilder()); - return fixedAST; default: throw new UnsupportedOperationException("No materialized view rebuild exists for mode " + mvRebuildMode); } @@ -752,90 +729,6 @@ private void fixUpASTJoinInsertIncrementalRebuild(ASTNode newAST) throws Semanti destParent.insertChild(childIndex, newChild); } - private void fixUpASTJoinInsertDeleteIncrementalRebuild(ASTNode newAST, MaterializedViewASTBuilder astBuilder) - throws SemanticException { - // Replace INSERT OVERWRITE by MERGE equivalent rewriting. - // Here we need to do this complex AST rewriting that generates the same plan - // that a MERGE clause would generate because CBO does not support MERGE yet. - // TODO: Support MERGE as first class member in CBO to simplify this logic. - // 1) Replace INSERT OVERWRITE by INSERT - ASTNode insertNode = new ASTSearcher().simpleBreadthFirstSearch( - newAST, HiveParser.TOK_QUERY, HiveParser.TOK_INSERT); - ASTNode destinationNode = (ASTNode) insertNode.getChild(0); - ASTNode newInsertInto = (ASTNode) ParseDriver.adaptor.create( - HiveParser.TOK_INSERT_INTO, "TOK_INSERT_INTO"); - newInsertInto.addChildren(destinationNode.getChildren()); - ASTNode destinationParentNode = (ASTNode) destinationNode.getParent(); - int childIndex = destinationNode.childIndex; - destinationParentNode.deleteChild(childIndex); - destinationParentNode.insertChild(childIndex, newInsertInto); - // 1.1) Extract name as we will need it afterwards: - // TOK_DESTINATION TOK_TAB TOK_TABNAME - ASTNode materializationNode = new ASTSearcher().simpleBreadthFirstSearch( - newInsertInto, HiveParser.TOK_INSERT_INTO, HiveParser.TOK_TAB, HiveParser.TOK_TABNAME); - - ASTNode subqueryNodeInputROJ = new ASTSearcher().simpleBreadthFirstSearch( - newAST, HiveParser.TOK_QUERY, HiveParser.TOK_FROM, HiveParser.TOK_RIGHTOUTERJOIN, - HiveParser.TOK_SUBQUERY); - ASTNode selectNodeInputROJ = new ASTSearcher().simpleBreadthFirstSearch( - subqueryNodeInputROJ, HiveParser.TOK_SUBQUERY, HiveParser.TOK_QUERY, - HiveParser.TOK_INSERT, HiveParser.TOK_SELECT); - astBuilder.createAcidSortNodes(TableName.getDbTable( - materializationNode.getChild(0).getText(), - materializationNode.getChild(1).getText())) - .forEach(astNode -> ParseDriver.adaptor.addChild(selectNodeInputROJ, astNode)); - - ASTNode whereClauseInInsert = findWhereClause(insertNode); - - // 2) Add filter condition to Insert - // Modifying filter condition. The incremental rewriting rule generated an OR - // clause where first disjunct contains the condition for the DELETE branch. - // TOK_WHERE - // or - // . <- DISJUNCT FOR - // TOK_TABLE_OR_COL - // $hdt$_0 - // ROW__IS__DELETED - // TOK_FUNCTION <- DISJUNCT FOR - // isnull - // . - // TOK_TABLE_OR_COL - // $hdt$_0 - // ROW__IS__DELETED - if (whereClauseInInsert.getChild(0).getType() != HiveParser.KW_OR) { - throw new SemanticException("OR clause expected below TOK_WHERE in incremental rewriting"); - } - // We bypass the OR clause and select the first disjunct - int indexDelete; - int indexInsert; - if (whereClauseInInsert.getChild(0).getChild(0).getType() == HiveParser.KW_AND) { - indexDelete = 0; - indexInsert = 1; - } else if (whereClauseInInsert.getChild(0).getChild(1).getType() == HiveParser.KW_AND) { - indexDelete = 1; - indexInsert = 0; - } else { - throw new SemanticException("Unexpected condition in incremental rewriting"); - } - ASTNode deletePredicate = - (ASTNode) ParseDriver.adaptor.dupTree(whereClauseInInsert.getChild(0).getChild(indexDelete)); - ASTNode newCondInInsert = (ASTNode) whereClauseInInsert.getChild(0).getChild(indexInsert); - ParseDriver.adaptor.setChild(whereClauseInInsert, 0, newCondInInsert); - - addDeleteBranch(insertNode, subqueryNodeInputROJ, deletePredicate, astBuilder); - - // 3) Add sort node to delete branch - ASTNode sortNode = astBuilder.createSortNodes( - astBuilder.createAcidSortNodes((ASTNode) subqueryNodeInputROJ.getChild(1))); - ParseDriver.adaptor.addChild(insertNode.getParent().getChild(2), sortNode); - - // 4) Now we set some tree properties related to multi-insert - // operation with INSERT/UPDATE - ctx.setOperation(Context.Operation.MERGE); - ctx.addDestNamePrefix(1, Context.DestClausePrefix.INSERT); - ctx.addDestNamePrefix(2, Context.DestClausePrefix.DELETE); - } - @Override protected boolean allowOutputMultipleTimes() { return true; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java deleted file mode 100644 index 10a9c5c27605..000000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveJoinInsertDeleteIncrementalRewritingRule.java +++ /dev/null @@ -1,212 +0,0 @@ -package org.apache.hadoop.hive.ql.optimizer.calcite.rules.views;/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.Union; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.tools.RelBuilder; -import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; -import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin; -import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject; -import org.apache.hadoop.hive.ql.parse.CalcitePlanner; - -import java.util.ArrayList; -import java.util.List; - -/** - * This rule will perform a rewriting to prepare the plan for incremental - * view maintenance in case there is no aggregation operator but some of the - * source tables has delete operations, so we can avoid the INSERT OVERWRITE and use a - * MULTI INSERT statement instead: one insert branch for inserted rows - * and another for inserting deleted rows to delete delta. - * Since CBO plan does not contain the INSERT branches we focus on the SELECT part of the plan in this rule. - * See also {@link CalcitePlanner} - * - * FROM (select mv.ROW__ID, mv.a, mv.b, true as flag from mv) mv - * RIGHT OUTER JOIN (SELECT _source_.ROW__IS_DELETED,_source_.a, _source_.b FROM _source_) source - * ON (mv.a <=> source.a AND mv.b <=> source.b) - * INSERT INTO TABLE mv_delete_delta - * SELECT mv.ROW__ID - * WHERE source.ROW__IS__DELETED AND flag - * INSERT INTO TABLE mv - * SELECT source.a, source.b - * WHERE NOT source.ROW__IS__DELETED - * SORT BY mv.ROW__ID; - */ -public class HiveJoinInsertDeleteIncrementalRewritingRule extends RelOptRule { - - public static final HiveJoinInsertDeleteIncrementalRewritingRule INSTANCE = - new HiveJoinInsertDeleteIncrementalRewritingRule(); - - private HiveJoinInsertDeleteIncrementalRewritingRule() { - super(operand(Union.class, any()), - HiveRelFactories.HIVE_BUILDER, - "HiveJoinInsertDeleteIncrementalRewritingRule"); - } - - @Override - public void onMatch(RelOptRuleCall call) { - final Union union = call.rel(0); - RexBuilder rexBuilder = union.getCluster().getRexBuilder(); - // First branch is query, second branch is MV - // 1) First branch is query, second branch is MV - final RelNode joinLeftInput = union.getInput(1); - final RelNode joinRightInput = union.getInput(0); - - // 2) Build conditions for join and start adding - // expressions for project operator - List projExprs = new ArrayList<>(); - List joinConjs = new ArrayList<>(); - for (int leftPos = 0; leftPos < joinLeftInput.getRowType().getFieldCount(); leftPos++) { - RexNode leftRef = rexBuilder.makeInputRef( - joinLeftInput.getRowType().getFieldList().get(leftPos).getType(), leftPos); - RexNode rightRef = rexBuilder.makeInputRef( - joinRightInput.getRowType().getFieldList().get(leftPos).getType(), - leftPos + joinLeftInput.getRowType().getFieldCount()); - - projExprs.add(rightRef); - - joinConjs.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, leftRef, rightRef)); - } - - RexNode joinCond = RexUtil.composeConjunction(rexBuilder, joinConjs); - - // 3) Build plan - RelNode newNode = call.builder() - .push(union.getInput(1)) - .push(union.getInput(0)) - .join(JoinRelType.RIGHT, joinCond) - .project(projExprs) - .build(); - call.transformTo(newNode); - } - - public static class FilterPropagator extends HiveRowIsDeletedPropagator { - - private boolean foundTopRightJoin; - - public FilterPropagator(RelBuilder relBuilder) { - super(relBuilder); - } - - @Override - public RelNode propagate(RelNode relNode) { - foundTopRightJoin = false; - return super.propagate(relNode); - } - - @Override - public RelNode visit(HiveProject project) { - if (!foundTopRightJoin) { - return visitChild(project, 0, project.getInput()); - } - - // continue traversal and propagate rowIsDeleted column - return super.visit(project); - } - - @Override - public RelNode visit(HiveJoin join) { - if (!foundTopRightJoin) { - if (join.getJoinType() != JoinRelType.RIGHT) { - // continue search for top Right Join node - return visitChildren(join); - } - - foundTopRightJoin = true; - return createFilter(join); - } - - // continue traversal and propagate rowIsDeleted column - return super.visit(join); - } - - private RelNode createFilter(HiveJoin join) { - RexBuilder rexBuilder = relBuilder.getRexBuilder(); - // This should be a Scan on the MV - RelNode leftInput = join.getLeft(); - - // This branch is querying the rows should be inserted/deleted into the view since the last rebuild. - RelNode rightInput = join.getRight(); - - RelNode tmpJoin = visitChild(join, 1, rightInput); - RelNode newRightInput = tmpJoin.getInput(1); - - List leftProjects = new ArrayList<>(leftInput.getRowType().getFieldCount() + 1); - List leftProjectNames = new ArrayList<>(leftInput.getRowType().getFieldCount() + 1); - for (int i = 0; i < leftInput.getRowType().getFieldCount(); ++i) { - RelDataTypeField relDataTypeField = leftInput.getRowType().getFieldList().get(i); - leftProjects.add(rexBuilder.makeInputRef(relDataTypeField.getType(), i)); - leftProjectNames.add(relDataTypeField.getName()); - } - List projects = new ArrayList<>(leftProjects.size() + newRightInput.getRowType().getFieldCount()); - projects.addAll(leftProjects); - List projectNames = new ArrayList<>(leftProjects.size() + newRightInput.getRowType().getFieldCount()); - projectNames.addAll(leftProjectNames); - - leftProjects.add(rexBuilder.makeLiteral(true)); - leftProjectNames.add("flag"); - - leftInput = relBuilder - .push(leftInput) - .project(leftProjects, leftProjectNames) - .build(); - - // Create input ref to flag. It is used in filter condition later. - int flagIndex = leftProjects.size() - 1; - RexNode flagNode = rexBuilder.makeInputRef( - leftInput.getRowType().getFieldList().get(flagIndex).getType(), flagIndex); - - // Create input ref to rowIsDeleteColumn. It is used in filter condition later. - RelDataType newRowType = newRightInput.getRowType(); - int rowIsDeletedIdx = newRowType.getFieldCount() - 1; - RexNode rowIsDeleted = rexBuilder.makeInputRef( - newRowType.getFieldList().get(rowIsDeletedIdx).getType(), - leftInput.getRowType().getFieldCount() + rowIsDeletedIdx); - - RexNode deleteBranchFilter = rexBuilder.makeCall(SqlStdOperatorTable.AND, flagNode, rowIsDeleted); - RexNode insertBranchFilter = rexBuilder.makeCall(SqlStdOperatorTable.NOT, rowIsDeleted); - - for (int i = 0; i < newRowType.getFieldCount() - 1; ++i) { - RelDataTypeField relDataTypeField = newRowType.getFieldList().get(i); - projects.add(rexBuilder.makeInputRef(relDataTypeField.getType(), leftInput.getRowType().getFieldCount() + i)); - projectNames.add(relDataTypeField.getName()); - } - - RexNode newJoinCondition = new InputRefShifter(leftInput.getRowType().getFieldCount() - 1, relBuilder) - .apply(join.getCondition()); - - // Create new Top Right Join and a Filter. The filter condition is used in CalcitePlanner.fixUpASTJoinIncrementalRebuild(). - return relBuilder - .push(leftInput) - .push(newRightInput) - .join(join.getJoinType(), newJoinCondition) - .filter(rexBuilder.makeCall(SqlStdOperatorTable.OR, deleteBranchFilter, insertBranchFilter)) - .project(projects, projectNames) - .build(); - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 9bc8e03d1644..441e19e8ee91 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -15978,8 +15978,7 @@ protected enum MaterializationRebuildMode { INSERT_OVERWRITE_REBUILD, AGGREGATE_INSERT_REBUILD, AGGREGATE_INSERT_DELETE_REBUILD, - JOIN_INSERT_REBUILD, - JOIN_INSERT_DELETE_REBUILD + JOIN_INSERT_REBUILD } /** diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_12.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_12.q new file mode 100644 index 000000000000..5a159b7e4b88 --- /dev/null +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_12.q @@ -0,0 +1,36 @@ +-- Test rebuild of materialized view without aggregate when source tables have delete operations since last rebuild. +-- Incremental rebuild is not available. + +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table t1 (a int, b int) stored as orc TBLPROPERTIES ('transactional'='true'); + +insert into t1 values +(3, 3), +(2, 1), +(2, 2), +(1, 2), +(1, 1); + +CREATE MATERIALIZED VIEW mat1 + TBLPROPERTIES ('transactional'='true') AS +SELECT a +FROM t1 +WHERE b < 10; + +delete from t1 where b = 2; + +explain +alter materialized view mat1 rebuild; + +alter materialized view mat1 rebuild; + +explain cbo +SELECT a +FROM t1 +WHERE b < 10; + +SELECT a +FROM t1 +WHERE b < 10; diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q index 5a540250838e..cdaf1abd091b 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q @@ -11,6 +11,8 @@ set hive.materializedview.rewriting=true; create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (2, 'bob', 3.14, 3), (2, 'bonnie', 172342.2, 3), @@ -20,6 +22,7 @@ insert into cmv_basetable_n6 values create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (3, 'calvin', 978.76, 3); diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_8.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_8.q index 651bcb4319e7..aef9cb9965c8 100644 --- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_8.q +++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_8.q @@ -7,6 +7,8 @@ set hive.materializedview.rewriting=true; create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (2, 'bob', 3.14, 3), (2, 'bonnie', 172342.2, 3), @@ -16,6 +18,7 @@ insert into cmv_basetable_n6 values create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true', 'transactional_properties'='insert_only'); insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (3, 'calvin', 978.76, 3); diff --git a/ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q b/ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q deleted file mode 100644 index b104c2b637c5..000000000000 --- a/ql/src/test/queries/clientpositive/materialized_view_join_rebuild.q +++ /dev/null @@ -1,33 +0,0 @@ --- Test Incremental rebuild of materialized view without aggregate when source tables have --- delete operations since last rebuild. --- The view projects only one column. - -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); - -insert into cmv_basetable_n6 values - (1, 'alfred', 10.30, 2), - (2, 'bob', 3.14, 3), - (2, 'bonnie', 172342.2, 3), - (3, 'calvin', 978.76, 3), - (3, 'charlie', 9.8, 1); - -create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); - -insert into cmv_basetable_2_n3 values - (1, 'alfred', 10.30, 2), - (3, 'calvin', 978.76, 3); - -CREATE MATERIALIZED VIEW cmv_mat_view_n6 - TBLPROPERTIES ('transactional'='true') AS - SELECT cmv_basetable_n6.a - FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) - WHERE cmv_basetable_2_n3.c > 10.0; - -DELETE from cmv_basetable_2_n3 WHERE a=1; - -ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD; - -SELECT * FROM cmv_mat_view_n6; diff --git a/ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q b/ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q deleted file mode 100644 index 8fec80b3d2d9..000000000000 --- a/ql/src/test/queries/clientpositive/materialized_view_repeated_rebuild.q +++ /dev/null @@ -1,37 +0,0 @@ --- Test Incremental rebuild of materialized view without aggregate when source tables have --- 1) delete operations since last rebuild. --- 2) delete records with the same join key from the other joined table - -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); - -insert into cmv_basetable_n6 values - (1, 'alfred', 10.30, 2), - (2, 'bob', 3.14, 3), - (2, 'bonnie', 172342.2, 3), - (3, 'calvin', 978.76, 3), - (3, 'charlie', 9.8, 1); - -create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true'); - -insert into cmv_basetable_2_n3 values - (1, 'alfred', 10.30, 2), - (3, 'calvin', 978.76, 3); - -CREATE MATERIALIZED VIEW cmv_mat_view_n6 - TBLPROPERTIES ('transactional'='true') AS - SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c - FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) - WHERE cmv_basetable_2_n3.c > 10.0; - -DELETE from cmv_basetable_2_n3 WHERE a=1; - -ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD; - -DELETE FROM cmv_basetable_n6 WHERE a=1; - -ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD; - -SELECT * FROM cmv_mat_view_n6; diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_12.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_12.q.out new file mode 100644 index 000000000000..81a33edecfa9 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_12.q.out @@ -0,0 +1,211 @@ +PREHOOK: query: create table t1 (a int, b int) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@t1 +POSTHOOK: query: create table t1 (a int, b int) stored as orc TBLPROPERTIES ('transactional'='true') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@t1 +PREHOOK: query: insert into t1 values +(3, 3), +(2, 1), +(2, 2), +(1, 2), +(1, 1) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@t1 +POSTHOOK: query: insert into t1 values +(3, 3), +(2, 1), +(2, 2), +(1, 2), +(1, 1) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t1.a SCRIPT [] +POSTHOOK: Lineage: t1.b SCRIPT [] +PREHOOK: query: CREATE MATERIALIZED VIEW mat1 + TBLPROPERTIES ('transactional'='true') AS +SELECT a +FROM t1 +WHERE b < 10 +PREHOOK: type: CREATE_MATERIALIZED_VIEW +PREHOOK: Input: default@t1 +PREHOOK: Output: database:default +PREHOOK: Output: default@mat1 +POSTHOOK: query: CREATE MATERIALIZED VIEW mat1 + TBLPROPERTIES ('transactional'='true') AS +SELECT a +FROM t1 +WHERE b < 10 +POSTHOOK: type: CREATE_MATERIALIZED_VIEW +POSTHOOK: Input: default@t1 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@mat1 +POSTHOOK: Lineage: mat1.a SIMPLE [(t1)t1.FieldSchema(name:a, type:int, comment:null), ] +PREHOOK: query: delete from t1 where b = 2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: delete from t1 where b = 2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +PREHOOK: query: explain +alter materialized view mat1 rebuild +PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +PREHOOK: Input: default@t1 +PREHOOK: Output: default@mat1 +POSTHOOK: query: explain +alter materialized view mat1 rebuild +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@mat1 +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-4 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: (b < 10) (type: boolean) + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (b < 10) (type: boolean) + Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.mat1 + Write Type: INSERT + Select Operator + expressions: _col0 (type: int) + outputColumnNames: a + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a) + minReductionHashAggr: 0.6666666 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'LONG' (type: string), UDFToLong(_col0) (type: bigint), UDFToLong(_col1) (type: bigint), (_col2 - _col3) (type: bigint), COALESCE(ndv_compute_bit_vector(_col4),0) (type: bigint), _col4 (type: binary) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 1 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: false + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.mat1 + Write Type: INSERT + + Stage: Stage-3 + Stats Work + Basic Stats Work: + Column Stats Desc: + Columns: a + Column Types: int + Table: default.mat1 + + Stage: Stage-4 + Materialized View Update + name: default.mat1 + update creation metadata: true + +PREHOOK: query: alter materialized view mat1 rebuild +PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +PREHOOK: Input: default@t1 +PREHOOK: Output: default@mat1 +POSTHOOK: query: alter materialized view mat1 rebuild +POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@mat1 +POSTHOOK: Lineage: mat1.a SIMPLE [(t1)t1.FieldSchema(name:a, type:int, comment:null), ] +PREHOOK: query: explain cbo +SELECT a +FROM t1 +WHERE b < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@mat1 +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: explain cbo +SELECT a +FROM t1 +WHERE b < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@mat1 +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +CBO PLAN: +HiveTableScan(table=[[default, mat1]], table:alias=[default.mat1]) + +PREHOOK: query: SELECT a +FROM t1 +WHERE b < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@mat1 +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT a +FROM t1 +WHERE b < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@mat1 +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +3 +2 +1 diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out index 6c6bf7c1b6ef..1b06d867b39c 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_5.q.out @@ -7,6 +7,8 @@ POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@cmv_basetable_n6 PREHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (2, 'bob', 3.14, 3), (2, 'bonnie', 172342.2, 3), @@ -16,6 +18,8 @@ PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@cmv_basetable_n6 POSTHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (2, 'bob', 3.14, 3), (2, 'bonnie', 172342.2, 3), @@ -37,12 +41,14 @@ POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@cmv_basetable_2_n3 PREHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (3, 'calvin', 978.76, 3) PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@cmv_basetable_2_n3 POSTHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (3, 'calvin', 978.76, 3) POSTHOOK: type: QUERY @@ -121,20 +127,20 @@ STAGE PLANS: TableScan alias: cmv_basetable_n6 filterExpr: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: may be used (ACID table) Map 3 @@ -142,20 +148,20 @@ STAGE PLANS: TableScan alias: cmv_basetable_2_n3 filterExpr: ((c > 10.1) and a is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((c > 10.1) and a is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: may be used (ACID table) Reducer 2 @@ -168,10 +174,10 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -198,6 +204,11 @@ POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 #### A masked pattern was here #### 1 +1 +1 +1 +1 +1 3 3 3 @@ -235,20 +246,20 @@ STAGE PLANS: TableScan alias: cmv_basetable_n6 filterExpr: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: may be used (ACID table) Map 4 @@ -256,7 +267,7 @@ STAGE PLANS: TableScan alias: cmv_basetable_2_n3 filterExpr: ((ROW__ID.writeid > 1L) and (c > 10) and a is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((ROW__ID.writeid > 1L) and (c > 10) and a is not null) (type: boolean) Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE @@ -391,9 +402,9 @@ Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\",\"c\":\"true\"}} bucketing_version 2 numFiles 2 - numRows 5 + numRows 10 rawDataSize 0 - totalSize 1516 + totalSize 1522 transactional true transactional_properties default #### A masked pattern was here #### @@ -474,6 +485,11 @@ POSTHOOK: Input: default@cmv_basetable_n6 POSTHOOK: Input: default@cmv_mat_view_n6 #### A masked pattern was here #### 1 +1 +1 +1 +1 +1 3 3 3 @@ -497,101 +513,70 @@ ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD PREHOOK: Input: default@cmv_basetable_2_n3 PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 PREHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: query: EXPLAIN ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Output: default@cmv_mat_view_n6 STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-3 depends on stages: Stage-2 - Stage-0 depends on stages: Stage-3 - Stage-4 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-4, Stage-5 - Stage-1 depends on stages: Stage-3 - Stage-5 depends on stages: Stage-1 + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-4 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-2 + Stage: Stage-1 Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: default.cmv_mat_view_n6 - Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: a (type: int), c (type: decimal(10,2)), true (type: boolean), ROW__ID (type: struct) - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) - Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: boolean), _col3 (type: struct) - Execution mode: llap - LLAP IO: may be used (ACID table) - Map 5 Map Operator Tree: TableScan alias: cmv_basetable_n6 filterExpr: a is not null (type: boolean) - properties: - acid.fetch.deleted.rows TRUE - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), ROW__IS__DELETED (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: may be used (ACID table) - Map 7 + Map 4 Map Operator Tree: TableScan alias: cmv_basetable_2_n3 - filterExpr: ((ROW__ID.writeid > 2L) and (c > 10) and a is not null) (type: boolean) - properties: - acid.fetch.deleted.rows TRUE - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + filterExpr: ((c > 10) and a is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((ROW__ID.writeid > 2L) and (c > 10) and a is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE + predicate: ((c > 10) and a is not null) (type: boolean) + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), c (type: decimal(10,2)), ROW__IS__DELETED (type: boolean) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int), c (type: decimal(10,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: decimal(10,2)), _col2 (type: boolean) + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(10,2)) Execution mode: llap LLAP IO: may be used (ACID table) Reducer 2 @@ -599,74 +584,41 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Right Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int), _col1 (type: decimal(10,2)) - 1 _col0 (type: int), _col1 (type: decimal(10,2)) - nullSafes: [true, true] - outputColumnNames: _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 5 Data size: 920 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (_col2 and _col6) (type: boolean) - Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col3 (type: struct) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: struct) - null sort order: z - sort order: + - Map-reduce partition columns: UDFToInteger(_col0) (type: int) - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (not _col6) (type: boolean) - Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col2 (type: decimal(10,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.cmv_mat_view_n6 + Write Type: INSERT Select Operator - expressions: _col4 (type: int), _col5 (type: decimal(10,2)) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: INSERT - Select Operator - expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - outputColumnNames: a, c - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + outputColumnNames: a, c + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: decimal(10,2)), _col6 (type: decimal(10,2)), _col7 (type: bigint), _col8 (type: binary) + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: decimal(10,2)), _col6 (type: decimal(10,2)), _col7 (type: bigint), _col8 (type: binary) Reducer 3 - Execution mode: llap - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: struct) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: DELETE - Reducer 4 Execution mode: llap Reduce Operator Tree: Group By Operator @@ -685,53 +637,11 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 6 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col3, _col4 - Statistics: Num rows: 2 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col3 (type: decimal(10,2)), (_col1 or _col4) (type: boolean) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 240 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) - Statistics: Num rows: 2 Data size: 240 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: boolean) - Stage: Stage-3 + Stage: Stage-2 Dependency Collection Stage: Stage-0 - Move Operator - tables: - replace: false - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: DELETE - - Stage: Stage-4 - Stats Work - Basic Stats Work: - - Stage: Stage-6 - Materialized View Update - name: default.cmv_mat_view_n6 - update creation metadata: true - - Stage: Stage-1 Move Operator tables: replace: false @@ -742,7 +652,7 @@ STAGE PLANS: name: default.cmv_mat_view_n6 Write Type: INSERT - Stage: Stage-5 + Stage: Stage-3 Stats Work Basic Stats Work: Column Stats Desc: @@ -750,19 +660,20 @@ STAGE PLANS: Column Types: int, decimal(10,2) Table: default.cmv_mat_view_n6 + Stage: Stage-4 + Materialized View Update + name: default.cmv_mat_view_n6 + update creation metadata: true + PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD PREHOOK: Input: default@cmv_basetable_2_n3 PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 PREHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] @@ -818,12 +729,14 @@ POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 POSTHOOK: Input: default@cmv_mat_view_n6 #### A masked pattern was here #### +2 +2 +2 +2 3 3 3 3 -2 -2 PREHOOK: query: DELETE FROM cmv_basetable_2_n3 WHERE a=2 PREHOOK: type: QUERY PREHOOK: Input: default@cmv_basetable_2_n3 @@ -837,101 +750,70 @@ ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD PREHOOK: Input: default@cmv_basetable_2_n3 PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 PREHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: query: EXPLAIN ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Output: default@cmv_mat_view_n6 STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-3 depends on stages: Stage-2 - Stage-0 depends on stages: Stage-3 - Stage-4 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-4, Stage-5 - Stage-1 depends on stages: Stage-3 - Stage-5 depends on stages: Stage-1 + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-4 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-2 + Stage: Stage-1 Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: default.cmv_mat_view_n6 - Statistics: Num rows: 6 Data size: 696 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: a (type: int), c (type: decimal(10,2)), true (type: boolean), ROW__ID (type: struct) - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 6 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) - Statistics: Num rows: 6 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: boolean), _col3 (type: struct) - Execution mode: llap - LLAP IO: may be used (ACID table) - Map 5 Map Operator Tree: TableScan alias: cmv_basetable_n6 filterExpr: a is not null (type: boolean) - properties: - acid.fetch.deleted.rows TRUE - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), ROW__IS__DELETED (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: may be used (ACID table) - Map 7 + Map 4 Map Operator Tree: TableScan alias: cmv_basetable_2_n3 - filterExpr: ((ROW__ID.writeid > 3L) and (c > 10) and a is not null) (type: boolean) - properties: - acid.fetch.deleted.rows TRUE + filterExpr: ((c > 10) and a is not null) (type: boolean) Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((ROW__ID.writeid > 3L) and (c > 10) and a is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE + predicate: ((c > 10) and a is not null) (type: boolean) + Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), c (type: decimal(10,2)), ROW__IS__DELETED (type: boolean) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int), c (type: decimal(10,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: decimal(10,2)), _col2 (type: boolean) + Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(10,2)) Execution mode: llap LLAP IO: may be used (ACID table) Reducer 2 @@ -939,74 +821,41 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Right Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int), _col1 (type: decimal(10,2)) - 1 _col0 (type: int), _col1 (type: decimal(10,2)) - nullSafes: [true, true] - outputColumnNames: _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 1640 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (_col2 and _col6) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col3 (type: struct) - outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: struct) - null sort order: z - sort order: + - Map-reduce partition columns: UDFToInteger(_col0) (type: int) - Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (not _col6) (type: boolean) - Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 7 Data size: 812 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col2 (type: decimal(10,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 812 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 812 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.cmv_mat_view_n6 + Write Type: INSERT Select Operator - expressions: _col4 (type: int), _col5 (type: decimal(10,2)) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: INSERT - Select Operator - expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - outputColumnNames: a, c - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + outputColumnNames: a, c + Statistics: Num rows: 7 Data size: 812 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) + minReductionHashAggr: 0.85714287 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: decimal(10,2)), _col6 (type: decimal(10,2)), _col7 (type: bigint), _col8 (type: binary) + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: decimal(10,2)), _col6 (type: decimal(10,2)), _col7 (type: bigint), _col8 (type: binary) Reducer 3 - Execution mode: llap - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: struct) - outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 152 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: DELETE - Reducer 4 Execution mode: llap Reduce Operator Tree: Group By Operator @@ -1025,53 +874,11 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 6 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col3, _col4 - Statistics: Num rows: 3 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col3 (type: decimal(10,2)), (_col1 or _col4) (type: boolean) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 3 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) - Statistics: Num rows: 3 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: boolean) - Stage: Stage-3 + Stage: Stage-2 Dependency Collection Stage: Stage-0 - Move Operator - tables: - replace: false - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: DELETE - - Stage: Stage-4 - Stats Work - Basic Stats Work: - - Stage: Stage-6 - Materialized View Update - name: default.cmv_mat_view_n6 - update creation metadata: true - - Stage: Stage-1 Move Operator tables: replace: false @@ -1082,7 +889,7 @@ STAGE PLANS: name: default.cmv_mat_view_n6 Write Type: INSERT - Stage: Stage-5 + Stage: Stage-3 Stats Work Basic Stats Work: Column Stats Desc: @@ -1090,19 +897,20 @@ STAGE PLANS: Column Types: int, decimal(10,2) Table: default.cmv_mat_view_n6 + Stage: Stage-4 + Materialized View Update + name: default.cmv_mat_view_n6 + update creation metadata: true + PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD PREHOOK: Input: default@cmv_basetable_2_n3 PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 PREHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] @@ -1209,20 +1017,20 @@ STAGE PLANS: TableScan alias: cmv_basetable_n6 filterExpr: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: llap LLAP IO: may be used (ACID table) Map 4 @@ -1257,14 +1065,14 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col2 - Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: int), _col2 (type: decimal(10,2)) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat @@ -1274,10 +1082,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: decimal(10,2)) outputColumnNames: a, c - Statistics: Num rows: 2 Data size: 232 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.6666666 mode: hash outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE @@ -1402,6 +1210,8 @@ POSTHOOK: Input: default@cmv_mat_view_n6 3 3 1 +1 +1 PREHOOK: query: drop materialized view cmv_mat_view_n6 PREHOOK: type: DROP_MATERIALIZED_VIEW PREHOOK: Input: default@cmv_mat_view_n6 diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out index 7aee7df9de4d..3f7007a1501f 100644 --- a/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out +++ b/ql/src/test/results/clientpositive/llap/materialized_view_create_rewrite_8.q.out @@ -7,6 +7,8 @@ POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@cmv_basetable_n6 PREHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (2, 'bob', 3.14, 3), (2, 'bonnie', 172342.2, 3), @@ -16,6 +18,8 @@ PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@cmv_basetable_n6 POSTHOOK: query: insert into cmv_basetable_n6 values + (1, 'alfred', 10.30, 2), + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (2, 'bob', 3.14, 3), (2, 'bonnie', 172342.2, 3), @@ -37,12 +41,14 @@ POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@cmv_basetable_2_n3 PREHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (3, 'calvin', 978.76, 3) PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table PREHOOK: Output: default@cmv_basetable_2_n3 POSTHOOK: query: insert into cmv_basetable_2_n3 values + (1, 'alfred', 10.30, 2), (1, 'alfred', 10.30, 2), (3, 'calvin', 978.76, 3) POSTHOOK: type: QUERY @@ -129,6 +135,11 @@ POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 #### A masked pattern was here #### 1 +1 +1 +1 +1 +1 3 3 3 @@ -166,20 +177,20 @@ STAGE PLANS: TableScan alias: cmv_basetable_n6 filterExpr: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: may be used (ACID table) Map 4 @@ -189,7 +200,7 @@ STAGE PLANS: filterExpr: ((ROW__ID.writeid > 1L) and (c > 10) and a is not null) (type: boolean) properties: insertonly.fetch.bucketid TRUE - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((ROW__ID.writeid > 1L) and (c > 10) and a is not null) (type: boolean) Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE @@ -344,6 +355,11 @@ POSTHOOK: Input: default@cmv_basetable_n6 POSTHOOK: Input: default@cmv_mat_view_n6 #### A masked pattern was here #### 1 +1 +1 +1 +1 +1 3 3 3 @@ -367,101 +383,70 @@ ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD PREHOOK: Input: default@cmv_basetable_2_n3 PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 PREHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: query: EXPLAIN ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Output: default@cmv_mat_view_n6 STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-3 depends on stages: Stage-2 - Stage-0 depends on stages: Stage-3 - Stage-4 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-4, Stage-5 - Stage-1 depends on stages: Stage-3 - Stage-5 depends on stages: Stage-1 + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-4 depends on stages: Stage-3 STAGE PLANS: - Stage: Stage-2 + Stage: Stage-1 Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) - Reducer 6 <- Map 5 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 - Map Operator Tree: - TableScan - alias: default.cmv_mat_view_n6 - Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: a (type: int), c (type: decimal(10,2)), true (type: boolean), ROW__ID (type: struct) - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) - Statistics: Num rows: 5 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: boolean), _col3 (type: struct) - Execution mode: vectorized, llap - LLAP IO: may be used (ACID table) - Map 5 Map Operator Tree: TableScan alias: cmv_basetable_n6 - filterExpr: ((ROW__ID.writeid > 1L) and a is not null) (type: boolean) - properties: - acid.fetch.deleted.rows TRUE - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + filterExpr: a is not null (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((ROW__ID.writeid > 1L) and a is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + predicate: a is not null (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), ROW__IS__DELETED (type: boolean) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: boolean) + Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE Execution mode: vectorized, llap LLAP IO: may be used (ACID table) - Map 7 + Map 4 Map Operator Tree: TableScan alias: cmv_basetable_2_n3 filterExpr: ((c > 10) and a is not null) (type: boolean) - properties: - acid.fetch.deleted.rows TRUE - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((c > 10) and a is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: a (type: int), c (type: decimal(10,2)), ROW__IS__DELETED (type: boolean) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 3 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE + expressions: a (type: int), c (type: decimal(10,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col1 (type: decimal(10,2)), _col2 (type: boolean) + Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(10,2)) Execution mode: vectorized, llap LLAP IO: may be used (ACID table) Reducer 2 @@ -469,74 +454,41 @@ STAGE PLANS: Reduce Operator Tree: Merge Join Operator condition map: - Right Outer Join 0 to 1 + Inner Join 0 to 1 keys: - 0 _col0 (type: int), _col1 (type: decimal(10,2)) - 1 _col0 (type: int), _col1 (type: decimal(10,2)) - nullSafes: [true, true] - outputColumnNames: _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (_col2 and _col6) (type: boolean) - Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col3 (type: struct) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: struct) - null sort order: z - sort order: + - Map-reduce partition columns: UDFToInteger(_col0) (type: int) - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: (not _col6) (type: boolean) - Statistics: Num rows: 1 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col2 + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col2 (type: decimal(10,2)) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.cmv_mat_view_n6 + Write Type: INSERT Select Operator - expressions: _col4 (type: int), _col5 (type: decimal(10,2)) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: INSERT - Select Operator - expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - outputColumnNames: a, c - Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE - Group By Operator - aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) - minReductionHashAggr: 0.4 - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + expressions: _col0 (type: int), _col1 (type: decimal(10,2)) + outputColumnNames: a, c + Statistics: Num rows: 9 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(c), max(c), count(c), compute_bit_vector_hll(c) + minReductionHashAggr: 0.8888889 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 + Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - null sort order: - sort order: - Statistics: Num rows: 1 Data size: 544 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: decimal(10,2)), _col6 (type: decimal(10,2)), _col7 (type: bigint), _col8 (type: binary) + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: decimal(10,2)), _col6 (type: decimal(10,2)), _col7 (type: bigint), _col8 (type: binary) Reducer 3 - Execution mode: vectorized, llap - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: struct) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 76 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: DELETE - Reducer 4 Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator @@ -555,53 +507,11 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Reducer 6 - Execution mode: llap - Reduce Operator Tree: - Merge Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: int) - 1 _col0 (type: int) - outputColumnNames: _col0, _col1, _col3, _col4 - Statistics: Num rows: 1 Data size: 124 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), _col3 (type: decimal(10,2)), (_col1 or _col4) (type: boolean) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: decimal(10,2)) - null sort order: zz - sort order: ++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: decimal(10,2)) - Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col2 (type: boolean) - Stage: Stage-3 + Stage: Stage-2 Dependency Collection Stage: Stage-0 - Move Operator - tables: - replace: false - table: - input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat - output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat - serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde - name: default.cmv_mat_view_n6 - Write Type: DELETE - - Stage: Stage-4 - Stats Work - Basic Stats Work: - - Stage: Stage-6 - Materialized View Update - name: default.cmv_mat_view_n6 - update creation metadata: true - - Stage: Stage-1 Move Operator tables: replace: false @@ -612,7 +522,7 @@ STAGE PLANS: name: default.cmv_mat_view_n6 Write Type: INSERT - Stage: Stage-5 + Stage: Stage-3 Stats Work Basic Stats Work: Column Stats Desc: @@ -620,19 +530,20 @@ STAGE PLANS: Column Types: int, decimal(10,2) Table: default.cmv_mat_view_n6 + Stage: Stage-4 + Materialized View Update + name: default.cmv_mat_view_n6 + update creation metadata: true + PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD PREHOOK: Input: default@cmv_basetable_2_n3 PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 PREHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD POSTHOOK: Input: default@cmv_basetable_2_n3 POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Output: default@cmv_mat_view_n6 POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out deleted file mode 100644 index 9f66625aaa2f..000000000000 --- a/ql/src/test/results/clientpositive/llap/materialized_view_join_rebuild.q.out +++ /dev/null @@ -1,108 +0,0 @@ -PREHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@cmv_basetable_n6 -PREHOOK: query: insert into cmv_basetable_n6 values - (1, 'alfred', 10.30, 2), - (2, 'bob', 3.14, 3), - (2, 'bonnie', 172342.2, 3), - (3, 'calvin', 978.76, 3), - (3, 'charlie', 9.8, 1) -PREHOOK: type: QUERY -PREHOOK: Input: _dummy_database@_dummy_table -PREHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: query: insert into cmv_basetable_n6 values - (1, 'alfred', 10.30, 2), - (2, 'bob', 3.14, 3), - (2, 'bonnie', 172342.2, 3), - (3, 'calvin', 978.76, 3), - (3, 'charlie', 9.8, 1) -POSTHOOK: type: QUERY -POSTHOOK: Input: _dummy_database@_dummy_table -POSTHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: Lineage: cmv_basetable_n6.a SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_n6.b SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_n6.c SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_n6.d SCRIPT [] -PREHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@cmv_basetable_2_n3 -PREHOOK: query: insert into cmv_basetable_2_n3 values - (1, 'alfred', 10.30, 2), - (3, 'calvin', 978.76, 3) -PREHOOK: type: QUERY -PREHOOK: Input: _dummy_database@_dummy_table -PREHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: query: insert into cmv_basetable_2_n3 values - (1, 'alfred', 10.30, 2), - (3, 'calvin', 978.76, 3) -POSTHOOK: type: QUERY -POSTHOOK: Input: _dummy_database@_dummy_table -POSTHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: Lineage: cmv_basetable_2_n3.a SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_2_n3.b SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_2_n3.c SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_2_n3.d SCRIPT [] -PREHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 - TBLPROPERTIES ('transactional'='true') AS - SELECT cmv_basetable_n6.a - FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) - WHERE cmv_basetable_2_n3.c > 10.0 -PREHOOK: type: CREATE_MATERIALIZED_VIEW -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Output: database:default -PREHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 - TBLPROPERTIES ('transactional'='true') AS - SELECT cmv_basetable_n6.a - FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) - WHERE cmv_basetable_2_n3.c > 10.0 -POSTHOOK: type: CREATE_MATERIALIZED_VIEW -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Output: database:default -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] -PREHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 -PREHOOK: type: QUERY -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Output: default@cmv_basetable_2_n3 -PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD -PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD -POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] -PREHOOK: query: SELECT * FROM cmv_mat_view_n6 -PREHOOK: type: QUERY -PREHOOK: Input: default@cmv_mat_view_n6 -#### A masked pattern was here #### -POSTHOOK: query: SELECT * FROM cmv_mat_view_n6 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@cmv_mat_view_n6 -#### A masked pattern was here #### -3 -3 diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out deleted file mode 100644 index 9739f87e96e3..000000000000 --- a/ql/src/test/results/clientpositive/llap/materialized_view_repeated_rebuild.q.out +++ /dev/null @@ -1,134 +0,0 @@ -PREHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: query: create table cmv_basetable_n6 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@cmv_basetable_n6 -PREHOOK: query: insert into cmv_basetable_n6 values - (1, 'alfred', 10.30, 2), - (2, 'bob', 3.14, 3), - (2, 'bonnie', 172342.2, 3), - (3, 'calvin', 978.76, 3), - (3, 'charlie', 9.8, 1) -PREHOOK: type: QUERY -PREHOOK: Input: _dummy_database@_dummy_table -PREHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: query: insert into cmv_basetable_n6 values - (1, 'alfred', 10.30, 2), - (2, 'bob', 3.14, 3), - (2, 'bonnie', 172342.2, 3), - (3, 'calvin', 978.76, 3), - (3, 'charlie', 9.8, 1) -POSTHOOK: type: QUERY -POSTHOOK: Input: _dummy_database@_dummy_table -POSTHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: Lineage: cmv_basetable_n6.a SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_n6.b SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_n6.c SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_n6.d SCRIPT [] -PREHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: query: create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@cmv_basetable_2_n3 -PREHOOK: query: insert into cmv_basetable_2_n3 values - (1, 'alfred', 10.30, 2), - (3, 'calvin', 978.76, 3) -PREHOOK: type: QUERY -PREHOOK: Input: _dummy_database@_dummy_table -PREHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: query: insert into cmv_basetable_2_n3 values - (1, 'alfred', 10.30, 2), - (3, 'calvin', 978.76, 3) -POSTHOOK: type: QUERY -POSTHOOK: Input: _dummy_database@_dummy_table -POSTHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: Lineage: cmv_basetable_2_n3.a SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_2_n3.b SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_2_n3.c SCRIPT [] -POSTHOOK: Lineage: cmv_basetable_2_n3.d SCRIPT [] -PREHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 - TBLPROPERTIES ('transactional'='true') AS - SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c - FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) - WHERE cmv_basetable_2_n3.c > 10.0 -PREHOOK: type: CREATE_MATERIALIZED_VIEW -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Output: database:default -PREHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: query: CREATE MATERIALIZED VIEW cmv_mat_view_n6 - TBLPROPERTIES ('transactional'='true') AS - SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c - FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a) - WHERE cmv_basetable_2_n3.c > 10.0 -POSTHOOK: type: CREATE_MATERIALIZED_VIEW -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Output: database:default -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] -POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] -PREHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 -PREHOOK: type: QUERY -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Output: default@cmv_basetable_2_n3 -POSTHOOK: query: DELETE from cmv_basetable_2_n3 WHERE a=1 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Output: default@cmv_basetable_2_n3 -PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD -PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD -POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] -POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] -PREHOOK: query: DELETE FROM cmv_basetable_n6 WHERE a=1 -PREHOOK: type: QUERY -PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Output: default@cmv_basetable_n6 -POSTHOOK: query: DELETE FROM cmv_basetable_n6 WHERE a=1 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Output: default@cmv_basetable_n6 -PREHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD -PREHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD -PREHOOK: Input: default@cmv_basetable_2_n3 -PREHOOK: Input: default@cmv_basetable_n6 -PREHOOK: Input: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 -PREHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: query: ALTER MATERIALIZED VIEW cmv_mat_view_n6 REBUILD -POSTHOOK: type: ALTER_MATERIALIZED_VIEW_REBUILD -POSTHOOK: Input: default@cmv_basetable_2_n3 -POSTHOOK: Input: default@cmv_basetable_n6 -POSTHOOK: Input: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Output: default@cmv_mat_view_n6 -POSTHOOK: Lineage: cmv_mat_view_n6.a SIMPLE [(cmv_basetable_n6)cmv_basetable_n6.FieldSchema(name:a, type:int, comment:null), ] -POSTHOOK: Lineage: cmv_mat_view_n6.c SIMPLE [(cmv_basetable_2_n3)cmv_basetable_2_n3.FieldSchema(name:c, type:decimal(10,2), comment:null), ] -PREHOOK: query: SELECT * FROM cmv_mat_view_n6 -PREHOOK: type: QUERY -PREHOOK: Input: default@cmv_mat_view_n6 -#### A masked pattern was here #### -POSTHOOK: query: SELECT * FROM cmv_mat_view_n6 -POSTHOOK: type: QUERY -POSTHOOK: Input: default@cmv_mat_view_n6 -#### A masked pattern was here #### -3 978.76 -3 978.76 From dabf39fb72e6f4e826d97084f37b28a004e2538e Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 12 Feb 2024 09:51:11 +0100 Subject: [PATCH 174/179] HIVE-26445: Use tez.local.mode.without.network for qtests (#5078) (Laszlo Bodor reviewed by Ayush Saxena) --- .../test/java/org/apache/iceberg/mr/hive/TestHiveShell.java | 3 +-- .../main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java index 8d2a9a294c7c..29941f809d09 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java @@ -209,8 +209,7 @@ private HiveConf initializeConf() { // Tez configuration hiveConf.setBoolean("tez.local.mode", true); - // TODO: enable below option once HIVE-26445 is investigated - // hiveConf.setBoolean("tez.local.mode.without.network", true); + hiveConf.setBoolean("tez.local.mode.without.network", true); // Disable vectorization for HiveIcebergInputFormat hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false); diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java index 0437417e4b47..d4d6235d8324 100644 --- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java +++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java @@ -367,8 +367,7 @@ public int getJobTrackerPort() throws UnsupportedOperationException { @Override public void setupConfiguration(Configuration conf) { conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - // TODO: enable below option once HIVE-26445 is investigated - // hiveConf.setBoolean("tez.local.mode.without.network", true); + conf.setBoolean("tez.local.mode.without.network", true); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true); conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); From e4f60bf8ac197224062751141211df72d79827a9 Mon Sep 17 00:00:00 2001 From: tarak271 Date: Mon, 12 Feb 2024 22:00:47 +0530 Subject: [PATCH 175/179] HIVE-27022: Extract compaction-related functionality from AcidHouseKeeper into a separate service (Taraka Rama Rao Lethavadla, reviewed by Denys Kuzmenko, Zsolt Miskolczi) Closes #4970 --- .../TestTimedOutTxnNotificationLogging.java | 4 +- .../hadoop/hive/ql/TestTxnCommands.java | 2 +- .../hadoop/hive/ql/TestTxnCommands2.java | 15 +++++- .../hive/ql/lockmgr/TestDbTxnManager.java | 2 +- .../hive/ql/lockmgr/TestDbTxnManager2.java | 7 ++- .../hive/metastore/conf/MetastoreConf.java | 17 +++++-- .../metastore/leader/HouseKeepingTasks.java | 7 +++ .../hadoop/hive/metastore/txn/TxnHandler.java | 1 + .../{ => service}/AcidHouseKeeperService.java | 50 ++++++++++-------- .../AcidOpenTxnsCounterService.java | 8 +-- .../{ => service}/AcidTxnCleanerService.java | 8 +-- .../service/CompactionHouseKeeperService.java | 51 +++++++++++++++++++ .../metastore/conf/TestMetastoreConf.java | 11 ++-- .../txn/TestAcidTxnCleanerService.java | 1 + .../apache/hive/streaming/TestStreaming.java | 2 +- 15 files changed, 143 insertions(+), 43 deletions(-) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => service}/AcidHouseKeeperService.java (65%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => service}/AcidOpenTxnsCounterService.java (88%) rename standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/{ => service}/AcidTxnCleanerService.java (88%) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/CompactionHouseKeeperService.java diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java index 559699cf3c36..130e4908b3c2 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestTimedOutTxnNotificationLogging.java @@ -34,8 +34,8 @@ import org.apache.hadoop.hive.metastore.messaging.event.filters.CatalogFilter; import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter; import org.apache.hadoop.hive.metastore.messaging.event.filters.ReplEventFilter; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; -import org.apache.hadoop.hive.metastore.txn.AcidTxnCleanerService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidTxnCleanerService; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java index 46c8e8244564..39e09a8eb178 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java @@ -66,7 +66,7 @@ import org.apache.hadoop.hive.metastore.api.TxnInfo; import org.apache.hadoop.hive.metastore.api.TxnState; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java index 9b2edfa10f57..3f574e384ede 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java @@ -63,7 +63,8 @@ import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest; import org.apache.hadoop.hive.metastore.api.AbortCompactionResponseElement; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.CompactionHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.ql.ddl.DDLTask; @@ -74,7 +75,7 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory; import org.apache.hadoop.hive.ql.processors.CommandProcessorException; -import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService; +import org.apache.hadoop.hive.metastore.txn.service.AcidOpenTxnsCounterService; import org.apache.hadoop.hive.ql.scheduled.ScheduledQueryExecutionContext; import org.apache.hadoop.hive.ql.scheduled.ScheduledQueryExecutionService; import org.apache.hadoop.hive.ql.schq.MockScheduledQueryService; @@ -1244,9 +1245,14 @@ void testInitiatorWithMultipleFailedCompactionsForVariousTblProperties(String tb MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.ACID_HOUSEKEEPER_SERVICE_INTERVAL, 10, TimeUnit.MILLISECONDS); + MetastoreConf.setTimeVar(hiveConf, MetastoreConf.ConfVars.COMPACTION_HOUSEKEEPER_SERVICE_INTERVAL, 10, + TimeUnit.MILLISECONDS); MetastoreTaskThread houseKeeper = new AcidHouseKeeperService(); + MetastoreTaskThread compactionHouseKeeper = new CompactionHouseKeeperService(); houseKeeper.setConf(hiveConf); + compactionHouseKeeper.setConf(hiveConf); houseKeeper.run(); + compactionHouseKeeper.run(); checkCompactionState(new CompactionsByState(numDidNotInitiateCompactions,numFailedCompactions,0,0,0,0,numFailedCompactions + numDidNotInitiateCompactions), countCompacts(txnHandler)); txnHandler.compact(new CompactionRequest("default", tblName, CompactionType.MAJOR)); @@ -1260,6 +1266,7 @@ void testInitiatorWithMultipleFailedCompactionsForVariousTblProperties(String tb checkCompactionState(new CompactionsByState(numDidNotInitiateCompactions,numFailedCompactions + 2,0,0,0,0,numFailedCompactions + 2 + numDidNotInitiateCompactions), countCompacts(txnHandler)); houseKeeper.run(); + compactionHouseKeeper.run(); //COMPACTOR_HISTORY_RETENTION_FAILED failed compacts left (and no other since we only have failed ones here) checkCompactionState(new CompactionsByState( MetastoreConf.getIntVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_DID_NOT_INITIATE), @@ -1287,6 +1294,7 @@ void testInitiatorWithMultipleFailedCompactionsForVariousTblProperties(String tb runCleaner(hiveConf); // transition to Success state houseKeeper.run(); + compactionHouseKeeper.run(); checkCompactionState(new CompactionsByState( MetastoreConf.getIntVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_DID_NOT_INITIATE), MetastoreConf.getIntVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED), 0, 0, 1, 0, @@ -3346,8 +3354,11 @@ public void testNoTxnComponentsForScheduledQueries() throws Exception { // Run AcidHouseKeeperService to cleanup the COMPLETED_TXN_COMPONENTS. MetastoreTaskThread houseKeeper = new AcidHouseKeeperService(); + MetastoreTaskThread compactionHouseKeeper = new CompactionHouseKeeperService(); houseKeeper.setConf(hiveConf); + compactionHouseKeeper.setConf(hiveConf); houseKeeper.run(); + compactionHouseKeeper.run(); // Check whether the table is compacted. fileStatuses = fs.globStatus(new Path(getWarehouseDir() + "/" + tableName + "/*")); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java index c2d93e0f95ed..b6f533490ffd 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.metrics.Metrics; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java index f7a02cc76746..250aa7382046 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java @@ -38,7 +38,8 @@ import org.apache.hadoop.hive.metastore.api.TxnType; import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.CompactionHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.io.AcidUtils; @@ -3509,8 +3510,11 @@ public void testRemoveDuplicateCompletedTxnComponents() throws Exception { 5, TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"COMPLETED_TXN_COMPONENTS\"")); MetastoreTaskThread houseKeeper = new AcidHouseKeeperService(); + MetastoreTaskThread compactionHouseKeeper = new CompactionHouseKeeperService(); houseKeeper.setConf(conf); + compactionHouseKeeper.setConf(conf); houseKeeper.run(); + compactionHouseKeeper.run(); Assert.assertEquals(TestTxnDbUtil.queryToString(conf, "select * from \"COMPLETED_TXN_COMPONENTS\""), 2, TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"COMPLETED_TXN_COMPONENTS\"")); @@ -3527,6 +3531,7 @@ public void testRemoveDuplicateCompletedTxnComponents() throws Exception { 4, TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"COMPLETED_TXN_COMPONENTS\"")); houseKeeper.run(); + compactionHouseKeeper.run(); Assert.assertEquals(TestTxnDbUtil.queryToString(conf, "select * from \"COMPLETED_TXN_COMPONENTS\""), 3, TestTxnDbUtil.countQueryAgent(conf, "select count(*) from \"COMPLETED_TXN_COMPONENTS\"")); diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index 792e27fc99cd..585b55c39f1d 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -94,14 +94,17 @@ public class MetastoreConf { static final String METASTORE_DELEGATION_MANAGER_CLASS = "org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager"; @VisibleForTesting - static final String ACID_HOUSE_KEEPER_SERVICE_CLASS = - "org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService"; + static final String ACID_HOUSEKEEPER_SERVICE_CLASS = + "org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService"; + @VisibleForTesting + static final String COMPACTION_HOUSEKEEPER_SERVICE_CLASS = + "org.apache.hadoop.hive.metastore.txn.service.CompactionHouseKeeperService"; @VisibleForTesting static final String ACID_TXN_CLEANER_SERVICE_CLASS = - "org.apache.hadoop.hive.metastore.txn.AcidTxnCleanerService"; + "org.apache.hadoop.hive.metastore.txn.service.AcidTxnCleanerService"; @VisibleForTesting static final String ACID_OPEN_TXNS_COUNTER_SERVICE_CLASS = - "org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService"; + "org.apache.hadoop.hive.metastore.txn.service.AcidOpenTxnsCounterService"; public static final String METASTORE_AUTHENTICATION_LDAP_USERMEMBERSHIPKEY_NAME = "metastore.authentication.ldap.userMembershipKey"; @@ -293,6 +296,9 @@ public enum ConfVars { ACID_HOUSEKEEPER_SERVICE_INTERVAL("metastore.acid.housekeeper.interval", "hive.metastore.acid.housekeeper.interval", 60, TimeUnit.SECONDS, "Time interval describing how often the acid housekeeper runs."), + COMPACTION_HOUSEKEEPER_SERVICE_INTERVAL("metastore.compaction.housekeeper.interval", + "hive.metastore.compaction.housekeeper.interval", 300, TimeUnit.SECONDS, + "Time interval describing how often the acid compaction housekeeper runs."), ACID_TXN_CLEANER_INTERVAL("metastore.acid.txn.cleaner.interval", "hive.metastore.acid.txn.cleaner.interval", 10, TimeUnit.SECONDS, "Time interval describing how often aborted and committed txns are cleaned."), @@ -1467,7 +1473,8 @@ public enum ConfVars { "always be started, regardless of whether the metastore is running in embedded mode " + "or in server mode. They must implement " + METASTORE_TASK_THREAD_CLASS), TASK_THREADS_REMOTE_ONLY("metastore.task.threads.remote", "metastore.task.threads.remote", - ACID_HOUSE_KEEPER_SERVICE_CLASS + "," + + ACID_HOUSEKEEPER_SERVICE_CLASS + "," + + COMPACTION_HOUSEKEEPER_SERVICE_CLASS + "," + ACID_TXN_CLEANER_SERVICE_CLASS + "," + ACID_OPEN_TXNS_COUNTER_SERVICE_CLASS + "," + MATERIALZIATIONS_REBUILD_LOCK_CLEANER_TASK_CLASS + "," + diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/HouseKeepingTasks.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/HouseKeepingTasks.java index 5db4bba3f4e5..3a4414fd0045 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/HouseKeepingTasks.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/leader/HouseKeepingTasks.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.MetastoreTaskThread; import org.apache.hadoop.hive.metastore.ThreadPool; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.service.CompactionHouseKeeperService; import org.apache.hadoop.hive.metastore.utils.JavaUtils; import java.util.ArrayList; @@ -58,9 +59,15 @@ public List getRemoteOnlyTasks() throws Exception { MetastoreConf.ConfVars.METASTORE_HOUSEKEEPING_THREADS_ON)) { return remoteOnlyTasks; } + boolean isCompactorEnabled = MetastoreConf.getBoolVar(configuration, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON) + || MetastoreConf.getBoolVar(configuration, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON); + Collection taskNames = MetastoreConf.getStringCollection(configuration, MetastoreConf.ConfVars.TASK_THREADS_REMOTE_ONLY); for (String taskName : taskNames) { + if (CompactionHouseKeeperService.class.getName().equals(taskName) && !isCompactorEnabled) { + continue; + } MetastoreTaskThread task = JavaUtils.newInstance(JavaUtils.getClass(taskName, MetastoreTaskThread.class)); remoteOnlyTasks.add(task); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java index 9be5e475b6ed..92437bdbba03 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -99,6 +99,7 @@ import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryCallProperties; import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryException; import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; import org.apache.hadoop.hive.metastore.utils.JavaUtils; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; import org.apache.hadoop.util.StringUtils; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidHouseKeeperService.java similarity index 65% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidHouseKeeperService.java index 0a16006c49eb..86799e90621d 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidHouseKeeperService.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidHouseKeeperService.java @@ -15,19 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.service; + +import com.google.common.collect.ImmutableMap; -import org.apache.commons.lang3.Functions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.MetastoreTaskThread; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.apache.commons.lang3.Functions.FailableRunnable; +import org.apache.commons.lang3.function.FailableRunnable; +import org.apache.commons.lang3.function.Failable; /** * Performs background tasks for Transaction management in Hive. @@ -38,16 +43,27 @@ public class AcidHouseKeeperService implements MetastoreTaskThread { private static final Logger LOG = LoggerFactory.getLogger(AcidHouseKeeperService.class); private Configuration conf; - private boolean isCompactorEnabled; - private TxnStore txnHandler; + protected TxnStore txnHandler; + protected String serviceName; + protected Map, String> tasks; + + public AcidHouseKeeperService() { + serviceName = this.getClass().getSimpleName(); + } @Override public void setConf(Configuration configuration) { conf = configuration; - isCompactorEnabled = - MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON) || MetastoreConf.getBoolVar(conf, - MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON); txnHandler = TxnUtils.getTxnStore(conf); + initTasks(); + } + + protected void initTasks(){ + tasks = ImmutableMap., String>builder() + .put(txnHandler::performTimeOuts, "Cleaning timed out txns and locks") + .put(txnHandler::performWriteSetGC, "Cleaning obsolete write set entries") + .put(txnHandler::cleanTxnToWriteIdTable, "Cleaning obsolete TXN_TO_WRITE_ID entries") + .build(); } @Override @@ -65,12 +81,12 @@ public void run() { TxnStore.MutexAPI.LockHandle handle = null; try { handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.HouseKeeper.name()); - LOG.info("Starting to run AcidHouseKeeperService."); + LOG.info("Starting to run {}", serviceName); long start = System.currentTimeMillis(); cleanTheHouse(); - LOG.debug("Total time AcidHouseKeeperService took: {} seconds.", elapsedSince(start)); - } catch (Throwable t) { - LOG.error("Unexpected error in thread: {}, message: {}", Thread.currentThread().getName(), t.getMessage(), t); + LOG.debug("Total time {} took: {} seconds.", serviceName, elapsedSince(start)); + } catch (Exception e) { + LOG.error("Unexpected exception in thread: {}, message: {}", Thread.currentThread().getName(), e.getMessage(), e); } finally { if (handle != null) { handle.releaseLocks(); @@ -79,18 +95,12 @@ public void run() { } private void cleanTheHouse() { - performTask(txnHandler::performTimeOuts, "Cleaning timed out txns and locks"); - performTask(txnHandler::performWriteSetGC, "Cleaning obsolete write set entries"); - performTask(txnHandler::cleanTxnToWriteIdTable, "Cleaning obsolete TXN_TO_WRITE_ID entries"); - if (isCompactorEnabled) { - performTask(txnHandler::removeDuplicateCompletedTxnComponents, "Cleaning duplicate COMPLETED_TXN_COMPONENTS entries"); - performTask(txnHandler::purgeCompactionHistory, "Cleaning obsolete compaction history entries"); - } + tasks.forEach(this::performTask); } private void performTask(FailableRunnable task, String description) { long start = System.currentTimeMillis(); - Functions.run(task); + Failable.run(task); LOG.debug("{} took {} seconds.", description, elapsedSince(start)); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidOpenTxnsCounterService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidOpenTxnsCounterService.java similarity index 88% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidOpenTxnsCounterService.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidOpenTxnsCounterService.java index fe788004398c..89dbff3f96fb 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidOpenTxnsCounterService.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidOpenTxnsCounterService.java @@ -15,10 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.service; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.MetastoreTaskThread; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,8 +57,8 @@ public void run() { lastLogTime = now; } } - catch (Throwable t) { - LOG.error("Unexpected error in thread: {}, message: {}", Thread.currentThread().getName(), t.getMessage(), t); + catch (Exception e) { + LOG.error("Unexpected exception in thread: {}, message: {}", Thread.currentThread().getName(), e.getMessage(), e); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidTxnCleanerService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidTxnCleanerService.java similarity index 88% rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidTxnCleanerService.java rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidTxnCleanerService.java index d2800bd008bc..06f284faee08 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AcidTxnCleanerService.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/AcidTxnCleanerService.java @@ -15,11 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.metastore.txn; +package org.apache.hadoop.hive.metastore.txn.service; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.MetastoreTaskThread; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,8 +62,8 @@ public void run() { long start = System.currentTimeMillis(); txnHandler.cleanEmptyAbortedAndCommittedTxns(); LOG.debug("Txn cleaner service took: {} seconds.", elapsedSince(start)); - } catch (Throwable t) { - LOG.error("Unexpected error in thread: {}, message: {}", Thread.currentThread().getName(), t.getMessage(), t); + } catch (Exception e) { + LOG.error("Unexpected exception in thread: {}, message: {}", Thread.currentThread().getName(), e.getMessage(), e); } finally { if (handle != null) { handle.releaseLocks(); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/CompactionHouseKeeperService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/CompactionHouseKeeperService.java new file mode 100644 index 000000000000..6eca48283445 --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/service/CompactionHouseKeeperService.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.metastore.txn.service; + +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.function.FailableRunnable; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; + +import java.util.concurrent.TimeUnit; + +/** + * Performs background tasks for Transaction management in Hive. + * Runs inside Hive Metastore Service. + */ +public class CompactionHouseKeeperService extends AcidHouseKeeperService { + + public CompactionHouseKeeperService() { + serviceName = this.getClass().getSimpleName(); + } + + @Override + protected void initTasks(){ + tasks = ImmutableMap., String>builder() + .put(txnHandler::removeDuplicateCompletedTxnComponents, + "Cleaning duplicate COMPLETED_TXN_COMPONENTS entries") + .put(txnHandler::purgeCompactionHistory, "Cleaning obsolete compaction history entries") + .build(); + } + + @Override + public long runFrequency(TimeUnit unit) { + return MetastoreConf.getTimeVar(getConf(), MetastoreConf.ConfVars.COMPACTION_HOUSEKEEPER_SERVICE_INTERVAL, + unit); + } +} diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java index a5a8145ccbd0..b5208c7fe75a 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/conf/TestMetastoreConf.java @@ -20,7 +20,8 @@ import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest; import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.txn.AcidTxnCleanerService; +import org.apache.hadoop.hive.metastore.txn.service.CompactionHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidTxnCleanerService; import org.hamcrest.CoreMatchers; import org.hamcrest.core.StringContains; import org.hamcrest.core.StringEndsWith; @@ -50,8 +51,8 @@ import org.apache.hadoop.hive.metastore.SerDeStorageSchemaReader; import org.apache.hadoop.hive.metastore.events.EventCleanerTask; import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; -import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidOpenTxnsCounterService; @Category(MetastoreUnitTest.class) public class TestMetastoreConf { @@ -500,8 +501,10 @@ public void testClassNames() { EventCleanerTask.class.getName()); Assert.assertEquals(MetastoreConf.METASTORE_DELEGATION_MANAGER_CLASS, MetastoreDelegationTokenManager.class.getName()); - Assert.assertEquals(MetastoreConf.ACID_HOUSE_KEEPER_SERVICE_CLASS, + Assert.assertEquals(MetastoreConf.ACID_HOUSEKEEPER_SERVICE_CLASS, AcidHouseKeeperService.class.getName()); + Assert.assertEquals(MetastoreConf.COMPACTION_HOUSEKEEPER_SERVICE_CLASS, + CompactionHouseKeeperService.class.getName()); Assert.assertEquals(MetastoreConf.ACID_TXN_CLEANER_SERVICE_CLASS, AcidTxnCleanerService.class.getName()); Assert.assertEquals(MetastoreConf.ACID_OPEN_TXNS_COUNTER_SERVICE_CLASS, diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java index 178b68fdfce1..0e27bf393f4f 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.TxnAbortedException; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.service.AcidTxnCleanerService; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.junit.After; import org.junit.Assert; diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java index 16c002cd572e..5ee780005a65 100644 --- a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java +++ b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java @@ -73,7 +73,7 @@ import org.apache.hadoop.hive.metastore.api.TxnInfo; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; +import org.apache.hadoop.hive.metastore.txn.service.AcidHouseKeeperService; import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.metastore.txn.TxnStore; From 87d3d595934f998562ed3bbc525b140a74ffbdd5 Mon Sep 17 00:00:00 2001 From: Dmitriy Fingerman Date: Tue, 13 Feb 2024 09:46:43 -0500 Subject: [PATCH 176/179] HIVE-27980: Iceberg: Compaction: Add support for OPTIMIZE TABLE syntax (Dmitriy Fingerman, reviewed by Attila Turoczy, Ayush Saxena, Butao Zhang) Closes #5028 --- .../iceberg_optimize_table_unpartitioned.q | 58 ++++ ...iceberg_optimize_table_unpartitioned.q.out | 310 ++++++++++++++++++ .../resources/testconfiguration.properties | 3 +- .../hadoop/hive/ql/parse/AlterClauseParser.g | 14 + .../hadoop/hive/ql/parse/HiveLexerParent.g | 1 + .../apache/hadoop/hive/ql/parse/HiveParser.g | 1 + .../hadoop/hive/ql/parse/IdentifiersParser.g | 1 + 7 files changed, 387 insertions(+), 1 deletion(-) create mode 100644 iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q create mode 100644 iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q new file mode 100644 index 000000000000..5fbc108125ed --- /dev/null +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q @@ -0,0 +1,58 @@ +-- SORT_QUERY_RESULTS +-- Mask neededVirtualColumns due to non-strict order +--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/ +-- Mask the totalSize value as it can have slight variability, causing test flakiness +--! qt:replace:/(\s+totalSize\s+)\S+(\s+)/$1#Masked#$2/ +-- Mask random uuid +--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask a random snapshot id +--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/ +-- Mask added file size +--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask total file size +--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ +-- Mask current-snapshot-timestamp-ms +--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/ +-- Mask the enqueue time which is based on current time +--! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ +-- Mask compaction id as they will be allocated in parallel threads +--! qt:replace:/^[0-9]/#Masked#/ + +set hive.llap.io.enabled=true; +set hive.vectorized.execution.enabled=true; +set hive.optimize.shared.work.merge.ts.schema=true; + +create table ice_orc ( + first_name string, + last_name string + ) +stored by iceberg stored as orc +tblproperties ('format-version'='2'); + +insert into ice_orc VALUES ('fn1','ln1'); +insert into ice_orc VALUES ('fn2','ln2'); +insert into ice_orc VALUES ('fn3','ln3'); +insert into ice_orc VALUES ('fn4','ln4'); +insert into ice_orc VALUES ('fn5','ln5'); +insert into ice_orc VALUES ('fn6','ln6'); +insert into ice_orc VALUES ('fn7','ln7'); + +update ice_orc set last_name = 'ln1a' where first_name='fn1'; +update ice_orc set last_name = 'ln2a' where first_name='fn2'; +update ice_orc set last_name = 'ln3a' where first_name='fn3'; +update ice_orc set last_name = 'ln4a' where first_name='fn4'; +update ice_orc set last_name = 'ln5a' where first_name='fn5'; +update ice_orc set last_name = 'ln6a' where first_name='fn6'; +update ice_orc set last_name = 'ln7a' where first_name='fn7'; + +delete from ice_orc where last_name in ('ln5a', 'ln6a', 'ln7a'); + +select * from ice_orc; +describe formatted ice_orc; + +explain optimize table ice_orc rewrite data; +optimize table ice_orc rewrite data; + +select * from ice_orc; +describe formatted ice_orc; +show compactions; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out new file mode 100644 index 000000000000..a4ea671dd05d --- /dev/null +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out @@ -0,0 +1,310 @@ +PREHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice_orc +POSTHOOK: query: create table ice_orc ( + first_name string, + last_name string + ) +stored by iceberg stored as orc +tblproperties ('format-version'='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn1','ln1') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn1','ln1') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn2','ln2') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn2','ln2') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn3','ln3') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn3','ln3') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn4','ln4') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn4','ln4') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn5','ln5') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn5','ln5') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn6','ln6') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn6','ln6') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: insert into ice_orc VALUES ('fn7','ln7') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@ice_orc +POSTHOOK: query: insert into ice_orc VALUES ('fn7','ln7') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln1a' where first_name='fn1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln2a' where first_name='fn2' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln3a' where first_name='fn3' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln4a' where first_name='fn4' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln5a' where first_name='fn5' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln6a' where first_name='fn6' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: update ice_orc set last_name = 'ln7a' where first_name='fn7' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: delete from ice_orc where last_name in ('ln5a', 'ln6a', 'ln7a') +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: delete from ice_orc where last_name in ('ln5a', 'ln6a', 'ln7a') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn1 ln1a +fn2 ln2a +fn3 ln3a +fn4 ln4a +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"1131\",\"changed-partition-count\":\"1\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 11 + numRows 11 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 15 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: explain optimize table ice_orc rewrite data +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: explain optimize table ice_orc rewrite data +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Compact + compaction type: major + table name: default.ice_orc + numberOfBuckets: 0 + table name: default.ice_orc + blocking: true + +PREHOOK: query: optimize table ice_orc rewrite data +PREHOOK: type: ALTERTABLE_COMPACT +PREHOOK: Input: default@ice_orc +PREHOOK: Output: default@ice_orc +POSTHOOK: query: optimize table ice_orc rewrite data +POSTHOOK: type: ALTERTABLE_COMPACT +POSTHOOK: Input: default@ice_orc +POSTHOOK: Output: default@ice_orc +PREHOOK: query: select * from ice_orc +PREHOOK: type: QUERY +PREHOOK: Input: default@ice_orc +#### A masked pattern was here #### +POSTHOOK: query: select * from ice_orc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice_orc +#### A masked pattern was here #### +fn1 ln1a +fn2 ln2a +fn3 ln3a +fn4 ln4a +PREHOOK: query: describe formatted ice_orc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice_orc +POSTHOOK: query: describe formatted ice_orc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice_orc +# col_name data_type comment +first_name string +last_name string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Retention: 0 +#### A masked pattern was here #### +Table Type: EXTERNAL_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"first_name\":\"true\",\"last_name\":\"true\"}} + EXTERNAL TRUE + bucketing_version 2 + current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} + current-snapshot-id #Masked# + current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-timestamp-ms #Masked# + format-version 2 + iceberg.orc.files.only true +#### A masked pattern was here #### + numFiles 1 + numRows 4 + parquet.compression zstd +#### A masked pattern was here #### + rawDataSize 0 + serialization.format 1 + snapshot-count 17 + storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler + table_type ICEBERG + totalSize #Masked# +#### A masked pattern was here #### + uuid #Masked# + write.delete.mode merge-on-read + write.format.default orc + write.merge.mode merge-on-read + write.update.mode merge-on-read + +# Storage Information +SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe +InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat +OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat +Compressed: No +Sort Columns: [] +PREHOOK: query: show compactions +PREHOOK: type: SHOW COMPACTIONS +POSTHOOK: query: show compactions +POSTHOOK: type: SHOW COMPACTIONS +CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId +#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 --- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 65b705d65181..290f4e3acad5 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -421,7 +421,8 @@ iceberg.llap.query.compactor.files=\ iceberg_major_compaction_partition_evolution.q,\ iceberg_major_compaction_partitioned.q,\ iceberg_major_compaction_schema_evolution.q,\ - iceberg_major_compaction_unpartitioned.q + iceberg_major_compaction_unpartitioned.q,\ + iceberg_optimize_table_unpartitioned.q iceberg.llap.only.query.files=\ llap_iceberg_read_orc.q,\ diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g index 8e8ec4e33f99..873a28eb05cd 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/AlterClauseParser.g @@ -52,6 +52,7 @@ alterStatement | KW_ALTER KW_MATERIALIZED KW_VIEW tableNameTree=tableName alterMaterializedViewStatementSuffix[$tableNameTree.tree] -> alterMaterializedViewStatementSuffix | KW_ALTER (KW_DATABASE|KW_SCHEMA) alterDatabaseStatementSuffix -> alterDatabaseStatementSuffix | KW_ALTER KW_DATACONNECTOR alterDataConnectorStatementSuffix -> alterDataConnectorStatementSuffix + | KW_OPTIMIZE KW_TABLE tableName optimizeTableStatementSuffix -> ^(TOK_ALTERTABLE tableName optimizeTableStatementSuffix) ; alterTableStatementSuffix @@ -99,6 +100,19 @@ alterTblPartitionStatementSuffix[boolean partition] | alterStatementSuffixAddCol | alterStatementSuffixUpdateColumns ; + +optimizeTableStatementSuffix +@init { gParent.pushMsg("optimize table statement suffix", state); } +@after { gParent.popMsg(state); } + : optimizeTblRewriteDataSuffix + ; + +optimizeTblRewriteDataSuffix +@init { gParent.msgs.push("compaction request"); } +@after { gParent.msgs.pop(); } + : KW_REWRITE KW_DATA + -> ^(TOK_ALTERTABLE_COMPACT Identifier["'MAJOR'"] TOK_BLOCKING) + ; alterStatementPartitionKeyType @init {gParent.msgs.push("alter partition key type"); } diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g index a26d66d214d6..078cd561b1b4 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveLexerParent.g @@ -402,6 +402,7 @@ KW_CHERRY_PICK: 'CHERRY-PICK'; KW_ORPHAN_FILES: 'ORPHAN-FILES'; KW_OLDER: 'OLDER'; KW_THAN: 'THAN'; +KW_OPTIMIZE: 'OPTIMIZE'; // Operators // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work. diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g index 586ea34af3ed..d9cdac0c0d19 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g @@ -594,6 +594,7 @@ import org.apache.hadoop.hive.conf.HiveConf; xlateMap.put("KW_EXCEPT", "EXCEPT"); xlateMap.put("KW_LOAD", "LOAD"); xlateMap.put("KW_DATA", "DATA"); + xlateMap.put("KW_OPTIMIZE", "OPTIMIZE"); xlateMap.put("KW_INPATH", "INPATH"); xlateMap.put("KW_IS", "IS"); xlateMap.put("KW_NULL", "NULL"); diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g index fc34da643891..37b7c4b0ee00 100644 --- a/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g +++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g @@ -987,6 +987,7 @@ nonReserved | KW_BRANCH | KW_SNAPSHOTS | KW_RETAIN | KW_RETENTION | KW_TAG | KW_FAST_FORWARD + | KW_OPTIMIZE ; //The following SQL2011 reserved keywords are used as function name only, but not as identifiers. From 97c684b67197e719237faddaf4a38523d02da7c7 Mon Sep 17 00:00:00 2001 From: AnmolSun <124231245+AnmolSun@users.noreply.github.com> Date: Wed, 14 Feb 2024 12:23:45 +0530 Subject: [PATCH 177/179] HIVE-28043: Upgrade ZooKeeper to 3.8.3 (#5046). (Anmol Sundaram, reviewed by Ayush Saxena) --- hcatalog/webhcat/svr/pom.xml | 10 ++++++++++ jdbc/pom.xml | 8 ++++++++ kafka-handler/pom.xml | 10 ++++++++++ llap-client/pom.xml | 8 ++++++++ llap-tez/pom.xml | 10 ++++++++++ pom.xml | 10 +++++++++- ql/pom.xml | 10 ++++++++++ shims/common/pom.xml | 8 ++++++++ standalone-metastore/metastore-common/pom.xml | 8 ++++++++ standalone-metastore/pom.xml | 2 +- 10 files changed, 82 insertions(+), 2 deletions(-) diff --git a/hcatalog/webhcat/svr/pom.xml b/hcatalog/webhcat/svr/pom.xml index 4d3df293372a..b4d07b347114 100644 --- a/hcatalog/webhcat/svr/pom.xml +++ b/hcatalog/webhcat/svr/pom.xml @@ -104,6 +104,16 @@ org.apache.zookeeper zookeeper + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + com.fasterxml.jackson.core diff --git a/jdbc/pom.xml b/jdbc/pom.xml index b9080be1898d..5ce447abd73e 100644 --- a/jdbc/pom.xml +++ b/jdbc/pom.xml @@ -82,6 +82,14 @@ org.apache.zookeeper zookeeper + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + io.netty * diff --git a/kafka-handler/pom.xml b/kafka-handler/pom.xml index 43d72065fc6a..007b719960a9 100644 --- a/kafka-handler/pom.xml +++ b/kafka-handler/pom.xml @@ -121,6 +121,16 @@ org.apache.zookeeper zookeeper test + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + io.confluent diff --git a/llap-client/pom.xml b/llap-client/pom.xml index 65e8fa22c7b8..8f18ad4acbd0 100644 --- a/llap-client/pom.xml +++ b/llap-client/pom.xml @@ -55,6 +55,14 @@ org.apache.zookeeper zookeeper + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + log4j log4j diff --git a/llap-tez/pom.xml b/llap-tez/pom.xml index 0c2622dfcbd1..79b665ba8780 100644 --- a/llap-tez/pom.xml +++ b/llap-tez/pom.xml @@ -48,6 +48,16 @@ org.apache.zookeeper zookeeper + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + org.apache.curator diff --git a/pom.xml b/pom.xml index b0644f9b3917..f89fef037b84 100644 --- a/pom.xml +++ b/pom.xml @@ -206,7 +206,7 @@ 2.3 2.12.2 2.3.4 - 3.7.2 + 3.8.3 1.1 2.4.0 5.2.0 @@ -685,6 +685,14 @@ zookeeper ${zookeeper.version} + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + org.slf4j slf4j-log4j12 diff --git a/ql/pom.xml b/ql/pom.xml index ec43088346b5..2b58b165abe5 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -372,6 +372,16 @@ org.apache.zookeeper zookeeper + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + org.apache.curator diff --git a/shims/common/pom.xml b/shims/common/pom.xml index 9f7f0df1f4dc..9d46c03d124d 100644 --- a/shims/common/pom.xml +++ b/shims/common/pom.xml @@ -73,6 +73,14 @@ org.apache.zookeeper zookeeper + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + org.apache.hadoop hadoop-core diff --git a/standalone-metastore/metastore-common/pom.xml b/standalone-metastore/metastore-common/pom.xml index cc8938f7a9cf..8256f15ecbe5 100644 --- a/standalone-metastore/metastore-common/pom.xml +++ b/standalone-metastore/metastore-common/pom.xml @@ -251,6 +251,14 @@ zookeeper ${zookeeper.version} + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + org.slf4j slf4j-log4j12 diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml index 9e9e47197fce..de59ffaf6902 100644 --- a/standalone-metastore/pom.xml +++ b/standalone-metastore/pom.xml @@ -99,7 +99,7 @@ 1.9.4 1.3 5.2.0 - 3.7.2 + 3.8.3 9.1.6 4.0.3 2.8.4 From a3926cc2e2180c359f3884cf0b79f710b147ae09 Mon Sep 17 00:00:00 2001 From: NZEC <29397373+armitage420@users.noreply.github.com> Date: Tue, 20 Feb 2024 10:28:04 +0530 Subject: [PATCH 178/179] HIVE-28065: Upgrade Bouncy castle to bcprov-jdk18on 1.77 (#5071). (Araika Singh, reviewed by Ayush Saxena, Butao Zhang) --- pom.xml | 6 +++--- service/pom.xml | 8 +++++++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index f89fef037b84..be2c431fdd35 100644 --- a/pom.xml +++ b/pom.xml @@ -111,7 +111,7 @@ 12.0.0 1.12.0 1.11.3 - 1.68 + 1.77 1.25.0 5.2.8 5.2.10 @@ -888,8 +888,8 @@ org.bouncycastle - bcprov-jdk15on - ${bcprov-jdk15on.version} + bcprov-jdk18on + ${bcprov-jdk18on.version} org.apache.santuario diff --git a/service/pom.xml b/service/pom.xml index 9d0a9d2f3f47..4a1515809e60 100644 --- a/service/pom.xml +++ b/service/pom.xml @@ -185,10 +185,16 @@ org.pac4j pac4j-saml-opensamlv3 + + + org.bouncycastle + bcprov-jdk15on + + org.bouncycastle - bcprov-jdk15on + bcprov-jdk18on org.apache.santuario From 5b76949da6fe65364a4e3766680871167131157f Mon Sep 17 00:00:00 2001 From: okumin Date: Tue, 20 Feb 2024 14:17:43 +0900 Subject: [PATCH 179/179] HIVE-27950: STACK UDTF returns wrong results when number of arguments is not a multiple of N (#4938) (okumin reviewed by Attila Turoczy, Zsolt Miskolczi and Sourabh Badhya) --- .../hive/ql/udf/generic/GenericUDTFStack.java | 20 +++++++--- .../clientnegative/udtf_stack_not_constant.q | 2 + .../queries/clientnegative/udtf_stack_null.q | 1 + .../clientnegative/udtf_stack_wrong_type1.q | 1 + .../test/queries/clientpositive/udtf_stack.q | 3 ++ .../udtf_stack_not_constant.q.out | 1 + .../clientnegative/udtf_stack_null.q.out | 1 + .../udtf_stack_wrong_type1.q.out | 1 + .../llap/allcolref_in_udf.q.out | 10 ++--- .../clientpositive/llap/udtf_stack.q.out | 39 +++++++++++++++++++ 10 files changed, 68 insertions(+), 11 deletions(-) create mode 100644 ql/src/test/queries/clientnegative/udtf_stack_not_constant.q create mode 100644 ql/src/test/queries/clientnegative/udtf_stack_null.q create mode 100644 ql/src/test/queries/clientnegative/udtf_stack_wrong_type1.q create mode 100644 ql/src/test/results/clientnegative/udtf_stack_not_constant.q.out create mode 100644 ql/src/test/results/clientnegative/udtf_stack_null.q.out create mode 100644 ql/src/test/results/clientnegative/udtf_stack_wrong_type1.q.out diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFStack.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFStack.java index f3cc2b4f6389..d75d3b627144 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFStack.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFStack.java @@ -23,13 +23,13 @@ import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ReturnObjectInspectorResolver; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantIntObjectInspector; import org.apache.hadoop.io.IntWritable; /** @@ -63,13 +63,21 @@ public StructObjectInspector initialize(ObjectInspector[] args) } if (!(args[0] instanceof ConstantObjectInspector)) { throw new UDFArgumentException( + "The first argument to STACK() must be a constant."); + } + final Object value = ((ConstantObjectInspector) args[0]).getWritableConstantValue(); + if (value == null) { + throw new UDFArgumentException("The first argument of STACK() must not be null."); + } + if (!(value instanceof IntWritable)) { + throw new UDFArgumentTypeException( + 0, "The first argument to STACK() must be a constant integer (got " + args[0].getTypeName() + " instead)."); } - numRows = (IntWritable) - ((ConstantObjectInspector)args[0]).getWritableConstantValue(); + numRows = (IntWritable) value; - if (numRows == null || numRows.get() < 1) { + if (numRows.get() < 1) { throw new UDFArgumentException( "STACK() expects its first argument to be >= 1."); } @@ -109,7 +117,7 @@ public StructObjectInspector initialize(ObjectInspector[] args) @Override public void process(Object[] args) - throws HiveException, UDFArgumentException { + throws HiveException { for (int ii = 0; ii < numRows.get(); ++ii) { for (int jj = 0; jj < numCols; ++jj) { int index = ii * numCols + jj + 1; @@ -117,7 +125,7 @@ public void process(Object[] args) forwardObj[jj] = returnOIResolvers.get(jj).convertIfNecessary(args[index], argOIs.get(index)); } else { - forwardObj[ii] = null; + forwardObj[jj] = null; } } forward(forwardObj); diff --git a/ql/src/test/queries/clientnegative/udtf_stack_not_constant.q b/ql/src/test/queries/clientnegative/udtf_stack_not_constant.q new file mode 100644 index 000000000000..f559ce304621 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udtf_stack_not_constant.q @@ -0,0 +1,2 @@ +--! qt:dataset:alltypesparquet +SELECT STACK(cint, 'a', 'b') FROM alltypesparquet; diff --git a/ql/src/test/queries/clientnegative/udtf_stack_null.q b/ql/src/test/queries/clientnegative/udtf_stack_null.q new file mode 100644 index 000000000000..f3832ffd4adb --- /dev/null +++ b/ql/src/test/queries/clientnegative/udtf_stack_null.q @@ -0,0 +1 @@ +SELECT stack(cast(null as int), 'a', 'b', 'c', 'd'); diff --git a/ql/src/test/queries/clientnegative/udtf_stack_wrong_type1.q b/ql/src/test/queries/clientnegative/udtf_stack_wrong_type1.q new file mode 100644 index 000000000000..90762becb234 --- /dev/null +++ b/ql/src/test/queries/clientnegative/udtf_stack_wrong_type1.q @@ -0,0 +1 @@ +SELECT stack('2', 'a', 'b', 'c', 'd'); diff --git a/ql/src/test/queries/clientpositive/udtf_stack.q b/ql/src/test/queries/clientpositive/udtf_stack.q index 8aa7a8cfe80e..def0c38f5d56 100644 --- a/ql/src/test/queries/clientpositive/udtf_stack.q +++ b/ql/src/test/queries/clientpositive/udtf_stack.q @@ -11,3 +11,6 @@ SELECT x, y FROM src LATERAL VIEW STACK(2, 'x', array(1), 'z', array(4)) a AS x, EXPLAIN SELECT stack(1, "en", "dbpedia", NULL ); SELECT stack(1, "en", "dbpedia", NULL ); + +EXPLAIN SELECT STACK(2, 'a', 'b', 'c', 'd', 'e'); +SELECT STACK(2, 'a', 'b', 'c', 'd', 'e'); diff --git a/ql/src/test/results/clientnegative/udtf_stack_not_constant.q.out b/ql/src/test/results/clientnegative/udtf_stack_not_constant.q.out new file mode 100644 index 000000000000..86b264a179de --- /dev/null +++ b/ql/src/test/results/clientnegative/udtf_stack_not_constant.q.out @@ -0,0 +1 @@ +FAILED: UDFArgumentException The first argument to STACK() must be a constant. diff --git a/ql/src/test/results/clientnegative/udtf_stack_null.q.out b/ql/src/test/results/clientnegative/udtf_stack_null.q.out new file mode 100644 index 000000000000..854a2361c3a9 --- /dev/null +++ b/ql/src/test/results/clientnegative/udtf_stack_null.q.out @@ -0,0 +1 @@ +FAILED: UDFArgumentException The first argument of STACK() must not be null. diff --git a/ql/src/test/results/clientnegative/udtf_stack_wrong_type1.q.out b/ql/src/test/results/clientnegative/udtf_stack_wrong_type1.q.out new file mode 100644 index 000000000000..d18a726b9c17 --- /dev/null +++ b/ql/src/test/results/clientnegative/udtf_stack_wrong_type1.q.out @@ -0,0 +1 @@ +FAILED: UDFArgumentTypeException The first argument to STACK() must be a constant integer (got string instead). diff --git a/ql/src/test/results/clientpositive/llap/allcolref_in_udf.q.out b/ql/src/test/results/clientpositive/llap/allcolref_in_udf.q.out index 8e20aaa35d86..236d52a2ded5 100644 --- a/ql/src/test/results/clientpositive/llap/allcolref_in_udf.q.out +++ b/ql/src/test/results/clientpositive/llap/allcolref_in_udf.q.out @@ -168,15 +168,15 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@src #### A masked pattern was here #### 4val_45val_5 4val_4 5val_5 -4val_45 NULL 5val_5 +4val_45 45val_5 NULL 4val_45val_5 4val_4 5val_5 -4val_45 NULL 5val_5 +4val_45 45val_5 NULL 4val_45val_5 4val_4 5val_5 -4val_45 NULL 5val_5 +4val_45 45val_5 NULL 8val_89val_9 8val_8 9val_9 -8val_89 NULL 9val_9 +8val_89 89val_9 NULL 9val_910val_10 9val_9 10val_10 -9val_910 NULL 10val_10 +9val_910 910val_10 NULL PREHOOK: query: create table allcolref as select array(key, value) from src PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src diff --git a/ql/src/test/results/clientpositive/llap/udtf_stack.q.out b/ql/src/test/results/clientpositive/llap/udtf_stack.q.out index 2e5bba2e388c..ad176cea611c 100644 --- a/ql/src/test/results/clientpositive/llap/udtf_stack.q.out +++ b/ql/src/test/results/clientpositive/llap/udtf_stack.q.out @@ -147,3 +147,42 @@ POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table #### A masked pattern was here #### en dbpedia NULL +PREHOOK: query: EXPLAIN SELECT STACK(2, 'a', 'b', 'c', 'd', 'e') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: EXPLAIN SELECT STACK(2, 'a', 'b', 'c', 'd', 'e') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + TableScan + alias: _dummy_table + Row Limit Per Split: 1 + Select Operator + expressions: 2 (type: int), 'a' (type: string), 'b' (type: string), 'c' (type: string), 'd' (type: string), 'e' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + UDTF Operator + function name: stack + Select Operator + expressions: col0 (type: string), col1 (type: string), col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + ListSink + +PREHOOK: query: SELECT STACK(2, 'a', 'b', 'c', 'd', 'e') +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +POSTHOOK: query: SELECT STACK(2, 'a', 'b', 'c', 'd', 'e') +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +#### A masked pattern was here #### +a b c +d e NULL